();
-
- public String getState(String key) {
- return (String) state.get(key).get();
- }
-
- public void setState(String key, String msg) {
- state.put(key, msg);
- }
-
- public void clash() {
- state.put("clasher", "was here");
- }
-}
-*/
diff --git a/akka-active-object-test/src/test/java/se/scalablesolutions/akka/api/MiscActiveObjectTest.java b/akka-active-object-test/src/test/java/se/scalablesolutions/akka/api/MiscActiveObjectTest.java
deleted file mode 100644
index 6ab6d2ff03..0000000000
--- a/akka-active-object-test/src/test/java/se/scalablesolutions/akka/api/MiscActiveObjectTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-package se.scalablesolutions.akka.api;
-
-import static se.scalablesolutions.akka.actor.TypedActor.link;
-import static se.scalablesolutions.akka.actor.TypedActor.newInstance;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import se.scalablesolutions.akka.config.OneForOneStrategy;
-import junit.framework.TestCase;
-
-/**
- * Small misc tests that do not fit anywhere else and does not require a separate testcase
- *
- * @author johanrask
- *
- */
-public class MiscTypedActorTest extends TestCase {
-
-
- /**
- * Verifies that both preRestart and postRestart methods are invoked when
- * an actor is restarted
- */
- public void testFailingPostRestartInvocation() throws InterruptedException {
- SimpleJavaPojo pojo = newInstance(SimpleJavaPojo.class,500);
- SimpleJavaPojo supervisor = newInstance(SimpleJavaPojo.class,500);
- link(supervisor,pojo,new OneForOneStrategy(3, 2000),new Class[]{Throwable.class});
- pojo.throwException();
- Thread.sleep(500);
- Assert.assertTrue(pojo.pre);
- Assert.assertTrue(pojo.post);
- }
-
-}
diff --git a/akka-active-object-test/src/test/java/se/scalablesolutions/akka/api/RemoteInMemoryStateTest.java b/akka-active-object-test/src/test/java/se/scalablesolutions/akka/api/RemoteInMemoryStateTest.java
deleted file mode 100644
index 89f7e92162..0000000000
--- a/akka-active-object-test/src/test/java/se/scalablesolutions/akka/api/RemoteInMemoryStateTest.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/**
- * Copyright (C) 2009-2010 Scalable Solutions AB
- */
-
-package se.scalablesolutions.akka.api;
-
-import se.scalablesolutions.akka.config.Config;
-import se.scalablesolutions.akka.actor.TypedActor;
-import se.scalablesolutions.akka.config.TypedActorConfigurator;
-import se.scalablesolutions.akka.remote.RemoteNode;
-
-import junit.framework.TestCase;
-
-public class RemoteInMemoryStateTest extends TestCase {
- static String messageLog = "";
-
- static {
- new Thread(new Runnable() {
- public void run() {
- RemoteNode.start();
- }
- }).start();
- try { Thread.currentThread().sleep(1000); } catch (Exception e) {}
- Config.config();
- }
- final TypedActorConfigurator conf = new TypedActorConfigurator();
-
- protected void tearDown() {
- conf.stop();
- }
-
- public void testMapShouldNotRollbackStateForStatefulServerInCaseOfSuccess() {
- InMemStateful stateful = TypedActor.newRemoteInstance(InMemStateful.class, 1000, "localhost", 9999);
- stateful.init();
- stateful.setMapState("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess", "init"); // set init state
- stateful.success("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess", "new state"); // transactionrequired
- assertEquals("new state", stateful.getMapState("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess"));
- }
-
- public void testMapShouldRollbackStateForStatefulServerInCaseOfFailure() {
- InMemStateful stateful = TypedActor.newRemoteInstance(InMemStateful.class, 10000, "localhost", 9999);
- stateful.init();
- stateful.setMapState("testShouldRollbackStateForStatefulServerInCaseOfFailure", "init"); // set init state
- InMemFailer failer = TypedActor.newRemoteInstance(InMemFailer.class, 1000, "localhost", 9999); //conf.getInstance(InMemFailer.class);
- try {
- stateful.failure("testShouldRollbackStateForStatefulServerInCaseOfFailure", "new state", failer); // call failing transactionrequired method
- fail("should have thrown an exception");
- } catch (RuntimeException e) {
- } // expected
- assertEquals("init", stateful.getMapState("testShouldRollbackStateForStatefulServerInCaseOfFailure")); // check that state is == init state
- }
-
- public void testVectorShouldNotRollbackStateForStatefulServerInCaseOfSuccess() {
- InMemStateful stateful = TypedActor.newRemoteInstance(InMemStateful.class, 10000, "localhost", 9999);
- stateful.init();
- stateful.setVectorState("init"); // set init state
- stateful.success("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess", "new state"); // transactionrequired
- assertEquals("new state", stateful.getVectorState());
- }
-
- public void testVectorShouldRollbackStateForStatefulServerInCaseOfFailure() {
- InMemStateful stateful = TypedActor.newRemoteInstance(InMemStateful.class, 10000, "localhost", 9999);
- stateful.init();
- stateful.setVectorState("init"); // set init state
- InMemFailer failer = TypedActor.newRemoteInstance(InMemFailer.class, 10000, "localhost", 9999); //conf.getInstance(InMemFailer.class);
- try {
- stateful.failure("testShouldRollbackStateForStatefulServerInCaseOfFailure", "new state", failer); // call failing transactionrequired method
- fail("should have thrown an exception");
- } catch (RuntimeException e) {
- } // expected
- assertEquals("init", stateful.getVectorState()); // check that state is == init state
- }
-
- public void testRefShouldNotRollbackStateForStatefulServerInCaseOfSuccess() {
- InMemStateful stateful = TypedActor.newRemoteInstance(InMemStateful.class, 10000, "localhost", 9999);
- stateful.init();
- stateful.setRefState("init"); // set init state
- stateful.success("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess", "new state"); // transactionrequired
- assertEquals("new state", stateful.getRefState());
- }
-
- public void testRefShouldRollbackStateForStatefulServerInCaseOfFailure() {
- InMemStateful stateful = TypedActor.newRemoteInstance(InMemStateful.class, 10000, "localhost", 9999);
- stateful.init();
- stateful.setRefState("init"); // set init state
- InMemFailer failer = TypedActor.newRemoteInstance(InMemFailer.class, 10000, "localhost", 9999); //conf.getInstance(InMemFailer.class);
- try {
- stateful.failure("testShouldRollbackStateForStatefulServerInCaseOfFailure", "new state", failer); // call failing transactionrequired method
- fail("should have thrown an exception");
- } catch (RuntimeException e) {
- } // expected
- assertEquals("init", stateful.getRefState()); // check that state is == init state
- }
- /*
- public void testNestedNonTransactionalMethodHangs() {
- InMemStateful stateful = conf.getInstance(InMemStateful.class);
- stateful.setMapState("testShouldRollbackStateForStatefulServerInCaseOfFailure", "init"); // set init state
- InMemFailer failer = conf.getInstance(InMemFailer.class);
- try {
- stateful.thisMethodHangs("testShouldRollbackStateForStatefulServerInCaseOfFailure", "new state", failer); // call failing transactionrequired method
- fail("should have thrown an exception");
- } catch (RuntimeException e) {
- } // expected
- assertEquals("init", stateful.getMapState("testShouldRollbackStateForStatefulServerInCaseOfFailure")); // check that state is == init state
- }
- */
- // public void testShouldRollbackStateForStatefulServerInCaseOfMessageClash()
- // {
- // InMemStateful stateful = conf.getInstance(InMemStateful.class);
- // stateful.setState("stateful", "init"); // set init state
- //
- // InMemClasher clasher = conf.getInstance(InMemClasher.class);
- // clasher.setState("clasher", "init"); // set init state
- //
- // // try {
- // // stateful.clashOk("stateful", "new state", clasher);
- // // } catch (RuntimeException e) { } // expected
- // // assertEquals("new state", stateful.getState("stateful")); // check that
- // // state is == init state
- // // assertEquals("was here", clasher.getState("clasher")); // check that
- // // state is == init state
- //
- // try {
- // stateful.clashNotOk("stateful", "new state", clasher);
- // fail("should have thrown an exception");
- // } catch (RuntimeException e) {
- // } // expected
- // assertEquals("init", stateful.getState("stateful")); // check that state is
- // // == init state
- // // assertEquals("init", clasher.getState("clasher")); // check that state
- // is
- // // == init state
- // }
-}
diff --git a/akka-active-object-test/src/test/java/se/scalablesolutions/akka/api/SimpleJavaPojo.java b/akka-active-object-test/src/test/java/se/scalablesolutions/akka/api/SimpleJavaPojo.java
deleted file mode 100644
index f2f05842e9..0000000000
--- a/akka-active-object-test/src/test/java/se/scalablesolutions/akka/api/SimpleJavaPojo.java
+++ /dev/null
@@ -1,36 +0,0 @@
-package se.scalablesolutions.akka.api;
-
-import se.scalablesolutions.akka.actor.annotation.prerestart;
-import se.scalablesolutions.akka.actor.annotation.postrestart;
-
-public class SimpleJavaPojo {
-
- public boolean pre = false;
- public boolean post = false;
-
- private String name;
-
- public void setName(String name) {
- this.name = name;
- }
-
- public String getName() {
- return name;
- }
-
- @prerestart
- public void pre() {
- System.out.println("** pre()");
- pre = true;
- }
-
- @postrestart
- public void post() {
- System.out.println("** post()");
- post = true;
- }
-
- public void throwException() {
- throw new RuntimeException();
- }
-}
diff --git a/akka-active-object-test/src/test/java/se/scalablesolutions/akka/api/TypedActorGuiceConfiguratorTest.java b/akka-active-object-test/src/test/java/se/scalablesolutions/akka/api/TypedActorGuiceConfiguratorTest.java
deleted file mode 100644
index e604b4da69..0000000000
--- a/akka-active-object-test/src/test/java/se/scalablesolutions/akka/api/TypedActorGuiceConfiguratorTest.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/**
- * Copyright (C) 2009-2010 Scalable Solutions AB
- */
-
-package se.scalablesolutions.akka.api;
-
-import com.google.inject.AbstractModule;
-import com.google.inject.Scopes;
-
-import junit.framework.TestCase;
-
-import se.scalablesolutions.akka.config.Config;
-import se.scalablesolutions.akka.config.TypedActorConfigurator;
-import static se.scalablesolutions.akka.config.JavaConfig.*;
-import se.scalablesolutions.akka.dispatch.*;
-
-public class TypedActorGuiceConfiguratorTest extends TestCase {
- static String messageLog = "";
-
- final private TypedActorConfigurator conf = new TypedActorConfigurator();
-
- protected void setUp() {
- Config.config();
- MessageDispatcher dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher("test");
-
- conf.addExternalGuiceModule(new AbstractModule() {
- protected void configure() {
- bind(Ext.class).to(ExtImpl.class).in(Scopes.SINGLETON);
- }
- }).configure(
- new RestartStrategy(new AllForOne(), 3, 5000, new Class[]{Exception.class}),
- new Component[]{
- new Component(
- Foo.class,
- new LifeCycle(new Permanent()),
- 1000,
- dispatcher),
- //new RemoteAddress("localhost", 9999)),
- new Component(
- Bar.class,
- BarImpl.class,
- new LifeCycle(new Permanent()),
- 1000,
- dispatcher)
- }).inject().supervise();
-
- }
-
- public void testGuiceTypedActorInjection() {
- messageLog = "";
- Foo foo = conf.getInstance(Foo.class);
- Bar bar = conf.getInstance(Bar.class);
- assertEquals(foo.getBar(), bar);
- }
-
- public void testGuiceExternalDependencyInjection() {
- messageLog = "";
- Bar bar = conf.getInstance(Bar.class);
- Ext ext = conf.getExternalDependency(Ext.class);
- assertTrue(bar.getExt().toString().equals(ext.toString()));
- }
-
- public void testLookupNonSupervisedInstance() {
- try {
- String str = conf.getInstance(String.class);
- fail("exception should have been thrown");
- } catch (Exception e) {
- assertEquals(IllegalStateException.class, e.getClass());
- }
- }
-
- public void testTypedActorInvocation() throws InterruptedException {
- messageLog = "";
- Foo foo = conf.getInstance(Foo.class);
- messageLog += foo.foo("foo ");
- foo.bar("bar ");
- messageLog += "before_bar ";
- Thread.sleep(500);
- assertEquals("foo return_foo before_bar ", messageLog);
- }
-
- public void testTypedActorInvocationsInvocation() throws InterruptedException {
- messageLog = "";
- Foo foo = conf.getInstance(Foo.class);
- Bar bar = conf.getInstance(Bar.class);
- messageLog += foo.foo("foo ");
- foo.bar("bar ");
- messageLog += "before_bar ";
- Thread.sleep(500);
- assertEquals("foo return_foo before_bar ", messageLog);
- }
-
-
- public void testForcedTimeout() {
- messageLog = "";
- Foo foo = conf.getInstance(Foo.class);
- try {
- foo.longRunning();
- fail("exception should have been thrown");
- } catch (se.scalablesolutions.akka.dispatch.FutureTimeoutException e) {
- }
- }
-
- public void testForcedException() {
- messageLog = "";
- Foo foo = conf.getInstance(Foo.class);
- try {
- foo.throwsException();
- fail("exception should have been thrown");
- } catch (RuntimeException e) {
- }
- }
-}
-
-
diff --git a/akka-core/src/main/resources/logback.xml b/akka-actor/src/main/resources/logback.xml
similarity index 100%
rename from akka-core/src/main/resources/logback.xml
rename to akka-actor/src/main/resources/logback.xml
diff --git a/akka-core/src/main/scala/actor/Actor.scala b/akka-actor/src/main/scala/actor/Actor.scala
similarity index 97%
rename from akka-core/src/main/scala/actor/Actor.scala
rename to akka-actor/src/main/scala/actor/Actor.scala
index 2435598c8c..6a5365f3fb 100644
--- a/akka-core/src/main/scala/actor/Actor.scala
+++ b/akka-actor/src/main/scala/actor/Actor.scala
@@ -7,7 +7,6 @@ package se.scalablesolutions.akka.actor
import se.scalablesolutions.akka.dispatch._
import se.scalablesolutions.akka.config.Config._
import se.scalablesolutions.akka.config.ScalaConfig._
-import se.scalablesolutions.akka.serialization.Serializer
import se.scalablesolutions.akka.util.Helpers.{narrow, narrowSilently}
import se.scalablesolutions.akka.util.{Logging, Duration}
import se.scalablesolutions.akka.AkkaException
@@ -502,3 +501,23 @@ private[actor] class AnyOptionAsTypedOption(anyOption: Option[Any]) {
*/
def asSilently[T: Manifest]: Option[T] = narrowSilently[T](anyOption)
}
+
+/**
+ * Marker interface for proxyable actors (such as typed actor).
+ *
+ * @author Jonas Bonér
+ */
+trait Proxyable {
+ private[actor] def swapProxiedActor(newInstance: Actor)
+}
+
+/**
+ * Represents the different Actor types.
+ *
+ * @author Jonas Bonér
+ */
+sealed trait ActorType
+object ActorType {
+ case object ScalaActor extends ActorType
+ case object TypedActor extends ActorType
+}
diff --git a/akka-core/src/main/scala/actor/ActorRef.scala b/akka-actor/src/main/scala/actor/ActorRef.scala
similarity index 93%
rename from akka-core/src/main/scala/actor/ActorRef.scala
rename to akka-actor/src/main/scala/actor/ActorRef.scala
index 0cf81083f4..2f17e830a9 100644
--- a/akka-core/src/main/scala/actor/ActorRef.scala
+++ b/akka-actor/src/main/scala/actor/ActorRef.scala
@@ -5,18 +5,15 @@
package se.scalablesolutions.akka.actor
import se.scalablesolutions.akka.dispatch._
-import se.scalablesolutions.akka.config.Config.config
+import se.scalablesolutions.akka.config.Config._
import se.scalablesolutions.akka.config.{AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy}
import se.scalablesolutions.akka.config.ScalaConfig._
import se.scalablesolutions.akka.stm.global._
import se.scalablesolutions.akka.stm.TransactionManagement._
import se.scalablesolutions.akka.stm.{TransactionManagement, TransactionSetAbortedException}
-import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._
-import se.scalablesolutions.akka.remote.{RemoteNode, RemoteServer, RemoteClient, MessageSerializer, RemoteRequestProtocolIdFactory}
-import se.scalablesolutions.akka.serialization.{Serializer, BinaryString}
-import se.scalablesolutions.akka.util.{HashCode, Logging, UUID, ReentrantGuard}
import se.scalablesolutions.akka.AkkaException
-import RemoteActorSerialization._
+import se.scalablesolutions.akka.util._
+import ReflectiveAccess._
import org.multiverse.api.ThreadLocalTransaction._
import org.multiverse.commitbarriers.CountDownCommitBarrier
@@ -33,8 +30,6 @@ import java.lang.reflect.Field
import scala.reflect.BeanProperty
-import com.google.protobuf.ByteString
-
/**
* ActorRef is an immutable and serializable handle to an Actor.
*
@@ -67,15 +62,18 @@ import com.google.protobuf.ByteString
*
* @author Jonas Bonér
*/
-trait ActorRef extends ActorRefShared with TransactionManagement with java.lang.Comparable[ActorRef] {
- scalaRef: ScalaActorRef =>
+trait ActorRef extends
+ ActorRefShared with
+ TransactionManagement with
+ Logging with
+ java.lang.Comparable[ActorRef] { scalaRef: ScalaActorRef =>
// Only mutable for RemoteServer in order to maintain identity across nodes
@volatile protected[akka] var _uuid = UUID.newUuid.toString
@volatile protected[this] var _isRunning = false
@volatile protected[this] var _isShutDown = false
@volatile protected[akka] var _isBeingRestarted = false
- @volatile protected[akka] var _homeAddress = new InetSocketAddress(RemoteServer.HOSTNAME, RemoteServer.PORT)
+ @volatile protected[akka] var _homeAddress = new InetSocketAddress(RemoteServerModule.HOSTNAME, RemoteServerModule.PORT)
@volatile protected[akka] var _futureTimeout: Option[ScheduledFuture[AnyRef]] = None
@volatile protected[akka] var startOnCreation = false
@volatile protected[akka] var registeredInRemoteNodeDuringSerialization = false
@@ -681,13 +679,11 @@ class LocalActorRef private[akka](
if (runActorInitialization && !isDeserialized) initializeActorInstance
private[akka] def this(clazz: Class[_ <: Actor]) = this(Left(Some(clazz)))
- private[akka] def this(factory: () => Actor) = this(Right(Some(factory)))
+ private[akka] def this(factory: () => Actor) = this(Right(Some(factory)))
// used only for deserialization
private[akka] def this(__uuid: String,
__id: String,
- __actorClassName: String,
- __actorBytes: Array[Byte],
__hostname: String,
__port: Int,
__isTransactor: Boolean,
@@ -697,16 +693,8 @@ class LocalActorRef private[akka](
__supervisor: Option[ActorRef],
__hotswap: Option[PartialFunction[Any, Unit]],
__loader: ClassLoader,
- __messages: List[RemoteRequestProtocol],
- __format: Format[_ <: Actor]) = {
- this(() => {
- val actorClass = __loader.loadClass(__actorClassName)
- if (__format.isInstanceOf[SerializerBasedActorFormat[_]])
- __format.asInstanceOf[SerializerBasedActorFormat[_]]
- .serializer
- .fromBinary(__actorBytes, Some(actorClass)).asInstanceOf[Actor]
- else actorClass.newInstance.asInstanceOf[Actor]
- })
+ __factory: () => Actor) = {
+ this(__factory)
loader = Some(__loader)
isDeserialized = true
_uuid = __uuid
@@ -721,7 +709,6 @@ class LocalActorRef private[akka](
actorSelfFields._1.set(actor, this)
actorSelfFields._2.set(actor, Some(this))
start
- __messages.foreach(message => this ! MessageSerializer.deserialize(message.getMessage))
checkReceiveTimeout
ActorRegistry.register(this)
}
@@ -741,10 +728,12 @@ class LocalActorRef private[akka](
/**
* Sets the dispatcher for this actor. Needs to be invoked before the actor is started.
*/
- def dispatcher_=(md: MessageDispatcher): Unit = {
- if (!isRunning || isBeingRestarted) _dispatcher = md
- else throw new ActorInitializationException(
+ def dispatcher_=(md: MessageDispatcher): Unit = guard.withGuard {
+ if (!isBeingRestarted) {
+ if (!isRunning) _dispatcher = md
+ else throw new ActorInitializationException(
"Can not swap dispatcher for " + toString + " after it has been started")
+ }
}
/**
@@ -755,19 +744,22 @@ class LocalActorRef private[akka](
/**
* Invoking 'makeRemote' means that an actor will be moved to and invoked on a remote host.
*/
- def makeRemote(hostname: String, port: Int): Unit =
+ def makeRemote(hostname: String, port: Int): Unit = {
+ ensureRemotingEnabled
if (!isRunning || isBeingRestarted) makeRemote(new InetSocketAddress(hostname, port))
else throw new ActorInitializationException(
"Can't make a running actor remote. Make sure you call 'makeRemote' before 'start'.")
+ }
/**
* Invoking 'makeRemote' means that an actor will be moved to and invoked on a remote host.
*/
def makeRemote(address: InetSocketAddress): Unit = guard.withGuard {
+ ensureRemotingEnabled
if (!isRunning || isBeingRestarted) {
_remoteAddress = Some(address)
- RemoteClient.register(address.getHostName, address.getPort, uuid)
- homeAddress = (RemoteServer.HOSTNAME, RemoteServer.PORT)
+ RemoteClientModule.register(address, uuid)
+ homeAddress = (RemoteServerModule.HOSTNAME, RemoteServerModule.PORT)
} else throw new ActorInitializationException(
"Can't make a running actor remote. Make sure you call 'makeRemote' before 'start'.")
}
@@ -839,9 +831,12 @@ class LocalActorRef private[akka](
_isShutDown = true
actor.shutdown
ActorRegistry.unregister(this)
- remoteAddress.foreach(address => RemoteClient.unregister(
- address.getHostName, address.getPort, uuid))
- RemoteNode.unregister(this)
+ if (isRemotingEnabled) {
+ remoteAddress.foreach { address =>
+ RemoteClientModule.unregister(address, uuid)
+ }
+ RemoteServerModule.unregister(this)
+ }
nullOutActorRefReferencesFor(actorInstance.get)
} //else if (isBeingRestarted) throw new ActorKilledException("Actor [" + toString + "] is being restarted.")
}
@@ -896,6 +891,7 @@ class LocalActorRef private[akka](
* To be invoked from within the actor itself.
*/
def startLinkRemote(actorRef: ActorRef, hostname: String, port: Int) = guard.withGuard {
+ ensureRemotingEnabled
try {
actorRef.makeRemote(hostname, port)
actorRef.start
@@ -921,6 +917,7 @@ class LocalActorRef private[akka](
* To be invoked from within the actor itself.
*/
def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = guard.withGuard {
+ ensureRemotingEnabled
val actor = spawnButDoNotStart(clazz)
actor.makeRemote(hostname, port)
actor.start
@@ -948,6 +945,7 @@ class LocalActorRef private[akka](
* To be invoked from within the actor itself.
*/
def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = guard.withGuard {
+ ensureRemotingEnabled
val actor = spawnButDoNotStart(clazz)
try {
actor.makeRemote(hostname, port)
@@ -984,9 +982,9 @@ class LocalActorRef private[akka](
protected[akka] def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit = {
joinTransaction(message)
- if (remoteAddress.isDefined) {
- RemoteClient.clientFor(remoteAddress.get).send[Any](
- createRemoteRequestProtocolBuilder(this, message, true, senderOption).build, None)
+ if (isRemotingEnabled && remoteAddress.isDefined) {
+ RemoteClientModule.send[Any](
+ message, senderOption, None, remoteAddress.get, timeout, true, this, None, ActorType.ScalaActor)
} else {
val invocation = new MessageInvocation(this, message, senderOption, None, transactionSet.get)
invocation.send
@@ -1000,9 +998,9 @@ class LocalActorRef private[akka](
senderFuture: Option[CompletableFuture[T]]): CompletableFuture[T] = {
joinTransaction(message)
- if (remoteAddress.isDefined) {
- val future = RemoteClient.clientFor(remoteAddress.get).send(
- createRemoteRequestProtocolBuilder(this, message, false, senderOption).build, senderFuture)
+ if (isRemotingEnabled && remoteAddress.isDefined) {
+ val future = RemoteClientModule.send[T](
+ message, senderOption, senderFuture, remoteAddress.get, timeout, false, this, None, ActorType.ScalaActor)
if (future.isDefined) future.get
else throw new IllegalActorStateException("Expected a future from remote call to actor " + toString)
} else {
@@ -1088,7 +1086,7 @@ class LocalActorRef private[akka](
Actor.log.debug("Restarting linked actors for actor [%s].", id)
restartLinkedActors(reason, maxNrOfRetries, withinTimeRange)
Actor.log.debug("Invoking 'preRestart' for failed actor instance [%s].", id)
- if (isTypedActorDispatcher(failedActor)) restartTypedActorDispatcher(failedActor, reason)
+ if (isProxyableDispatcher(failedActor)) restartProxyableDispatcher(failedActor, reason)
else restartActor(failedActor, reason)
_isBeingRestarted = false
}
@@ -1107,8 +1105,9 @@ class LocalActorRef private[akka](
}
protected[akka] def registerSupervisorAsRemoteActor: Option[String] = guard.withGuard {
+ ensureRemotingEnabled
if (_supervisor.isDefined) {
- RemoteClient.clientFor(remoteAddress.get).registerSupervisorForActor(this)
+ remoteAddress.foreach(address => RemoteClientModule.registerSupervisorForActor(address, this))
Some(_supervisor.get.uuid)
} else None
}
@@ -1126,9 +1125,9 @@ class LocalActorRef private[akka](
// ========= PRIVATE FUNCTIONS =========
- private def isTypedActorDispatcher(a: Actor): Boolean = a.isInstanceOf[TypedActor]
+ private def isProxyableDispatcher(a: Actor): Boolean = a.isInstanceOf[Proxyable]
- private def restartTypedActorDispatcher(failedActor: Actor, reason: Throwable) = {
+ private def restartProxyableDispatcher(failedActor: Actor, reason: Throwable) = {
failedActor.preRestart(reason)
failedActor.postRestart(reason)
}
@@ -1140,8 +1139,8 @@ class LocalActorRef private[akka](
freshActor.init
freshActor.initTransactionalState
actorInstance.set(freshActor)
- if (failedActor.isInstanceOf[TypedActor]) failedActor.asInstanceOf[TypedActor].swapInstanceInProxy(freshActor)
- if (dispatcher.isShutdown) dispatcher.start
+ if (failedActor.isInstanceOf[Proxyable])
+ failedActor.asInstanceOf[Proxyable].swapProxiedActor(freshActor)
Actor.log.debug("Invoking 'postRestart' for new actor instance [%s].", id)
freshActor.postRestart(reason)
}
@@ -1317,6 +1316,7 @@ class LocalActorRef private[akka](
checkReceiveTimeout
}
+/*
private def serializeMessage(message: AnyRef): AnyRef = if (Actor.SERIALIZE_MESSAGES) {
if (!message.isInstanceOf[String] &&
!message.isInstanceOf[Byte] &&
@@ -1340,6 +1340,7 @@ class LocalActorRef private[akka](
Serializer.Java.deepClone(message)
} else message
} else message
+ */
}
/**
@@ -1348,7 +1349,7 @@ class LocalActorRef private[akka](
* @author Jonas Bonér
*/
object RemoteActorSystemMessage {
- val Stop = BinaryString("RemoteActorRef:stop")
+ val Stop = "RemoteActorRef:stop".intern
}
/**
@@ -1358,26 +1359,33 @@ object RemoteActorSystemMessage {
* @author Jonas Bonér
*/
private[akka] case class RemoteActorRef private[akka] (
- uuuid: String, val className: String, val hostname: String, val port: Int, _timeout: Long, loader: Option[ClassLoader])
- // uuid: String, className: String, hostname: String, port: Int, timeOut: Long, isOnRemoteHost: Boolean) extends ActorRef {
+ uuuid: String,
+ val className: String,
+ val hostname: String,
+ val port: Int,
+ _timeout: Long,
+ loader: Option[ClassLoader])
extends ActorRef with ScalaActorRef {
+ ensureRemotingEnabled
+
_uuid = uuuid
timeout = _timeout
start
- lazy val remoteClient = RemoteClient.clientFor(hostname, port, loader)
+ lazy val remoteClient = RemoteClientModule.clientFor(hostname, port, loader)
- def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit = {
- remoteClient.send[Any](createRemoteRequestProtocolBuilder(this, message, true, senderOption).build, None)
- }
+ def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit =
+ RemoteClientModule.send[Any](
+ message, senderOption, None, remoteAddress.get, timeout, true, this, None, ActorType.ScalaActor)
def postMessageToMailboxAndCreateFutureResultWithTimeout[T](
message: Any,
timeout: Long,
senderOption: Option[ActorRef],
senderFuture: Option[CompletableFuture[T]]): CompletableFuture[T] = {
- val future = remoteClient.send(createRemoteRequestProtocolBuilder(this, message, false, senderOption).build, senderFuture)
+ val future = RemoteClientModule.send[T](
+ message, senderOption, senderFuture, remoteAddress.get, timeout, false, this, None, ActorType.ScalaActor)
if (future.isDefined) future.get
else throw new IllegalActorStateException("Expected a future from remote call to actor " + toString)
}
@@ -1400,6 +1408,8 @@ private[akka] case class RemoteActorRef private[akka] (
protected[akka] def registerSupervisorAsRemoteActor: Option[String] = None
+ val remoteAddress: Option[InetSocketAddress] = Some(new InetSocketAddress(hostname, port))
+
// ==== NOT SUPPORTED ====
def actorClass: Class[_ <: Actor] = unsupported
def dispatcher_=(md: MessageDispatcher): Unit = unsupported
@@ -1410,7 +1420,6 @@ private[akka] case class RemoteActorRef private[akka] (
def makeRemote(hostname: String, port: Int): Unit = unsupported
def makeRemote(address: InetSocketAddress): Unit = unsupported
def homeAddress_=(address: InetSocketAddress): Unit = unsupported
- def remoteAddress: Option[InetSocketAddress] = unsupported
def link(actorRef: ActorRef): Unit = unsupported
def unlink(actorRef: ActorRef): Unit = unsupported
def startLink(actorRef: ActorRef): Unit = unsupported
@@ -1463,6 +1472,7 @@ trait ActorRefShared {
* from ActorRef -> ScalaActorRef and back
*/
trait ScalaActorRef extends ActorRefShared { ref: ActorRef =>
+
/**
* Identifier for actor, does not have to be a unique one. Default is the 'uuid'.
*
@@ -1472,7 +1482,8 @@ trait ScalaActorRef extends ActorRefShared { ref: ActorRef =>
* upon restart, remote restart etc.
*/
def id: String
- def id_=(id: String):Unit
+
+ def id_=(id: String): Unit
/**
* User overridable callback/setting.
@@ -1532,7 +1543,7 @@ trait ScalaActorRef extends ActorRefShared { ref: ActorRef =>
def sender: Option[ActorRef] = {
// Five lines of map-performance-avoidance, could be just: currentMessage map { _.sender }
val msg = currentMessage
- if(msg.isEmpty) None
+ if (msg.isEmpty) None
else msg.get.sender
}
@@ -1543,7 +1554,7 @@ trait ScalaActorRef extends ActorRefShared { ref: ActorRef =>
def senderFuture(): Option[CompletableFuture[Any]] = {
// Five lines of map-performance-avoidance, could be just: currentMessage map { _.senderFuture }
val msg = currentMessage
- if(msg.isEmpty) None
+ if (msg.isEmpty) None
else msg.get.senderFuture
}
@@ -1583,15 +1594,13 @@ trait ScalaActorRef extends ActorRefShared { ref: ActorRef =>
def !!(message: Any, timeout: Long = this.timeout)(implicit sender: Option[ActorRef] = None): Option[Any] = {
if (isRunning) {
val future = postMessageToMailboxAndCreateFutureResultWithTimeout[Any](message, timeout, sender, None)
- val isTypedActor = message.isInstanceOf[JoinPoint]
- if (isTypedActor && TypedActor.isOneWay(message.asInstanceOf[JoinPoint])) {
- future.asInstanceOf[CompletableFuture[Option[_]]].completeWithResult(None)
- }
+ val isMessageJoinPoint = if (isTypedActorEnabled) TypedActorModule.resolveFutureIfMessageIsJoinPoint(message, future)
+ else false
try {
future.await
} catch {
case e: FutureTimeoutException =>
- if (isTypedActor) throw e
+ if (isMessageJoinPoint) throw e
else None
}
if (future.exception.isDefined) throw future.exception.get
@@ -1670,8 +1679,10 @@ trait ScalaActorRef extends ActorRefShared { ref: ActorRef =>
/**
* Atomically create (from actor class), start and make an actor remote.
*/
- def spawnRemote[T <: Actor: Manifest](hostname: String, port: Int): ActorRef =
+ def spawnRemote[T <: Actor: Manifest](hostname: String, port: Int): ActorRef = {
+ ensureRemotingEnabled
spawnRemote(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]],hostname,port)
+ }
/**
@@ -1680,10 +1691,11 @@ trait ScalaActorRef extends ActorRefShared { ref: ActorRef =>
def spawnLink[T <: Actor: Manifest]: ActorRef =
spawnLink(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]])
-
/**
* Atomically create (from actor class), start, link and make an actor remote.
*/
- def spawnLinkRemote[T <: Actor : Manifest](hostname: String, port: Int): ActorRef =
+ def spawnLinkRemote[T <: Actor : Manifest](hostname: String, port: Int): ActorRef = {
+ ensureRemotingEnabled
spawnLinkRemote(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]],hostname,port)
+ }
}
diff --git a/akka-core/src/main/scala/actor/ActorRegistry.scala b/akka-actor/src/main/scala/actor/ActorRegistry.scala
similarity index 95%
rename from akka-core/src/main/scala/actor/ActorRegistry.scala
rename to akka-actor/src/main/scala/actor/ActorRegistry.scala
index 51de155723..e8c38f2b76 100644
--- a/akka-core/src/main/scala/actor/ActorRegistry.scala
+++ b/akka-actor/src/main/scala/actor/ActorRegistry.scala
@@ -8,10 +8,15 @@ import scala.collection.mutable.ListBuffer
import scala.reflect.Manifest
import java.util.concurrent.{ConcurrentSkipListSet, ConcurrentHashMap}
-import java.util.{Set=>JSet}
+import java.util.{Set => JSet}
import se.scalablesolutions.akka.util.ListenerManagement
+/**
+ * Base trait for ActorRegistry events, allows listen to when an actor is added and removed from the ActorRegistry.
+ *
+ * @author Jonas Bonér
+ */
sealed trait ActorRegistryEvent
case class ActorRegistered(actor: ActorRef) extends ActorRegistryEvent
case class ActorUnregistered(actor: ActorRef) extends ActorRegistryEvent
diff --git a/akka-core/src/main/scala/actor/Agent.scala b/akka-actor/src/main/scala/actor/Agent.scala
similarity index 100%
rename from akka-core/src/main/scala/actor/Agent.scala
rename to akka-actor/src/main/scala/actor/Agent.scala
diff --git a/akka-core/src/main/scala/actor/BootableActorLoaderService.scala b/akka-actor/src/main/scala/actor/BootableActorLoaderService.scala
similarity index 100%
rename from akka-core/src/main/scala/actor/BootableActorLoaderService.scala
rename to akka-actor/src/main/scala/actor/BootableActorLoaderService.scala
diff --git a/akka-core/src/main/scala/actor/Fsm.scala b/akka-actor/src/main/scala/actor/FSM.scala
similarity index 93%
rename from akka-core/src/main/scala/actor/Fsm.scala
rename to akka-actor/src/main/scala/actor/FSM.scala
index 6248a2575c..0bdc04fc48 100644
--- a/akka-core/src/main/scala/actor/Fsm.scala
+++ b/akka-actor/src/main/scala/actor/FSM.scala
@@ -1,11 +1,15 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
package se.scalablesolutions.akka.actor
import se.scalablesolutions.akka.stm.Ref
import se.scalablesolutions.akka.stm.local._
+
import java.util.concurrent.{ScheduledFuture, TimeUnit}
-trait FSM[S] {
- this: Actor =>
+trait FSM[S] { this: Actor =>
type StateFunction = scala.PartialFunction[Event, State]
@@ -20,7 +24,6 @@ trait FSM[S] {
State(NextState, currentState.stateFunction, stateData, currentState.timeout)
}
-
override final protected def receive: Receive = {
case value => {
timeoutFuture = timeoutFuture.flatMap {ref => ref.cancel(true); None}
diff --git a/akka-core/src/main/scala/actor/Implicits.scala b/akka-actor/src/main/scala/actor/Implicits.scala
similarity index 100%
rename from akka-core/src/main/scala/actor/Implicits.scala
rename to akka-actor/src/main/scala/actor/Implicits.scala
diff --git a/akka-core/src/main/scala/actor/Scheduler.scala b/akka-actor/src/main/scala/actor/Scheduler.scala
similarity index 99%
rename from akka-core/src/main/scala/actor/Scheduler.scala
rename to akka-actor/src/main/scala/actor/Scheduler.scala
index 50db44a1d0..5dc57056cc 100644
--- a/akka-core/src/main/scala/actor/Scheduler.scala
+++ b/akka-actor/src/main/scala/actor/Scheduler.scala
@@ -16,6 +16,7 @@
package se.scalablesolutions.akka.actor
import scala.collection.JavaConversions
+
import java.util.concurrent._
import se.scalablesolutions.akka.util.Logging
diff --git a/akka-core/src/main/scala/actor/Supervisor.scala b/akka-actor/src/main/scala/actor/Supervisor.scala
similarity index 96%
rename from akka-core/src/main/scala/actor/Supervisor.scala
rename to akka-actor/src/main/scala/actor/Supervisor.scala
index b146a74c12..1af351a33d 100644
--- a/akka-core/src/main/scala/actor/Supervisor.scala
+++ b/akka-actor/src/main/scala/actor/Supervisor.scala
@@ -6,9 +6,9 @@ package se.scalablesolutions.akka.actor
import se.scalablesolutions.akka.config.ScalaConfig._
import se.scalablesolutions.akka.config.{AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy}
-import se.scalablesolutions.akka.util.Logging
-import se.scalablesolutions.akka.remote.RemoteServer
import se.scalablesolutions.akka.AkkaException
+import se.scalablesolutions.akka.util._
+import ReflectiveAccess._
import Actor._
import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap}
@@ -162,8 +162,10 @@ sealed class Supervisor private[akka] (
_childActors.put(className, actorRef :: currentActors)
actorRef.lifeCycle = Some(lifeCycle)
supervisor.link(actorRef)
- remoteAddress.foreach(address => RemoteServer.registerActor(
- new InetSocketAddress(address.hostname, address.port), actorRef.uuid, actorRef))
+ remoteAddress.foreach { address =>
+ RemoteServerModule.registerActor(
+ new InetSocketAddress(address.hostname, address.port), actorRef.uuid, actorRef)
+ }
case supervisorConfig @ SupervisorConfig(_, _) => // recursive supervisor configuration
val childSupervisor = Supervisor(supervisorConfig)
supervisor.link(childSupervisor.supervisor)
diff --git a/akka-core/src/main/scala/actor/UntypedActor.scala b/akka-actor/src/main/scala/actor/UntypedActor.scala
similarity index 100%
rename from akka-core/src/main/scala/actor/UntypedActor.scala
rename to akka-actor/src/main/scala/actor/UntypedActor.scala
diff --git a/akka-core/src/main/scala/config/Config.scala b/akka-actor/src/main/scala/config/Config.scala
similarity index 83%
rename from akka-core/src/main/scala/config/Config.scala
rename to akka-actor/src/main/scala/config/Config.scala
index 04fe54848d..3b50d613c1 100644
--- a/akka-core/src/main/scala/config/Config.scala
+++ b/akka-actor/src/main/scala/config/Config.scala
@@ -4,19 +4,27 @@
package se.scalablesolutions.akka.config
-import se.scalablesolutions.akka.util.Logging
import se.scalablesolutions.akka.AkkaException
+import se.scalablesolutions.akka.util.Logging
+import se.scalablesolutions.akka.actor.{ActorRef, IllegalActorStateException}
+import se.scalablesolutions.akka.dispatch.CompletableFuture
import net.lag.configgy.{Config => CConfig, Configgy, ParseException}
+import java.net.InetSocketAddress
+import java.lang.reflect.Method
+
class ConfigurationException(message: String) extends AkkaException(message)
+class ModuleNotAvailableException(message: String) extends AkkaException(message)
+
+object ConfigLogger extends Logging
/**
* Loads up the configuration (from the akka.conf file).
*
* @author Jonas Bonér
*/
-object Config extends Logging {
+object Config {
val VERSION = "1.0-SNAPSHOT"
// Set Multiverse options for max speed
@@ -37,7 +45,7 @@ object Config extends Logging {
val configFile = System.getProperty("akka.config", "")
try {
Configgy.configure(configFile)
- log.info("Config loaded from -Dakka.config=%s", configFile)
+ ConfigLogger.log.info("Config loaded from -Dakka.config=%s", configFile)
} catch {
case e: ParseException => throw new ConfigurationException(
"Config could not be loaded from -Dakka.config=" + configFile +
@@ -47,7 +55,7 @@ object Config extends Logging {
} else if (getClass.getClassLoader.getResource("akka.conf") != null) {
try {
Configgy.configureFromResource("akka.conf", getClass.getClassLoader)
- log.info("Config loaded from the application classpath.")
+ ConfigLogger.log.info("Config loaded from the application classpath.")
} catch {
case e: ParseException => throw new ConfigurationException(
"Can't load 'akka.conf' config file from application classpath," +
@@ -58,7 +66,7 @@ object Config extends Logging {
try {
val configFile = HOME.get + "/config/akka.conf"
Configgy.configure(configFile)
- log.info("AKKA_HOME is defined as [%s], config loaded from [%s].", HOME.get, configFile)
+ ConfigLogger.log.info("AKKA_HOME is defined as [%s], config loaded from [%s].", HOME.get, configFile)
} catch {
case e: ParseException => throw new ConfigurationException(
"AKKA_HOME is defined as [" + HOME.get + "] " +
@@ -67,7 +75,7 @@ object Config extends Logging {
}
Configgy.config
} else {
- log.warning(
+ ConfigLogger.log.warning(
"\nCan't load 'akka.conf'." +
"\nOne of the three ways of locating the 'akka.conf' file needs to be defined:" +
"\n\t1. Define the '-Dakka.config=...' system property option." +
diff --git a/akka-core/src/main/scala/config/Configuration.scala b/akka-actor/src/main/scala/config/Configuration.scala
similarity index 100%
rename from akka-core/src/main/scala/config/Configuration.scala
rename to akka-actor/src/main/scala/config/Configuration.scala
diff --git a/akka-core/src/main/scala/config/Configurator.scala b/akka-actor/src/main/scala/config/Configurator.scala
similarity index 100%
rename from akka-core/src/main/scala/config/Configurator.scala
rename to akka-actor/src/main/scala/config/Configurator.scala
diff --git a/akka-core/src/main/scala/config/SupervisionConfig.scala b/akka-actor/src/main/scala/config/SupervisionConfig.scala
similarity index 100%
rename from akka-core/src/main/scala/config/SupervisionConfig.scala
rename to akka-actor/src/main/scala/config/SupervisionConfig.scala
diff --git a/akka-core/src/main/scala/dataflow/DataFlowVariable.scala b/akka-actor/src/main/scala/dataflow/DataFlowVariable.scala
similarity index 100%
rename from akka-core/src/main/scala/dataflow/DataFlowVariable.scala
rename to akka-actor/src/main/scala/dataflow/DataFlowVariable.scala
diff --git a/akka-core/src/main/scala/dispatch/AbstractReactorBasedEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/AbstractReactorBasedEventDrivenDispatcher.scala
similarity index 100%
rename from akka-core/src/main/scala/dispatch/AbstractReactorBasedEventDrivenDispatcher.scala
rename to akka-actor/src/main/scala/dispatch/AbstractReactorBasedEventDrivenDispatcher.scala
diff --git a/akka-core/src/main/scala/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/dispatch/Dispatchers.scala
similarity index 100%
rename from akka-core/src/main/scala/dispatch/Dispatchers.scala
rename to akka-actor/src/main/scala/dispatch/Dispatchers.scala
diff --git a/akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala
similarity index 100%
rename from akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala
rename to akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala
diff --git a/akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala
similarity index 100%
rename from akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala
rename to akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala
diff --git a/akka-core/src/main/scala/dispatch/Future.scala b/akka-actor/src/main/scala/dispatch/Future.scala
similarity index 100%
rename from akka-core/src/main/scala/dispatch/Future.scala
rename to akka-actor/src/main/scala/dispatch/Future.scala
diff --git a/akka-core/src/main/scala/dispatch/HawtDispatcher.scala b/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala
similarity index 100%
rename from akka-core/src/main/scala/dispatch/HawtDispatcher.scala
rename to akka-actor/src/main/scala/dispatch/HawtDispatcher.scala
diff --git a/akka-core/src/main/scala/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/dispatch/MessageHandling.scala
similarity index 100%
rename from akka-core/src/main/scala/dispatch/MessageHandling.scala
rename to akka-actor/src/main/scala/dispatch/MessageHandling.scala
diff --git a/akka-core/src/main/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcher.scala
similarity index 100%
rename from akka-core/src/main/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcher.scala
rename to akka-actor/src/main/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcher.scala
diff --git a/akka-core/src/main/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcher.scala
similarity index 100%
rename from akka-core/src/main/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcher.scala
rename to akka-actor/src/main/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcher.scala
diff --git a/akka-core/src/main/scala/dispatch/ThreadBasedDispatcher.scala b/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala
similarity index 100%
rename from akka-core/src/main/scala/dispatch/ThreadBasedDispatcher.scala
rename to akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala
diff --git a/akka-core/src/main/scala/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala
similarity index 100%
rename from akka-core/src/main/scala/dispatch/ThreadPoolBuilder.scala
rename to akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala
diff --git a/akka-core/src/main/scala/routing/Iterators.scala b/akka-actor/src/main/scala/routing/Iterators.scala
similarity index 100%
rename from akka-core/src/main/scala/routing/Iterators.scala
rename to akka-actor/src/main/scala/routing/Iterators.scala
diff --git a/akka-core/src/main/scala/routing/Listeners.scala b/akka-actor/src/main/scala/routing/Listeners.scala
similarity index 100%
rename from akka-core/src/main/scala/routing/Listeners.scala
rename to akka-actor/src/main/scala/routing/Listeners.scala
diff --git a/akka-core/src/main/scala/routing/Routers.scala b/akka-actor/src/main/scala/routing/Routers.scala
similarity index 100%
rename from akka-core/src/main/scala/routing/Routers.scala
rename to akka-actor/src/main/scala/routing/Routers.scala
diff --git a/akka-core/src/main/scala/routing/Routing.scala b/akka-actor/src/main/scala/routing/Routing.scala
similarity index 100%
rename from akka-core/src/main/scala/routing/Routing.scala
rename to akka-actor/src/main/scala/routing/Routing.scala
diff --git a/akka-core/src/main/scala/stm/JTA.scala b/akka-actor/src/main/scala/stm/JTA.scala
similarity index 98%
rename from akka-core/src/main/scala/stm/JTA.scala
rename to akka-actor/src/main/scala/stm/JTA.scala
index 80a0cda4ec..485f3e5104 100644
--- a/akka-core/src/main/scala/stm/JTA.scala
+++ b/akka-actor/src/main/scala/stm/JTA.scala
@@ -106,7 +106,8 @@ object TransactionContainer extends Logging {
*
* @author Jonas Bonér
*/
-class TransactionContainer private (val tm: Either[Option[UserTransaction], Option[TransactionManager]]) {
+class TransactionContainer private (
+ val tm: Either[Option[UserTransaction], Option[TransactionManager]]) extends Logging {
def registerSynchronization(sync: Synchronization) = {
TransactionContainer.findSynchronizationRegistry match { // try to use SynchronizationRegistry in JNDI
diff --git a/akka-core/src/main/scala/stm/Ref.scala b/akka-actor/src/main/scala/stm/Ref.scala
similarity index 100%
rename from akka-core/src/main/scala/stm/Ref.scala
rename to akka-actor/src/main/scala/stm/Ref.scala
diff --git a/akka-core/src/main/scala/stm/Transaction.scala b/akka-actor/src/main/scala/stm/Transaction.scala
similarity index 100%
rename from akka-core/src/main/scala/stm/Transaction.scala
rename to akka-actor/src/main/scala/stm/Transaction.scala
diff --git a/akka-core/src/main/scala/stm/TransactionFactory.scala b/akka-actor/src/main/scala/stm/TransactionFactory.scala
similarity index 100%
rename from akka-core/src/main/scala/stm/TransactionFactory.scala
rename to akka-actor/src/main/scala/stm/TransactionFactory.scala
diff --git a/akka-core/src/main/scala/stm/TransactionFactoryBuilder.scala b/akka-actor/src/main/scala/stm/TransactionFactoryBuilder.scala
similarity index 100%
rename from akka-core/src/main/scala/stm/TransactionFactoryBuilder.scala
rename to akka-actor/src/main/scala/stm/TransactionFactoryBuilder.scala
diff --git a/akka-core/src/main/scala/stm/TransactionManagement.scala b/akka-actor/src/main/scala/stm/TransactionManagement.scala
similarity index 100%
rename from akka-core/src/main/scala/stm/TransactionManagement.scala
rename to akka-actor/src/main/scala/stm/TransactionManagement.scala
diff --git a/akka-core/src/main/scala/stm/TransactionalMap.scala b/akka-actor/src/main/scala/stm/TransactionalMap.scala
similarity index 100%
rename from akka-core/src/main/scala/stm/TransactionalMap.scala
rename to akka-actor/src/main/scala/stm/TransactionalMap.scala
diff --git a/akka-core/src/main/scala/stm/TransactionalVector.scala b/akka-actor/src/main/scala/stm/TransactionalVector.scala
similarity index 100%
rename from akka-core/src/main/scala/stm/TransactionalVector.scala
rename to akka-actor/src/main/scala/stm/TransactionalVector.scala
diff --git a/akka-core/src/main/scala/stm/global/Atomic.scala b/akka-actor/src/main/scala/stm/global/Atomic.scala
similarity index 100%
rename from akka-core/src/main/scala/stm/global/Atomic.scala
rename to akka-actor/src/main/scala/stm/global/Atomic.scala
diff --git a/akka-core/src/main/scala/stm/global/GlobalStm.scala b/akka-actor/src/main/scala/stm/global/GlobalStm.scala
similarity index 78%
rename from akka-core/src/main/scala/stm/global/GlobalStm.scala
rename to akka-actor/src/main/scala/stm/global/GlobalStm.scala
index 1fd53ffe51..f2dfce8a96 100644
--- a/akka-core/src/main/scala/stm/global/GlobalStm.scala
+++ b/akka-actor/src/main/scala/stm/global/GlobalStm.scala
@@ -9,6 +9,8 @@ import se.scalablesolutions.akka.util.Logging
import org.multiverse.api.{Transaction => MultiverseTransaction}
import org.multiverse.templates.TransactionalCallable
+object GlobalStm extends Logging
+
/**
* Global transaction management, global in the context of multiple threads.
* Use this if you need to have one transaction span multiple threads (or Actors).
@@ -23,12 +25,14 @@ import org.multiverse.templates.TransactionalCallable
* }
*
*/
-class GlobalStm extends TransactionManagement with Logging {
+class GlobalStm extends TransactionManagement {
val DefaultGlobalTransactionConfig = TransactionConfig()
- val DefaultGlobalTransactionFactory = TransactionFactory(DefaultGlobalTransactionConfig, "DefaultGlobalTransaction")
+ val DefaultGlobalTransactionFactory = TransactionFactory(
+ DefaultGlobalTransactionConfig, "DefaultGlobalTransaction")
- def atomic[T](body: => T)(implicit factory: TransactionFactory = DefaultGlobalTransactionFactory): T = atomic(factory)(body)
+ def atomic[T](body: => T)(implicit factory: TransactionFactory = DefaultGlobalTransactionFactory): T =
+ atomic(factory)(body)
def atomic[T](factory: TransactionFactory)(body: => T): T = {
factory.boilerplate.execute(new TransactionalCallable[T]() {
@@ -37,7 +41,8 @@ class GlobalStm extends TransactionManagement with Logging {
factory.addHooks
val result = body
val txSet = getTransactionSetInScope
- log.trace("Committing global transaction [" + mtx + "]\n\tand joining transaction set [" + txSet + "]")
+ GlobalStm.log.trace(
+ "Committing global transaction [" + mtx + "]\n\tand joining transaction set [" + txSet + "]")
try {
txSet.tryJoinCommit(
mtx,
diff --git a/akka-core/src/main/scala/stm/global/package.scala b/akka-actor/src/main/scala/stm/global/package.scala
similarity index 100%
rename from akka-core/src/main/scala/stm/global/package.scala
rename to akka-actor/src/main/scala/stm/global/package.scala
diff --git a/akka-core/src/main/scala/stm/local/Atomic.scala b/akka-actor/src/main/scala/stm/local/Atomic.scala
similarity index 100%
rename from akka-core/src/main/scala/stm/local/Atomic.scala
rename to akka-actor/src/main/scala/stm/local/Atomic.scala
diff --git a/akka-core/src/main/scala/stm/local/LocalStm.scala b/akka-actor/src/main/scala/stm/local/LocalStm.scala
similarity index 79%
rename from akka-core/src/main/scala/stm/local/LocalStm.scala
rename to akka-actor/src/main/scala/stm/local/LocalStm.scala
index 477027aab2..f0e60206f6 100644
--- a/akka-core/src/main/scala/stm/local/LocalStm.scala
+++ b/akka-actor/src/main/scala/stm/local/LocalStm.scala
@@ -9,6 +9,8 @@ import se.scalablesolutions.akka.util.Logging
import org.multiverse.api.{Transaction => MultiverseTransaction}
import org.multiverse.templates.TransactionalCallable
+object LocalStm extends Logging
+
/**
* Local transaction management, local in the context of threads.
* Use this if you do not need to have one transaction span
@@ -27,16 +29,18 @@ import org.multiverse.templates.TransactionalCallable
class LocalStm extends TransactionManagement with Logging {
val DefaultLocalTransactionConfig = TransactionConfig()
- val DefaultLocalTransactionFactory = TransactionFactory(DefaultLocalTransactionConfig, "DefaultLocalTransaction")
+ val DefaultLocalTransactionFactory = TransactionFactory(
+ DefaultLocalTransactionConfig, "DefaultLocalTransaction")
- def atomic[T](body: => T)(implicit factory: TransactionFactory = DefaultLocalTransactionFactory): T = atomic(factory)(body)
+ def atomic[T](body: => T)(implicit factory: TransactionFactory = DefaultLocalTransactionFactory): T =
+ atomic(factory)(body)
def atomic[T](factory: TransactionFactory)(body: => T): T = {
factory.boilerplate.execute(new TransactionalCallable[T]() {
def call(mtx: MultiverseTransaction): T = {
factory.addHooks
val result = body
- log.trace("Committing local transaction [" + mtx + "]")
+ LocalStm.log.trace("Committing local transaction [" + mtx + "]")
result
}
})
diff --git a/akka-core/src/main/scala/stm/local/package.scala b/akka-actor/src/main/scala/stm/local/package.scala
similarity index 100%
rename from akka-core/src/main/scala/stm/local/package.scala
rename to akka-actor/src/main/scala/stm/local/package.scala
diff --git a/akka-core/src/main/scala/stm/transactional.scala b/akka-actor/src/main/scala/stm/transactional.scala
similarity index 100%
rename from akka-core/src/main/scala/stm/transactional.scala
rename to akka-actor/src/main/scala/stm/transactional.scala
diff --git a/akka-core/src/main/scala/util/AkkaException.scala b/akka-actor/src/main/scala/util/AkkaException.scala
similarity index 100%
rename from akka-core/src/main/scala/util/AkkaException.scala
rename to akka-actor/src/main/scala/util/AkkaException.scala
diff --git a/akka-core/src/main/scala/util/Bootable.scala b/akka-actor/src/main/scala/util/Bootable.scala
similarity index 100%
rename from akka-core/src/main/scala/util/Bootable.scala
rename to akka-actor/src/main/scala/util/Bootable.scala
diff --git a/akka-core/src/main/scala/util/Duration.scala b/akka-actor/src/main/scala/util/Duration.scala
similarity index 100%
rename from akka-core/src/main/scala/util/Duration.scala
rename to akka-actor/src/main/scala/util/Duration.scala
diff --git a/akka-core/src/main/scala/util/HashCode.scala b/akka-actor/src/main/scala/util/HashCode.scala
similarity index 100%
rename from akka-core/src/main/scala/util/HashCode.scala
rename to akka-actor/src/main/scala/util/HashCode.scala
diff --git a/akka-core/src/main/scala/util/Helpers.scala b/akka-actor/src/main/scala/util/Helpers.scala
similarity index 100%
rename from akka-core/src/main/scala/util/Helpers.scala
rename to akka-actor/src/main/scala/util/Helpers.scala
diff --git a/akka-core/src/main/scala/util/ListenerManagement.scala b/akka-actor/src/main/scala/util/ListenerManagement.scala
similarity index 100%
rename from akka-core/src/main/scala/util/ListenerManagement.scala
rename to akka-actor/src/main/scala/util/ListenerManagement.scala
diff --git a/akka-core/src/main/scala/util/LockUtil.scala b/akka-actor/src/main/scala/util/LockUtil.scala
similarity index 100%
rename from akka-core/src/main/scala/util/LockUtil.scala
rename to akka-actor/src/main/scala/util/LockUtil.scala
diff --git a/akka-core/src/main/scala/util/Logging.scala b/akka-actor/src/main/scala/util/Logging.scala
similarity index 100%
rename from akka-core/src/main/scala/util/Logging.scala
rename to akka-actor/src/main/scala/util/Logging.scala
diff --git a/akka-actor/src/main/scala/util/ReflectiveAccess.scala b/akka-actor/src/main/scala/util/ReflectiveAccess.scala
new file mode 100644
index 0000000000..da9fb2f3c6
--- /dev/null
+++ b/akka-actor/src/main/scala/util/ReflectiveAccess.scala
@@ -0,0 +1,193 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package se.scalablesolutions.akka.util
+
+import se.scalablesolutions.akka.actor.{ActorRef, IllegalActorStateException, ActorType}
+import se.scalablesolutions.akka.dispatch.{Future, CompletableFuture}
+import se.scalablesolutions.akka.config.{Config, ModuleNotAvailableException}
+
+import java.net.InetSocketAddress
+
+/**
+ * Helper class for reflective access to different modules in order to allow optional loading of modules.
+ *
+ * @author Jonas Bonér
+ */
+object ReflectiveAccess {
+
+ val loader = getClass.getClassLoader
+
+ lazy val isRemotingEnabled = RemoteClientModule.isRemotingEnabled
+ lazy val isTypedActorEnabled = TypedActorModule.isTypedActorEnabled
+
+ def ensureRemotingEnabled = RemoteClientModule.ensureRemotingEnabled
+ def ensureTypedActorEnabled = TypedActorModule.ensureTypedActorEnabled
+
+ /**
+ * Reflective access to the RemoteClient module.
+ *
+ * @author Jonas Bonér
+ */
+ object RemoteClientModule {
+
+ type RemoteClient = {
+ def send[T](
+ message: Any,
+ senderOption: Option[ActorRef],
+ senderFuture: Option[CompletableFuture[_]],
+ remoteAddress: InetSocketAddress,
+ timeout: Long,
+ isOneWay: Boolean,
+ actorRef: ActorRef,
+ typedActorInfo: Option[Tuple2[String, String]],
+ actorType: ActorType): Option[CompletableFuture[T]]
+ def registerSupervisorForActor(actorRef: ActorRef)
+ }
+
+ type RemoteClientObject = {
+ def register(hostname: String, port: Int, uuid: String): Unit
+ def unregister(hostname: String, port: Int, uuid: String): Unit
+ def clientFor(address: InetSocketAddress): RemoteClient
+ def clientFor(hostname: String, port: Int, loader: Option[ClassLoader]): RemoteClient
+ }
+
+ lazy val isRemotingEnabled = remoteClientObjectInstance.isDefined
+
+ def ensureRemotingEnabled = if (!isRemotingEnabled) throw new ModuleNotAvailableException(
+ "Can't load the remoting module, make sure that akka-remote.jar is on the classpath")
+
+ val remoteClientObjectInstance: Option[RemoteClientObject] = {
+ try {
+ val clazz = loader.loadClass("se.scalablesolutions.akka.remote.RemoteClient$")
+ val ctor = clazz.getDeclaredConstructor(Array[Class[_]](): _*)
+ ctor.setAccessible(true)
+ Some(ctor.newInstance(Array[AnyRef](): _*).asInstanceOf[RemoteClientObject])
+ } catch { case e => None }
+ }
+
+ def register(address: InetSocketAddress, uuid: String) = {
+ ensureRemotingEnabled
+ remoteClientObjectInstance.get.register(address.getHostName, address.getPort, uuid)
+ }
+
+ def unregister(address: InetSocketAddress, uuid: String) = {
+ ensureRemotingEnabled
+ remoteClientObjectInstance.get.unregister(address.getHostName, address.getPort, uuid)
+ }
+
+ def registerSupervisorForActor(remoteAddress: InetSocketAddress, actorRef: ActorRef) = {
+ ensureRemotingEnabled
+ val remoteClient = remoteClientObjectInstance.get.clientFor(remoteAddress)
+ remoteClient.registerSupervisorForActor(actorRef)
+ }
+
+ def clientFor(hostname: String, port: Int, loader: Option[ClassLoader]): RemoteClient = {
+ ensureRemotingEnabled
+ remoteClientObjectInstance.get.clientFor(hostname, port, loader)
+ }
+
+ def send[T](
+ message: Any,
+ senderOption: Option[ActorRef],
+ senderFuture: Option[CompletableFuture[_]],
+ remoteAddress: InetSocketAddress,
+ timeout: Long,
+ isOneWay: Boolean,
+ actorRef: ActorRef,
+ typedActorInfo: Option[Tuple2[String, String]],
+ actorType: ActorType): Option[CompletableFuture[T]] = {
+ ensureRemotingEnabled
+ clientFor(remoteAddress.getHostName, remoteAddress.getPort, None).send[T](
+ message, senderOption, senderFuture, remoteAddress, timeout, isOneWay, actorRef, typedActorInfo, actorType)
+ }
+ }
+
+ /**
+ * Reflective access to the RemoteServer module.
+ *
+ * @author Jonas Bonér
+ */
+ object RemoteServerModule {
+ val HOSTNAME = Config.config.getString("akka.remote.server.hostname", "localhost")
+ val PORT = Config.config.getInt("akka.remote.server.port", 9999)
+
+ type RemoteServerObject = {
+ def registerActor(address: InetSocketAddress, uuid: String, actor: ActorRef): Unit
+ def registerTypedActor(address: InetSocketAddress, name: String, typedActor: AnyRef): Unit
+ }
+
+ type RemoteNodeObject = {
+ def unregister(actorRef: ActorRef): Unit
+ }
+
+ val remoteServerObjectInstance: Option[RemoteServerObject] = {
+ try {
+ val clazz = loader.loadClass("se.scalablesolutions.akka.remote.RemoteServer$")
+ val ctor = clazz.getDeclaredConstructor(Array[Class[_]](): _*)
+ ctor.setAccessible(true)
+ Some(ctor.newInstance(Array[AnyRef](): _*).asInstanceOf[RemoteServerObject])
+ } catch { case e => None }
+ }
+
+ val remoteNodeObjectInstance: Option[RemoteNodeObject] = {
+ try {
+ val clazz = loader.loadClass("se.scalablesolutions.akka.remote.RemoteNode$")
+ val ctor = clazz.getDeclaredConstructor(Array[Class[_]](): _*)
+ ctor.setAccessible(true)
+ Some(ctor.newInstance(Array[AnyRef](): _*).asInstanceOf[RemoteNodeObject])
+ } catch { case e => None }
+ }
+
+ def registerActor(address: InetSocketAddress, uuid: String, actorRef: ActorRef) = {
+ ensureRemotingEnabled
+ remoteServerObjectInstance.get.registerActor(address, uuid, actorRef)
+ }
+
+ def registerTypedActor(address: InetSocketAddress, implementationClassName: String, proxy: AnyRef) = {
+ ensureRemotingEnabled
+ remoteServerObjectInstance.get.registerTypedActor(address, implementationClassName, proxy)
+ }
+
+ def unregister(actorRef: ActorRef) = {
+ ensureRemotingEnabled
+ remoteNodeObjectInstance.get.unregister(actorRef)
+ }
+ }
+
+ /**
+ * Reflective access to the TypedActors module.
+ *
+ * @author Jonas Bonér
+ */
+ object TypedActorModule {
+
+ type TypedActorObject = {
+ def isJoinPoint(message: Any): Boolean
+ def isJoinPointAndOneWay(message: Any): Boolean
+ }
+
+ lazy val isTypedActorEnabled = typedActorObjectInstance.isDefined
+
+ def ensureTypedActorEnabled = if (!isTypedActorEnabled) throw new ModuleNotAvailableException(
+ "Can't load the typed actor module, make sure that akka-typed-actor.jar is on the classpath")
+
+ val typedActorObjectInstance: Option[TypedActorObject] = {
+ try {
+ val clazz = loader.loadClass("se.scalablesolutions.akka.actor.TypedActor$")
+ val ctor = clazz.getDeclaredConstructor(Array[Class[_]](): _*)
+ ctor.setAccessible(true)
+ Some(ctor.newInstance(Array[AnyRef](): _*).asInstanceOf[TypedActorObject])
+ } catch { case e => None }
+ }
+
+ def resolveFutureIfMessageIsJoinPoint(message: Any, future: Future[_]): Boolean = {
+ ensureTypedActorEnabled
+ if (typedActorObjectInstance.get.isJoinPointAndOneWay(message)) {
+ future.asInstanceOf[CompletableFuture[Option[_]]].completeWithResult(None)
+ }
+ typedActorObjectInstance.get.isJoinPoint(message)
+ }
+ }
+}
diff --git a/akka-core/src/main/scala/util/Uuid.scala b/akka-actor/src/main/scala/util/Uuid.scala
similarity index 100%
rename from akka-core/src/main/scala/util/Uuid.scala
rename to akka-actor/src/main/scala/util/Uuid.scala
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/stm/Address.java b/akka-actor/src/test/java/se/scalablesolutions/akka/stm/Address.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/stm/Address.java
rename to akka-actor/src/test/java/se/scalablesolutions/akka/stm/Address.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/stm/CounterExample.java b/akka-actor/src/test/java/se/scalablesolutions/akka/stm/CounterExample.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/stm/CounterExample.java
rename to akka-actor/src/test/java/se/scalablesolutions/akka/stm/CounterExample.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/stm/JavaStmTests.java b/akka-actor/src/test/java/se/scalablesolutions/akka/stm/JavaStmTests.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/stm/JavaStmTests.java
rename to akka-actor/src/test/java/se/scalablesolutions/akka/stm/JavaStmTests.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/stm/RefExample.java b/akka-actor/src/test/java/se/scalablesolutions/akka/stm/RefExample.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/stm/RefExample.java
rename to akka-actor/src/test/java/se/scalablesolutions/akka/stm/RefExample.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/stm/StmExamples.java b/akka-actor/src/test/java/se/scalablesolutions/akka/stm/StmExamples.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/stm/StmExamples.java
rename to akka-actor/src/test/java/se/scalablesolutions/akka/stm/StmExamples.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/stm/TransactionFactoryExample.java b/akka-actor/src/test/java/se/scalablesolutions/akka/stm/TransactionFactoryExample.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/stm/TransactionFactoryExample.java
rename to akka-actor/src/test/java/se/scalablesolutions/akka/stm/TransactionFactoryExample.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/stm/TransactionalMapExample.java b/akka-actor/src/test/java/se/scalablesolutions/akka/stm/TransactionalMapExample.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/stm/TransactionalMapExample.java
rename to akka-actor/src/test/java/se/scalablesolutions/akka/stm/TransactionalMapExample.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/stm/TransactionalVectorExample.java b/akka-actor/src/test/java/se/scalablesolutions/akka/stm/TransactionalVectorExample.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/stm/TransactionalVectorExample.java
rename to akka-actor/src/test/java/se/scalablesolutions/akka/stm/TransactionalVectorExample.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/stm/User.java b/akka-actor/src/test/java/se/scalablesolutions/akka/stm/User.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/stm/User.java
rename to akka-actor/src/test/java/se/scalablesolutions/akka/stm/User.java
diff --git a/akka-core/src/test/resources/logback-test.xml b/akka-actor/src/test/resources/logback-test.xml
similarity index 100%
rename from akka-core/src/test/resources/logback-test.xml
rename to akka-actor/src/test/resources/logback-test.xml
diff --git a/akka-actor/src/test/scala/Messages.scala b/akka-actor/src/test/scala/Messages.scala
new file mode 100644
index 0000000000..7e22dd9d7c
--- /dev/null
+++ b/akka-actor/src/test/scala/Messages.scala
@@ -0,0 +1,13 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package se.scalablesolutions.akka
+
+abstract class TestMessage
+
+case object Ping extends TestMessage
+case object Pong extends TestMessage
+case object OneWay extends TestMessage
+case object Die extends TestMessage
+case object NotifySupervisorExit extends TestMessage
diff --git a/akka-core/src/test/scala/actor/actor/ActorFireForgetRequestReplySpec.scala b/akka-actor/src/test/scala/actor/actor/ActorFireForgetRequestReplySpec.scala
similarity index 100%
rename from akka-core/src/test/scala/actor/actor/ActorFireForgetRequestReplySpec.scala
rename to akka-actor/src/test/scala/actor/actor/ActorFireForgetRequestReplySpec.scala
diff --git a/akka-core/src/test/scala/actor/actor/AgentSpec.scala b/akka-actor/src/test/scala/actor/actor/AgentSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/actor/actor/AgentSpec.scala
rename to akka-actor/src/test/scala/actor/actor/AgentSpec.scala
diff --git a/akka-core/src/test/scala/actor/actor/Bench.scala b/akka-actor/src/test/scala/actor/actor/Bench.scala
similarity index 100%
rename from akka-core/src/test/scala/actor/actor/Bench.scala
rename to akka-actor/src/test/scala/actor/actor/Bench.scala
diff --git a/akka-core/src/test/scala/actor/actor/FsmActorSpec.scala b/akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/actor/actor/FsmActorSpec.scala
rename to akka-actor/src/test/scala/actor/actor/FSMActorSpec.scala
diff --git a/akka-core/src/test/scala/actor/actor/ForwardActorSpec.scala b/akka-actor/src/test/scala/actor/actor/ForwardActorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/actor/actor/ForwardActorSpec.scala
rename to akka-actor/src/test/scala/actor/actor/ForwardActorSpec.scala
diff --git a/akka-core/src/test/scala/actor/actor/ReceiveTimeoutSpec.scala b/akka-actor/src/test/scala/actor/actor/ReceiveTimeoutSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/actor/actor/ReceiveTimeoutSpec.scala
rename to akka-actor/src/test/scala/actor/actor/ReceiveTimeoutSpec.scala
diff --git a/akka-core/src/test/scala/actor/actor/TransactorSpec.scala b/akka-actor/src/test/scala/actor/actor/TransactorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/actor/actor/TransactorSpec.scala
rename to akka-actor/src/test/scala/actor/actor/TransactorSpec.scala
diff --git a/akka-core/src/test/scala/actor/supervisor/RestartStrategySpec.scala b/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala
similarity index 100%
rename from akka-core/src/test/scala/actor/supervisor/RestartStrategySpec.scala
rename to akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala
diff --git a/akka-core/src/test/scala/actor/supervisor/SupervisorHierarchySpec.scala b/akka-actor/src/test/scala/actor/supervisor/SupervisorHierarchySpec.scala
similarity index 100%
rename from akka-core/src/test/scala/actor/supervisor/SupervisorHierarchySpec.scala
rename to akka-actor/src/test/scala/actor/supervisor/SupervisorHierarchySpec.scala
diff --git a/akka-core/src/test/scala/actor/supervisor/SupervisorSpec.scala b/akka-actor/src/test/scala/actor/supervisor/SupervisorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/actor/supervisor/SupervisorSpec.scala
rename to akka-actor/src/test/scala/actor/supervisor/SupervisorSpec.scala
diff --git a/akka-core/src/test/scala/dataflow/DataFlowSpec.scala b/akka-actor/src/test/scala/dataflow/DataFlowSpec.scala
similarity index 99%
rename from akka-core/src/test/scala/dataflow/DataFlowSpec.scala
rename to akka-actor/src/test/scala/dataflow/DataFlowSpec.scala
index cdc8bf2455..2997715452 100644
--- a/akka-core/src/test/scala/dataflow/DataFlowSpec.scala
+++ b/akka-actor/src/test/scala/dataflow/DataFlowSpec.scala
@@ -70,4 +70,4 @@ class DataFlowTest extends Spec with ShouldMatchers with BeforeAndAfterAll {
List(x,y,z).foreach(_.shutdown)
}
}
-}
+}
\ No newline at end of file
diff --git a/akka-core/src/test/scala/dispatch/DispatchersSpec.scala b/akka-actor/src/test/scala/dispatch/DispatchersSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/dispatch/DispatchersSpec.scala
rename to akka-actor/src/test/scala/dispatch/DispatchersSpec.scala
diff --git a/akka-core/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala
rename to akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala
diff --git a/akka-core/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorsSpec.scala b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorsSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorsSpec.scala
rename to akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorsSpec.scala
diff --git a/akka-core/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala
rename to akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala
diff --git a/akka-core/src/test/scala/dispatch/FutureSpec.scala b/akka-actor/src/test/scala/dispatch/FutureSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/dispatch/FutureSpec.scala
rename to akka-actor/src/test/scala/dispatch/FutureSpec.scala
diff --git a/akka-core/src/test/scala/dispatch/HawtDispatcherActorSpec.scala b/akka-actor/src/test/scala/dispatch/HawtDispatcherActorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/dispatch/HawtDispatcherActorSpec.scala
rename to akka-actor/src/test/scala/dispatch/HawtDispatcherActorSpec.scala
diff --git a/akka-core/src/test/scala/dispatch/HawtDispatcherEchoServer.scala b/akka-actor/src/test/scala/dispatch/HawtDispatcherEchoServer.scala
similarity index 100%
rename from akka-core/src/test/scala/dispatch/HawtDispatcherEchoServer.scala
rename to akka-actor/src/test/scala/dispatch/HawtDispatcherEchoServer.scala
diff --git a/akka-core/src/test/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcherActorSpec.scala b/akka-actor/src/test/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcherActorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcherActorSpec.scala
rename to akka-actor/src/test/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcherActorSpec.scala
diff --git a/akka-core/src/test/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcherActorSpec.scala b/akka-actor/src/test/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcherActorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcherActorSpec.scala
rename to akka-actor/src/test/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcherActorSpec.scala
diff --git a/akka-core/src/test/scala/dispatch/ThreadBasedActorSpec.scala b/akka-actor/src/test/scala/dispatch/ThreadBasedActorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/dispatch/ThreadBasedActorSpec.scala
rename to akka-actor/src/test/scala/dispatch/ThreadBasedActorSpec.scala
diff --git a/akka-core/src/test/scala/dispatch/ThreadBasedDispatcherSpec.scala b/akka-actor/src/test/scala/dispatch/ThreadBasedDispatcherSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/dispatch/ThreadBasedDispatcherSpec.scala
rename to akka-actor/src/test/scala/dispatch/ThreadBasedDispatcherSpec.scala
diff --git a/akka-core/src/test/scala/misc/ActorRegistrySpec.scala b/akka-actor/src/test/scala/misc/ActorRegistrySpec.scala
similarity index 100%
rename from akka-core/src/test/scala/misc/ActorRegistrySpec.scala
rename to akka-actor/src/test/scala/misc/ActorRegistrySpec.scala
diff --git a/akka-core/src/test/scala/misc/SchedulerSpec.scala b/akka-actor/src/test/scala/misc/SchedulerSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/misc/SchedulerSpec.scala
rename to akka-actor/src/test/scala/misc/SchedulerSpec.scala
diff --git a/akka-core/src/test/scala/routing/RoutingSpec.scala b/akka-actor/src/test/scala/routing/RoutingSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/routing/RoutingSpec.scala
rename to akka-actor/src/test/scala/routing/RoutingSpec.scala
diff --git a/akka-core/src/test/scala/stm/JavaStmSpec.scala b/akka-actor/src/test/scala/stm/JavaStmSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/stm/JavaStmSpec.scala
rename to akka-actor/src/test/scala/stm/JavaStmSpec.scala
diff --git a/akka-core/src/test/scala/stm/RefSpec.scala b/akka-actor/src/test/scala/stm/RefSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/stm/RefSpec.scala
rename to akka-actor/src/test/scala/stm/RefSpec.scala
diff --git a/akka-core/src/test/scala/stm/StmSpec.scala b/akka-actor/src/test/scala/stm/StmSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/stm/StmSpec.scala
rename to akka-actor/src/test/scala/stm/StmSpec.scala
diff --git a/akka-core/src/test/scala/ticket/Ticket001Spec.scala b/akka-actor/src/test/scala/ticket/Ticket001Spec.scala
similarity index 100%
rename from akka-core/src/test/scala/ticket/Ticket001Spec.scala
rename to akka-actor/src/test/scala/ticket/Ticket001Spec.scala
diff --git a/akka-core/.ensime b/akka-core/.ensime
deleted file mode 100644
index 0b21e8eb5c..0000000000
--- a/akka-core/.ensime
+++ /dev/null
@@ -1,79 +0,0 @@
-(
-
- ;; Where you unpacked the ENSIME distribution.
- :server-root "/Users/jboner/config/emacs-config/lib/ensime"
-
- ;; The command with which to invoke the ENSIME server. Change this to
- ;; "bin/server.bat" if your're on Windows.
- :server-cmd "bin/server.sh"
-
-
- ;; The host to connect to. Connecting to remote ENSIME servers is not
- ;; currently supported.
- ;; ------------------------------
- ;; :server-host "localhost"
-
-
- ;; Assume a standard sbt directory structure. Look in default sbt
- ;; locations for dependencies, sources, target, etc.
- ;;
- ;; Note for sbt subprojects: Each subproject needs it's own .ensime
- ;; file.
- ;; -----------------------------
- :use-sbt t
- :sbt-compile-conf "compile"
-
-
- ;; Use an existing pom.xml to determine the dependencies
- ;; for the project. A Maven-style directory structure is assumed.
- ;; -----------------------------
- ;; :use-maven t
- ;; :maven-compile-scopes "compile"
- ;; :maven-runtime-scopes "runtime"
-
-
- ;; Use an existing ivy.xml to determine the dependencies
- ;; for the project. A Maven-style directory structure is assumed.
- ;; -----------------------------
- ;; :use-ivy t
- ;; :ivy-compile-conf "compile"
- ;; :ivy-runtime-conf "compile"
-
-
- ;; The home package for your project.
- ;; Used by ENSIME to populate the project outline view.
- ;; ------------------------------
- :project-package "se.scalablesolutions.akka"
-
-
- ;; :sources ([dir | file]*)
- ;; Include source files by directory(recursively) or by filename.
- ;; ------------------------------
- :sources ("src/main/")
-
-
- ;; :dependency-jars ([dir | file]*)
- ;; Include jars by directory(recursively) or by filename.
- ;; ------------------------------
- ;; :dependency-jars ("lib")
-
-
- ;; :dependency-dirs ([dir | file]*)
- ;; Include directories of .class files.
- ;; ------------------------------
- ;; :dependency-dirs ("target/classes")
-
-
- ;; :target dir
- ;; Specify the target of the project build process. Should be
- ;; the directory where .class files are written
- ;;
- ;; The target is used to populate the classpath when launching
- ;; the inferior scala repl.
- ;; ------------------------------
- ;; :target "target/classes"
-
- )
-
-
-
diff --git a/akka-core/src/test/scala/TestClasses.bak b/akka-core/src/test/scala/TestClasses.bak
deleted file mode 100644
index 5a0ec08c19..0000000000
--- a/akka-core/src/test/scala/TestClasses.bak
+++ /dev/null
@@ -1,102 +0,0 @@
-/**
- * Copyright (C) 2009-2010 Scalable Solutions AB
- */
-
-package se.scalablesolutions.akka.actor
-
-import se.scalablesolutions.akka.serialization.Serializable
-import se.scalablesolutions.akka.actor.annotation.transactionrequired
-import se.scalablesolutions.akka.actor.annotation.prerestart
-import se.scalablesolutions.akka.actor.annotation.postrestart
-import se.scalablesolutions.akka.actor.annotation.inittransactionalstate
-import se.scalablesolutions.akka.actor.annotation.oneway
-import se.scalablesolutions.akka.stm._
-
-import com.google.inject.Inject
-
-trait Bar {
- @oneway
- def bar(msg: String): String
- def getExt: Ext
-}
-
-class BarImpl extends Bar {
- @Inject private var ext: Ext = _
- def getExt: Ext = ext
- def bar(msg: String) = msg
-}
-
-trait Ext
-class ExtImpl extends Ext
-
-class Foo extends Serializable.JavaJSON {
- @Inject
- private var bar: Bar = _
- def body = this
- def getBar = bar
- def foo(msg: String): String = msg + "_foo "
- def bar(msg: String): String = bar.bar(msg)
- def longRunning = {
- Thread.sleep(10000)
- "test"
- }
- def throwsException: String = {
- if (true) throw new RuntimeException("Expected exception; to test fault-tolerance")
- "test"
- }
-}
-
-@serializable class InMemFailer {
- def fail = throw new RuntimeException("Expected exception; to test fault-tolerance")
-}
-
-@transactionrequired
-class InMemStateful {
- private lazy val mapState = TransactionalState.newMap[String, String]
- private lazy val vectorState = TransactionalState.newVector[String]
- private lazy val refState = TransactionalState.newRef[String]
-
- def getMapState(key: String): String = mapState.get(key).get
- def getVectorState: String = vectorState.last
- def getRefState: String = refState.get.get
- def setMapState(key: String, msg: String): Unit = mapState.put(key, msg)
- def setVectorState(msg: String): Unit = vectorState.add(msg)
- def setRefState(msg: String): Unit = refState.swap(msg)
- def success(key: String, msg: String): Unit = {
- mapState.put(key, msg)
- vectorState.add(msg)
- refState.swap(msg)
- }
-
- def success(key: String, msg: String, nested: InMemStatefulNested): Unit = {
- mapState.put(key, msg)
- vectorState.add(msg)
- refState.swap(msg)
- nested.success(key, msg)
- }
-
- def failure(key: String, msg: String, failer: InMemFailer): String = {
- mapState.put(key, msg)
- vectorState.add(msg)
- refState.swap(msg)
- failer.fail
- msg
- }
-
- def failure(key: String, msg: String, nested: InMemStatefulNested, failer: InMemFailer): String = {
- mapState.put(key, msg)
- vectorState.add(msg)
- refState.swap(msg)
- nested.failure(key, msg, failer)
- msg
- }
-
- def thisMethodHangs(key: String, msg: String, failer: InMemFailer) = setMapState(key, msg)
-
- @prerestart def preRestart = println("################ PRE RESTART")
- @postrestart def postRestart = println("################ POST RESTART")
-}
-
-@transactionrequired
-class InMemStatefulNested extends InMemStateful
-
diff --git a/akka-karaf/akka-features/src/main/resources/features.xml b/akka-karaf/akka-features/src/main/resources/features.xml
index db48a13785..067e64f193 100644
--- a/akka-karaf/akka-features/src/main/resources/features.xml
+++ b/akka-karaf/akka-features/src/main/resources/features.xml
@@ -13,10 +13,10 @@
mvn:sjson.json/sjson/0.6-SNAPSHOT
-
+
sjson
mvn:se.scalablesolutions.akka.akka-wrap/jgroups-wrapper_2.8.0.RC3_osgi/2.9.0.GA
mvn:org.jboss.netty/netty/3.2.0.CR1
- mvn:se.scalablesolutions.akka/akka-core_2.8.0.RC3_osgi/0.9
+ mvn:se.scalablesolutions.akka/akka-remote_2.8.0.RC3_osgi/0.9
diff --git a/akka-core/src/main/protocol/RemoteProtocol.proto b/akka-remote/src/main/protocol/RemoteProtocol.proto
similarity index 98%
rename from akka-core/src/main/protocol/RemoteProtocol.proto
rename to akka-remote/src/main/protocol/RemoteProtocol.proto
index 6cf9bfd534..567bf54eba 100644
--- a/akka-core/src/main/protocol/RemoteProtocol.proto
+++ b/akka-remote/src/main/protocol/RemoteProtocol.proto
@@ -7,7 +7,7 @@ option optimize_for = SPEED;
/******************************************
Compile with:
- cd ./akka-core/src/main/protocol
+ cd ./akka-remote/src/main/protocol
protoc RemoteProtocol.proto --java_out ../java
*******************************************/
diff --git a/akka-core/src/main/scala/remote/BootableRemoteActorService.scala b/akka-remote/src/main/scala/remote/BootableRemoteActorService.scala
similarity index 100%
rename from akka-core/src/main/scala/remote/BootableRemoteActorService.scala
rename to akka-remote/src/main/scala/remote/BootableRemoteActorService.scala
diff --git a/akka-core/src/main/scala/remote/Cluster.scala b/akka-remote/src/main/scala/remote/Cluster.scala
similarity index 100%
rename from akka-core/src/main/scala/remote/Cluster.scala
rename to akka-remote/src/main/scala/remote/Cluster.scala
diff --git a/akka-core/src/main/scala/remote/JGroupsClusterActor.scala b/akka-remote/src/main/scala/remote/JGroupsClusterActor.scala
similarity index 100%
rename from akka-core/src/main/scala/remote/JGroupsClusterActor.scala
rename to akka-remote/src/main/scala/remote/JGroupsClusterActor.scala
diff --git a/akka-core/src/main/scala/remote/MessageSerializer.scala b/akka-remote/src/main/scala/remote/MessageSerializer.scala
similarity index 100%
rename from akka-core/src/main/scala/remote/MessageSerializer.scala
rename to akka-remote/src/main/scala/remote/MessageSerializer.scala
index 8ef6f5d590..49f38524f9 100644
--- a/akka-core/src/main/scala/remote/MessageSerializer.scala
+++ b/akka-remote/src/main/scala/remote/MessageSerializer.scala
@@ -6,9 +6,9 @@ package se.scalablesolutions.akka.remote
import se.scalablesolutions.akka.serialization.{Serializer, Serializable}
import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._
+import se.scalablesolutions.akka.util._
import com.google.protobuf.{Message, ByteString}
-import se.scalablesolutions.akka.util._
object MessageSerializer extends Logging {
private var SERIALIZER_JAVA: Serializer.Java = Serializer.Java
diff --git a/akka-core/src/main/scala/remote/RemoteClient.scala b/akka-remote/src/main/scala/remote/RemoteClient.scala
similarity index 92%
rename from akka-core/src/main/scala/remote/RemoteClient.scala
rename to akka-remote/src/main/scala/remote/RemoteClient.scala
index 47d9aa357a..35578477ff 100644
--- a/akka-core/src/main/scala/remote/RemoteClient.scala
+++ b/akka-remote/src/main/scala/remote/RemoteClient.scala
@@ -5,12 +5,14 @@
package se.scalablesolutions.akka.remote
import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._
-import se.scalablesolutions.akka.actor.{Exit, Actor, ActorRef, RemoteActorRef, IllegalActorStateException}
+import se.scalablesolutions.akka.actor.{Exit, Actor, ActorRef, ActorType, RemoteActorRef, RemoteActorSerialization, IllegalActorStateException}
import se.scalablesolutions.akka.dispatch.{DefaultCompletableFuture, CompletableFuture}
import se.scalablesolutions.akka.util.{ListenerManagement, UUID, Logging, Duration}
import se.scalablesolutions.akka.config.Config._
import se.scalablesolutions.akka.AkkaException
import Actor._
+import RemoteActorSerialization._
+
import org.jboss.netty.channel._
import group.DefaultChannelGroup
import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory
@@ -118,7 +120,8 @@ object RemoteClient extends Logging {
private[akka] def clientFor(hostname: String, port: Int, loader: Option[ClassLoader]): RemoteClient =
clientFor(new InetSocketAddress(hostname, port), loader)
- private[akka] def clientFor(address: InetSocketAddress, loader: Option[ClassLoader]): RemoteClient = synchronized {
+ private[akka] def clientFor(
+ address: InetSocketAddress, loader: Option[ClassLoader]): RemoteClient = synchronized {
val hostname = address.getHostName
val port = address.getPort
val hash = hostname + ':' + port
@@ -151,7 +154,7 @@ object RemoteClient extends Logging {
remoteClients.clear
}
- private[akka] def register(hostname: String, port: Int, uuid: String) = synchronized {
+ def register(hostname: String, port: Int, uuid: String) = synchronized {
actorsFor(RemoteServer.Address(hostname, port)) += uuid
}
@@ -177,7 +180,8 @@ object RemoteClient extends Logging {
*
* @author Jonas Bonér
*/
-class RemoteClient private[akka] (val hostname: String, val port: Int, val loader: Option[ClassLoader] = None)
+class RemoteClient private[akka] (
+ val hostname: String, val port: Int, val loader: Option[ClassLoader] = None)
extends Logging with ListenerManagement {
val name = "RemoteClient@" + hostname + "::" + port
@@ -190,13 +194,13 @@ class RemoteClient private[akka] (val hostname: String, val port: Int, val loade
//FIXME rewrite to a wrapper object (minimize volatile access and maximize encapsulation)
@volatile private[remote] var isRunning = false
@volatile private var bootstrap: ClientBootstrap = _
- @volatile private var reconnectionTimeWindowStart = 0L
@volatile private[remote] var connection: ChannelFuture = _
@volatile private[remote] var openChannels: DefaultChannelGroup = _
@volatile private var timer: HashedWheelTimer = _
- private val reconnectionTimeWindow =
- Duration(config.getInt("akka.remote.client.reconnection-time-window", 600), TIME_UNIT).toMillis
+ private val reconnectionTimeWindow = Duration(config.getInt(
+ "akka.remote.client.reconnection-time-window", 600), TIME_UNIT).toMillis
+ @volatile private var reconnectionTimeWindowStart = 0L
def connect = synchronized {
if (!isRunning) {
@@ -250,7 +254,24 @@ class RemoteClient private[akka] (val hostname: String, val port: Int, val loade
protected override def manageLifeCycleOfListeners = false
- def send[T](request: RemoteRequestProtocol, senderFuture: Option[CompletableFuture[T]]): Option[CompletableFuture[T]] = if (isRunning) {
+ def send[T](
+ message: Any,
+ senderOption: Option[ActorRef],
+ senderFuture: Option[CompletableFuture[T]],
+ remoteAddress: InetSocketAddress,
+ timeout: Long,
+ isOneWay: Boolean,
+ actorRef: ActorRef,
+ typedActorInfo: Option[Tuple2[String, String]],
+ actorType: ActorType): Option[CompletableFuture[T]] = {
+ send(createRemoteRequestProtocolBuilder(
+ actorRef, message, isOneWay, senderOption, typedActorInfo, actorType).build, senderFuture)
+ }
+
+ def send[T](
+ request: RemoteRequestProtocol,
+ senderFuture: Option[CompletableFuture[T]]):
+ Option[CompletableFuture[T]] = if (isRunning) {
if (request.getIsOneWay) {
connection.getChannel.write(request)
None
@@ -264,8 +285,9 @@ class RemoteClient private[akka] (val hostname: String, val port: Int, val loade
}
}
} else {
- val exception = new RemoteClientException("Remote client is not running, make sure you have invoked 'RemoteClient.connect' before using it.", this)
- foreachListener(_ ! RemoteClientError(exception, this))
+ val exception = new RemoteClientException(
+ "Remote client is not running, make sure you have invoked 'RemoteClient.connect' before using it.", this)
+ foreachListener(l => l ! RemoteClientError(exception, this))
throw exception
}
@@ -448,25 +470,3 @@ class RemoteClientHandler(
.newInstance(exception.getMessage).asInstanceOf[Throwable]
}
}
-
-object RemoteDisconnectTest {
-import se.scalablesolutions.akka.actor.{Actor,ActorRef}
-
- class TestClientActor extends Actor {
- def receive = {
- case ("send ping",akt:ActorRef) => akt ! "ping"
- case "pong" => {
- log.debug("got pong")
- }
- }
- }
-
- class TestServerActor extends Actor {
- def receive = {
- case "ping" => {
- log.debug("got ping")
- self reply "pong"
- }
- }
- }
-}
diff --git a/akka-core/src/main/scala/remote/RemoteServer.scala b/akka-remote/src/main/scala/remote/RemoteServer.scala
similarity index 98%
rename from akka-core/src/main/scala/remote/RemoteServer.scala
rename to akka-remote/src/main/scala/remote/RemoteServer.scala
index 7c9f73258c..2ce24b3fff 100644
--- a/akka-core/src/main/scala/remote/RemoteServer.scala
+++ b/akka-remote/src/main/scala/remote/RemoteServer.scala
@@ -13,6 +13,7 @@ import se.scalablesolutions.akka.actor._
import se.scalablesolutions.akka.actor.Actor._
import se.scalablesolutions.akka.util._
import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._
+import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType._
import se.scalablesolutions.akka.config.Config._
import org.jboss.netty.bootstrap.ServerBootstrap
@@ -63,7 +64,7 @@ object RemoteNode extends RemoteServer
*/
object RemoteServer {
val HOSTNAME = config.getString("akka.remote.server.hostname", "localhost")
- val PORT = config.getInt("akka.remote.server.port", 9999)
+ val PORT = config.getInt("akka.remote.server.port", 9999)
val CONNECTION_TIMEOUT_MILLIS = Duration(config.getInt("akka.remote.server.connection-timeout", 1), TIME_UNIT)
@@ -445,9 +446,9 @@ class RemoteServerHandler(
private def handleRemoteRequestProtocol(request: RemoteRequestProtocol, channel: Channel) = {
log.debug("Received RemoteRequestProtocol[\n%s]", request.toString)
val actorType = request.getActorInfo.getActorType
- if (actorType == ActorType.SCALA_ACTOR) dispatchToActor(request, channel)
- else if (actorType == ActorType.JAVA_ACTOR) throw new IllegalActorStateException("ActorType JAVA_ACTOR is currently not supported")
- else if (actorType == ActorType.TYPED_ACTOR) dispatchToTypedActor(request, channel)
+ if (actorType == SCALA_ACTOR) dispatchToActor(request, channel)
+ else if (actorType == JAVA_ACTOR) throw new IllegalActorStateException("ActorType JAVA_ACTOR is currently not supported")
+ else if (actorType == TYPED_ACTOR) dispatchToTypedActor(request, channel)
else throw new IllegalActorStateException("Unknown ActorType [" + actorType + "]")
}
diff --git a/akka-core/src/main/scala/serialization/Binary.scala b/akka-remote/src/main/scala/serialization/Binary.scala
similarity index 100%
rename from akka-core/src/main/scala/serialization/Binary.scala
rename to akka-remote/src/main/scala/serialization/Binary.scala
diff --git a/akka-core/src/main/scala/serialization/Compression.scala b/akka-remote/src/main/scala/serialization/Compression.scala
similarity index 82%
rename from akka-core/src/main/scala/serialization/Compression.scala
rename to akka-remote/src/main/scala/serialization/Compression.scala
index 5b8df9ada7..bbb8d95421 100644
--- a/akka-core/src/main/scala/serialization/Compression.scala
+++ b/akka-remote/src/main/scala/serialization/Compression.scala
@@ -14,8 +14,8 @@ object Compression {
*/
object LZF {
import voldemort.store.compress.lzf._
- def compress(bytes: Array[Byte]): Array[Byte] = LZFEncoder.encode(bytes)
- def uncompress(bytes: Array[Byte]): Array[Byte] = LZFDecoder.decode(bytes)
+ def compress(bytes: Array[Byte]): Array[Byte] = LZFEncoder encode bytes
+ def uncompress(bytes: Array[Byte]): Array[Byte] = LZFDecoder decode bytes
}
}
diff --git a/akka-core/src/main/scala/serialization/Serializable.scala b/akka-remote/src/main/scala/serialization/Serializable.scala
similarity index 100%
rename from akka-core/src/main/scala/serialization/Serializable.scala
rename to akka-remote/src/main/scala/serialization/Serializable.scala
diff --git a/akka-core/src/main/scala/actor/SerializationProtocol.scala b/akka-remote/src/main/scala/serialization/SerializationProtocol.scala
similarity index 81%
rename from akka-core/src/main/scala/actor/SerializationProtocol.scala
rename to akka-remote/src/main/scala/serialization/SerializationProtocol.scala
index b3f7caf8c2..3f54f8e921 100644
--- a/akka-core/src/main/scala/actor/SerializationProtocol.scala
+++ b/akka-remote/src/main/scala/serialization/SerializationProtocol.scala
@@ -11,6 +11,7 @@ import se.scalablesolutions.akka.stm.TransactionManagement._
import se.scalablesolutions.akka.stm.TransactionManagement
import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._
import se.scalablesolutions.akka.remote.{RemoteServer, RemoteRequestProtocolIdFactory, MessageSerializer}
+import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType._
import se.scalablesolutions.akka.serialization.Serializer
import com.google.protobuf.ByteString
@@ -84,7 +85,8 @@ object ActorSerialization {
def toBinaryJ[T <: Actor](a: ActorRef, format: Format[T]): Array[Byte] =
toBinary(a)(format)
- private def toSerializedActorRefProtocol[T <: Actor](actorRef: ActorRef, format: Format[T]): SerializedActorRefProtocol = {
+ private def toSerializedActorRefProtocol[T <: Actor](
+ actorRef: ActorRef, format: Format[T]): SerializedActorRefProtocol = {
val lifeCycleProtocol: Option[LifeCycleProtocol] = {
def setScope(builder: LifeCycleProtocol.Builder, scope: Scope) = scope match {
case Permanent => builder.setLifeCycle(LifeCycleType.PERMANENT)
@@ -152,11 +154,19 @@ object ActorSerialization {
.asInstanceOf[PartialFunction[Any, Unit]])
else None
+ val classLoader = loader.getOrElse(getClass.getClassLoader)
+
+ val factory = () => {
+ val actorClass = classLoader.loadClass(protocol.getActorClassname)
+ if (format.isInstanceOf[SerializerBasedActorFormat[_]])
+ format.asInstanceOf[SerializerBasedActorFormat[_]].serializer.fromBinary(
+ protocol.getActorInstance.toByteArray, Some(actorClass)).asInstanceOf[Actor]
+ else actorClass.newInstance.asInstanceOf[Actor]
+ }
+
val ar = new LocalActorRef(
protocol.getUuid,
protocol.getId,
- protocol.getActorClassname,
- protocol.getActorInstance.toByteArray,
protocol.getOriginalAddress.getHostname,
protocol.getOriginalAddress.getPort,
if (protocol.hasIsTransactor) protocol.getIsTransactor else false,
@@ -165,12 +175,15 @@ object ActorSerialization {
lifeCycle,
supervisor,
hotswap,
- loader.getOrElse(getClass.getClassLoader), // TODO: should we fall back to getClass.getClassLoader?
- protocol.getMessagesList.toArray.toList.asInstanceOf[List[RemoteRequestProtocol]], format)
+ classLoader, // TODO: should we fall back to getClass.getClassLoader?
+ factory)
- if (format.isInstanceOf[SerializerBasedActorFormat[_]] == false)
- format.fromBinary(protocol.getActorInstance.toByteArray, ar.actor.asInstanceOf[T])
- ar
+ val messages = protocol.getMessagesList.toArray.toList.asInstanceOf[List[RemoteRequestProtocol]]
+ messages.foreach(message => ar ! MessageSerializer.deserialize(message.getMessage))
+
+ if (format.isInstanceOf[SerializerBasedActorFormat[_]] == false)
+ format.fromBinary(protocol.getActorInstance.toByteArray, ar.actor.asInstanceOf[T])
+ ar
}
}
@@ -224,30 +237,48 @@ object RemoteActorSerialization {
.build
}
- def createRemoteRequestProtocolBuilder(actorRef: ActorRef, message: Any, isOneWay: Boolean, senderOption: Option[ActorRef]):
+ def createRemoteRequestProtocolBuilder(
+ actorRef: ActorRef,
+ message: Any,
+ isOneWay: Boolean,
+ senderOption: Option[ActorRef],
+ typedActorInfo: Option[Tuple2[String, String]],
+ actorType: ActorType):
RemoteRequestProtocol.Builder = {
import actorRef._
- val actorInfo = ActorInfoProtocol.newBuilder
+ val actorInfoBuilder = ActorInfoProtocol.newBuilder
.setUuid(uuid)
.setTarget(actorClassName)
.setTimeout(timeout)
- .setActorType(ActorType.SCALA_ACTOR)
- .build
- val request = RemoteRequestProtocol.newBuilder
+ typedActorInfo.foreach { typedActor =>
+ actorInfoBuilder.setTypedActorInfo(
+ TypedActorInfoProtocol.newBuilder
+ .setInterface(typedActor._1)
+ .setMethod(typedActor._2)
+ .build)
+ }
+
+ actorType match {
+ case ActorType.ScalaActor => actorInfoBuilder.setActorType(SCALA_ACTOR)
+ case ActorType.TypedActor => actorInfoBuilder.setActorType(TYPED_ACTOR)
+ }
+ val actorInfo = actorInfoBuilder.build
+
+ val requestBuilder = RemoteRequestProtocol.newBuilder
.setId(RemoteRequestProtocolIdFactory.nextId)
.setMessage(MessageSerializer.serialize(message))
.setActorInfo(actorInfo)
.setIsOneWay(isOneWay)
val id = registerSupervisorAsRemoteActor
- if (id.isDefined) request.setSupervisorUuid(id.get)
+ if (id.isDefined) requestBuilder.setSupervisorUuid(id.get)
senderOption.foreach { sender =>
RemoteServer.getOrCreateServer(sender.homeAddress).register(sender.uuid, sender)
- request.setSender(toRemoteActorRefProtocol(sender))
+ requestBuilder.setSender(toRemoteActorRefProtocol(sender))
}
- request
+ requestBuilder
}
}
diff --git a/akka-core/src/main/scala/serialization/Serializer.scala b/akka-remote/src/main/scala/serialization/Serializer.scala
similarity index 100%
rename from akka-core/src/main/scala/serialization/Serializer.scala
rename to akka-remote/src/main/scala/serialization/Serializer.scala
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/ProtobufProtocol.java b/akka-remote/src/test/java/se/scalablesolutions/akka/actor/ProtobufProtocol.java
similarity index 98%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/ProtobufProtocol.java
rename to akka-remote/src/test/java/se/scalablesolutions/akka/actor/ProtobufProtocol.java
index 683f008729..183d2025d0 100644
--- a/akka-core/src/test/java/se/scalablesolutions/akka/actor/ProtobufProtocol.java
+++ b/akka-remote/src/test/java/se/scalablesolutions/akka/actor/ProtobufProtocol.java
@@ -15,47 +15,47 @@ public final class ProtobufProtocol {
initFields();
}
private ProtobufPOJO(boolean noInit) {}
-
+
private static final ProtobufPOJO defaultInstance;
public static ProtobufPOJO getDefaultInstance() {
return defaultInstance;
}
-
+
public ProtobufPOJO getDefaultInstanceForType() {
return defaultInstance;
}
-
+
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return se.scalablesolutions.akka.actor.ProtobufProtocol.internal_static_se_scalablesolutions_akka_actor_ProtobufPOJO_descriptor;
}
-
+
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return se.scalablesolutions.akka.actor.ProtobufProtocol.internal_static_se_scalablesolutions_akka_actor_ProtobufPOJO_fieldAccessorTable;
}
-
+
// required uint64 id = 1;
public static final int ID_FIELD_NUMBER = 1;
private boolean hasId;
private long id_ = 0L;
public boolean hasId() { return hasId; }
public long getId() { return id_; }
-
+
// required string name = 2;
public static final int NAME_FIELD_NUMBER = 2;
private boolean hasName;
private java.lang.String name_ = "";
public boolean hasName() { return hasName; }
public java.lang.String getName() { return name_; }
-
+
// required bool status = 3;
public static final int STATUS_FIELD_NUMBER = 3;
private boolean hasStatus;
private boolean status_ = false;
public boolean hasStatus() { return hasStatus; }
public boolean getStatus() { return status_; }
-
+
private void initFields() {
}
public final boolean isInitialized() {
@@ -64,7 +64,7 @@ public final class ProtobufProtocol {
if (!hasStatus) return false;
return true;
}
-
+
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
@@ -79,12 +79,12 @@ public final class ProtobufProtocol {
}
getUnknownFields().writeTo(output);
}
-
+
private int memoizedSerializedSize = -1;
public int getSerializedSize() {
int size = memoizedSerializedSize;
if (size != -1) return size;
-
+
size = 0;
if (hasId()) {
size += com.google.protobuf.CodedOutputStream
@@ -102,7 +102,7 @@ public final class ProtobufProtocol {
memoizedSerializedSize = size;
return size;
}
-
+
public static se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO parseFrom(
com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException {
@@ -169,31 +169,31 @@ public final class ProtobufProtocol {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
-
+
public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO prototype) {
return newBuilder().mergeFrom(prototype);
}
public Builder toBuilder() { return newBuilder(this); }
-
+
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder {
private se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO result;
-
+
// Construct using se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO.newBuilder()
private Builder() {}
-
+
private static Builder create() {
Builder builder = new Builder();
builder.result = new se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO();
return builder;
}
-
+
protected se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO internalGetResult() {
return result;
}
-
+
public Builder clear() {
if (result == null) {
throw new IllegalStateException(
@@ -202,20 +202,20 @@ public final class ProtobufProtocol {
result = new se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO();
return this;
}
-
+
public Builder clone() {
return create().mergeFrom(result);
}
-
+
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
return se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO.getDescriptor();
}
-
+
public se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO getDefaultInstanceForType() {
return se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO.getDefaultInstance();
}
-
+
public boolean isInitialized() {
return result.isInitialized();
}
@@ -225,7 +225,7 @@ public final class ProtobufProtocol {
}
return buildPartial();
}
-
+
private se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException {
if (!isInitialized()) {
@@ -234,7 +234,7 @@ public final class ProtobufProtocol {
}
return buildPartial();
}
-
+
public se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO buildPartial() {
if (result == null) {
throw new IllegalStateException(
@@ -244,7 +244,7 @@ public final class ProtobufProtocol {
result = null;
return returnMe;
}
-
+
public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO) {
return mergeFrom((se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO)other);
@@ -253,7 +253,7 @@ public final class ProtobufProtocol {
return this;
}
}
-
+
public Builder mergeFrom(se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO other) {
if (other == se.scalablesolutions.akka.actor.ProtobufProtocol.ProtobufPOJO.getDefaultInstance()) return this;
if (other.hasId()) {
@@ -268,7 +268,7 @@ public final class ProtobufProtocol {
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
-
+
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
@@ -305,8 +305,8 @@ public final class ProtobufProtocol {
}
}
}
-
-
+
+
// required uint64 id = 1;
public boolean hasId() {
return result.hasId();
@@ -324,7 +324,7 @@ public final class ProtobufProtocol {
result.id_ = 0L;
return this;
}
-
+
// required string name = 2;
public boolean hasName() {
return result.hasName();
@@ -345,7 +345,7 @@ public final class ProtobufProtocol {
result.name_ = getDefaultInstance().getName();
return this;
}
-
+
// required bool status = 3;
public boolean hasStatus() {
return result.hasStatus();
@@ -363,19 +363,19 @@ public final class ProtobufProtocol {
result.status_ = false;
return this;
}
-
+
// @@protoc_insertion_point(builder_scope:se.scalablesolutions.akka.actor.ProtobufPOJO)
}
-
+
static {
defaultInstance = new ProtobufPOJO(true);
se.scalablesolutions.akka.actor.ProtobufProtocol.internalForceInit();
defaultInstance.initFields();
}
-
+
// @@protoc_insertion_point(class_scope:se.scalablesolutions.akka.actor.ProtobufPOJO)
}
-
+
public static final class Counter extends
com.google.protobuf.GeneratedMessage {
// Use Counter.newBuilder() to construct.
@@ -383,40 +383,40 @@ public final class ProtobufProtocol {
initFields();
}
private Counter(boolean noInit) {}
-
+
private static final Counter defaultInstance;
public static Counter getDefaultInstance() {
return defaultInstance;
}
-
+
public Counter getDefaultInstanceForType() {
return defaultInstance;
}
-
+
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return se.scalablesolutions.akka.actor.ProtobufProtocol.internal_static_se_scalablesolutions_akka_actor_Counter_descriptor;
}
-
+
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return se.scalablesolutions.akka.actor.ProtobufProtocol.internal_static_se_scalablesolutions_akka_actor_Counter_fieldAccessorTable;
}
-
+
// required uint32 count = 1;
public static final int COUNT_FIELD_NUMBER = 1;
private boolean hasCount;
private int count_ = 0;
public boolean hasCount() { return hasCount; }
public int getCount() { return count_; }
-
+
private void initFields() {
}
public final boolean isInitialized() {
if (!hasCount) return false;
return true;
}
-
+
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
@@ -425,12 +425,12 @@ public final class ProtobufProtocol {
}
getUnknownFields().writeTo(output);
}
-
+
private int memoizedSerializedSize = -1;
public int getSerializedSize() {
int size = memoizedSerializedSize;
if (size != -1) return size;
-
+
size = 0;
if (hasCount()) {
size += com.google.protobuf.CodedOutputStream
@@ -440,7 +440,7 @@ public final class ProtobufProtocol {
memoizedSerializedSize = size;
return size;
}
-
+
public static se.scalablesolutions.akka.actor.ProtobufProtocol.Counter parseFrom(
com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException {
@@ -507,31 +507,31 @@ public final class ProtobufProtocol {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
-
+
public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(se.scalablesolutions.akka.actor.ProtobufProtocol.Counter prototype) {
return newBuilder().mergeFrom(prototype);
}
public Builder toBuilder() { return newBuilder(this); }
-
+
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder {
private se.scalablesolutions.akka.actor.ProtobufProtocol.Counter result;
-
+
// Construct using se.scalablesolutions.akka.actor.ProtobufProtocol.Counter.newBuilder()
private Builder() {}
-
+
private static Builder create() {
Builder builder = new Builder();
builder.result = new se.scalablesolutions.akka.actor.ProtobufProtocol.Counter();
return builder;
}
-
+
protected se.scalablesolutions.akka.actor.ProtobufProtocol.Counter internalGetResult() {
return result;
}
-
+
public Builder clear() {
if (result == null) {
throw new IllegalStateException(
@@ -540,20 +540,20 @@ public final class ProtobufProtocol {
result = new se.scalablesolutions.akka.actor.ProtobufProtocol.Counter();
return this;
}
-
+
public Builder clone() {
return create().mergeFrom(result);
}
-
+
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
return se.scalablesolutions.akka.actor.ProtobufProtocol.Counter.getDescriptor();
}
-
+
public se.scalablesolutions.akka.actor.ProtobufProtocol.Counter getDefaultInstanceForType() {
return se.scalablesolutions.akka.actor.ProtobufProtocol.Counter.getDefaultInstance();
}
-
+
public boolean isInitialized() {
return result.isInitialized();
}
@@ -563,7 +563,7 @@ public final class ProtobufProtocol {
}
return buildPartial();
}
-
+
private se.scalablesolutions.akka.actor.ProtobufProtocol.Counter buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException {
if (!isInitialized()) {
@@ -572,7 +572,7 @@ public final class ProtobufProtocol {
}
return buildPartial();
}
-
+
public se.scalablesolutions.akka.actor.ProtobufProtocol.Counter buildPartial() {
if (result == null) {
throw new IllegalStateException(
@@ -582,7 +582,7 @@ public final class ProtobufProtocol {
result = null;
return returnMe;
}
-
+
public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof se.scalablesolutions.akka.actor.ProtobufProtocol.Counter) {
return mergeFrom((se.scalablesolutions.akka.actor.ProtobufProtocol.Counter)other);
@@ -591,7 +591,7 @@ public final class ProtobufProtocol {
return this;
}
}
-
+
public Builder mergeFrom(se.scalablesolutions.akka.actor.ProtobufProtocol.Counter other) {
if (other == se.scalablesolutions.akka.actor.ProtobufProtocol.Counter.getDefaultInstance()) return this;
if (other.hasCount()) {
@@ -600,7 +600,7 @@ public final class ProtobufProtocol {
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
-
+
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
@@ -629,8 +629,8 @@ public final class ProtobufProtocol {
}
}
}
-
-
+
+
// required uint32 count = 1;
public boolean hasCount() {
return result.hasCount();
@@ -648,19 +648,19 @@ public final class ProtobufProtocol {
result.count_ = 0;
return this;
}
-
+
// @@protoc_insertion_point(builder_scope:se.scalablesolutions.akka.actor.Counter)
}
-
+
static {
defaultInstance = new Counter(true);
se.scalablesolutions.akka.actor.ProtobufProtocol.internalForceInit();
defaultInstance.initFields();
}
-
+
// @@protoc_insertion_point(class_scope:se.scalablesolutions.akka.actor.Counter)
}
-
+
public static final class DualCounter extends
com.google.protobuf.GeneratedMessage {
// Use DualCounter.newBuilder() to construct.
@@ -668,40 +668,40 @@ public final class ProtobufProtocol {
initFields();
}
private DualCounter(boolean noInit) {}
-
+
private static final DualCounter defaultInstance;
public static DualCounter getDefaultInstance() {
return defaultInstance;
}
-
+
public DualCounter getDefaultInstanceForType() {
return defaultInstance;
}
-
+
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return se.scalablesolutions.akka.actor.ProtobufProtocol.internal_static_se_scalablesolutions_akka_actor_DualCounter_descriptor;
}
-
+
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return se.scalablesolutions.akka.actor.ProtobufProtocol.internal_static_se_scalablesolutions_akka_actor_DualCounter_fieldAccessorTable;
}
-
+
// required uint32 count1 = 1;
public static final int COUNT1_FIELD_NUMBER = 1;
private boolean hasCount1;
private int count1_ = 0;
public boolean hasCount1() { return hasCount1; }
public int getCount1() { return count1_; }
-
+
// required uint32 count2 = 2;
public static final int COUNT2_FIELD_NUMBER = 2;
private boolean hasCount2;
private int count2_ = 0;
public boolean hasCount2() { return hasCount2; }
public int getCount2() { return count2_; }
-
+
private void initFields() {
}
public final boolean isInitialized() {
@@ -709,7 +709,7 @@ public final class ProtobufProtocol {
if (!hasCount2) return false;
return true;
}
-
+
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
@@ -721,12 +721,12 @@ public final class ProtobufProtocol {
}
getUnknownFields().writeTo(output);
}
-
+
private int memoizedSerializedSize = -1;
public int getSerializedSize() {
int size = memoizedSerializedSize;
if (size != -1) return size;
-
+
size = 0;
if (hasCount1()) {
size += com.google.protobuf.CodedOutputStream
@@ -740,7 +740,7 @@ public final class ProtobufProtocol {
memoizedSerializedSize = size;
return size;
}
-
+
public static se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter parseFrom(
com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException {
@@ -807,31 +807,31 @@ public final class ProtobufProtocol {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
-
+
public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter prototype) {
return newBuilder().mergeFrom(prototype);
}
public Builder toBuilder() { return newBuilder(this); }
-
+
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder {
private se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter result;
-
+
// Construct using se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter.newBuilder()
private Builder() {}
-
+
private static Builder create() {
Builder builder = new Builder();
builder.result = new se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter();
return builder;
}
-
+
protected se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter internalGetResult() {
return result;
}
-
+
public Builder clear() {
if (result == null) {
throw new IllegalStateException(
@@ -840,20 +840,20 @@ public final class ProtobufProtocol {
result = new se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter();
return this;
}
-
+
public Builder clone() {
return create().mergeFrom(result);
}
-
+
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
return se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter.getDescriptor();
}
-
+
public se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter getDefaultInstanceForType() {
return se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter.getDefaultInstance();
}
-
+
public boolean isInitialized() {
return result.isInitialized();
}
@@ -863,7 +863,7 @@ public final class ProtobufProtocol {
}
return buildPartial();
}
-
+
private se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException {
if (!isInitialized()) {
@@ -872,7 +872,7 @@ public final class ProtobufProtocol {
}
return buildPartial();
}
-
+
public se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter buildPartial() {
if (result == null) {
throw new IllegalStateException(
@@ -882,7 +882,7 @@ public final class ProtobufProtocol {
result = null;
return returnMe;
}
-
+
public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter) {
return mergeFrom((se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter)other);
@@ -891,7 +891,7 @@ public final class ProtobufProtocol {
return this;
}
}
-
+
public Builder mergeFrom(se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter other) {
if (other == se.scalablesolutions.akka.actor.ProtobufProtocol.DualCounter.getDefaultInstance()) return this;
if (other.hasCount1()) {
@@ -903,7 +903,7 @@ public final class ProtobufProtocol {
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
-
+
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
@@ -936,8 +936,8 @@ public final class ProtobufProtocol {
}
}
}
-
-
+
+
// required uint32 count1 = 1;
public boolean hasCount1() {
return result.hasCount1();
@@ -955,7 +955,7 @@ public final class ProtobufProtocol {
result.count1_ = 0;
return this;
}
-
+
// required uint32 count2 = 2;
public boolean hasCount2() {
return result.hasCount2();
@@ -973,19 +973,19 @@ public final class ProtobufProtocol {
result.count2_ = 0;
return this;
}
-
+
// @@protoc_insertion_point(builder_scope:se.scalablesolutions.akka.actor.DualCounter)
}
-
+
static {
defaultInstance = new DualCounter(true);
se.scalablesolutions.akka.actor.ProtobufProtocol.internalForceInit();
defaultInstance.initFields();
}
-
+
// @@protoc_insertion_point(class_scope:se.scalablesolutions.akka.actor.DualCounter)
}
-
+
private static com.google.protobuf.Descriptors.Descriptor
internal_static_se_scalablesolutions_akka_actor_ProtobufPOJO_descriptor;
private static
@@ -1001,7 +1001,7 @@ public final class ProtobufProtocol {
private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_se_scalablesolutions_akka_actor_DualCounter_fieldAccessorTable;
-
+
public static com.google.protobuf.Descriptors.FileDescriptor
getDescriptor() {
return descriptor;
@@ -1053,8 +1053,8 @@ public final class ProtobufProtocol {
new com.google.protobuf.Descriptors.FileDescriptor[] {
}, assigner);
}
-
+
public static void internalForceInit() {}
-
+
// @@protoc_insertion_point(outer_class_scope)
}
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOne.java b/akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOne.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOne.java
rename to akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOne.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOneImpl.java b/akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOneImpl.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOneImpl.java
rename to akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOneImpl.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwo.java b/akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwo.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwo.java
rename to akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwo.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwoImpl.java b/akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwoImpl.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwoImpl.java
rename to akka-remote/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwoImpl.java
diff --git a/akka-core/src/main/java/se/scalablesolutions/akka/config/DependencyBinding.java b/akka-remote/src/test/java/se/scalablesolutions/akka/config/DependencyBinding.java
similarity index 100%
rename from akka-core/src/main/java/se/scalablesolutions/akka/config/DependencyBinding.java
rename to akka-remote/src/test/java/se/scalablesolutions/akka/config/DependencyBinding.java
diff --git a/akka-core/src/main/java/se/scalablesolutions/akka/config/TypedActorGuiceModule.java b/akka-remote/src/test/java/se/scalablesolutions/akka/config/TypedActorGuiceModule.java
similarity index 100%
rename from akka-core/src/main/java/se/scalablesolutions/akka/config/TypedActorGuiceModule.java
rename to akka-remote/src/test/java/se/scalablesolutions/akka/config/TypedActorGuiceModule.java
diff --git a/akka-core/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java b/akka-remote/src/test/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java
similarity index 100%
rename from akka-core/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java
rename to akka-remote/src/test/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java
diff --git a/akka-core/src/test/protocol/ProtobufProtocol.proto b/akka-remote/src/test/protocol/ProtobufProtocol.proto
similarity index 92%
rename from akka-core/src/test/protocol/ProtobufProtocol.proto
rename to akka-remote/src/test/protocol/ProtobufProtocol.proto
index 35ffec95e3..a63ebdff62 100644
--- a/akka-core/src/test/protocol/ProtobufProtocol.proto
+++ b/akka-remote/src/test/protocol/ProtobufProtocol.proto
@@ -6,7 +6,7 @@ package se.scalablesolutions.akka.actor;
/*
Compile with:
- cd ./akka-core/src/test/protocol
+ cd ./akka-remote/src/test/protocol
protoc ProtobufProtocol.proto --java_out ../java
*/
diff --git a/akka-core/src/main/resources/META-INF/aop.xml b/akka-remote/src/test/resources/META-INF/aop.xml
similarity index 100%
rename from akka-core/src/main/resources/META-INF/aop.xml
rename to akka-remote/src/test/resources/META-INF/aop.xml
diff --git a/akka-remote/src/test/resources/logback-test.xml b/akka-remote/src/test/resources/logback-test.xml
new file mode 100644
index 0000000000..78eae40ec4
--- /dev/null
+++ b/akka-remote/src/test/resources/logback-test.xml
@@ -0,0 +1,21 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+ [%4p] [%d{ISO8601}] [%t] %c{1}: %m%n
+
+
+
+
+
+
+
diff --git a/akka-core/src/test/scala/Messages.scala b/akka-remote/src/test/scala/Messages.scala
similarity index 83%
rename from akka-core/src/test/scala/Messages.scala
rename to akka-remote/src/test/scala/Messages.scala
index ad1fcf8885..2b99155626 100644
--- a/akka-core/src/test/scala/Messages.scala
+++ b/akka-remote/src/test/scala/Messages.scala
@@ -8,14 +8,6 @@ import se.scalablesolutions.akka.serialization.Serializable
import sbinary._
import sbinary.Operations._
-sealed abstract class TestMessage
-
-case object Ping extends TestMessage
-case object Pong extends TestMessage
-case object OneWay extends TestMessage
-case object Die extends TestMessage
-case object NotifySupervisorExit extends TestMessage
-
case class User(val usernamePassword: Tuple2[String, String],
val email: String,
val age: Int)
diff --git a/akka-core/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala b/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala
rename to akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala
diff --git a/akka-core/src/test/scala/remote/RemoteSupervisorSpec.scala b/akka-remote/src/test/scala/remote/RemoteSupervisorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/remote/RemoteSupervisorSpec.scala
rename to akka-remote/src/test/scala/remote/RemoteSupervisorSpec.scala
diff --git a/akka-core/src/test/scala/remote/RemoteTransactionalTypedActorSpec.scala b/akka-remote/src/test/scala/remote/RemoteTransactionalTypedActorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/remote/RemoteTransactionalTypedActorSpec.scala
rename to akka-remote/src/test/scala/remote/RemoteTransactionalTypedActorSpec.scala
diff --git a/akka-core/src/test/scala/remote/RemoteTypedActorSpec.scala b/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/remote/RemoteTypedActorSpec.scala
rename to akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala
diff --git a/akka-core/src/test/scala/remote/ServerInitiatedRemoteActorSample.scala b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSample.scala
similarity index 100%
rename from akka-core/src/test/scala/remote/ServerInitiatedRemoteActorSample.scala
rename to akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSample.scala
diff --git a/akka-core/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala
rename to akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala
diff --git a/akka-core/src/test/scala/remote/ShutdownSpec.scala b/akka-remote/src/test/scala/remote/ShutdownSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/remote/ShutdownSpec.scala
rename to akka-remote/src/test/scala/remote/ShutdownSpec.scala
diff --git a/akka-core/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala b/akka-remote/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala
rename to akka-remote/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala
diff --git a/akka-core/src/test/scala/serialization/SerializableTypeClassActorSpec.scala b/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/serialization/SerializableTypeClassActorSpec.scala
rename to akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala
diff --git a/akka-core/src/test/scala/serialization/SerializerSpec.scala b/akka-remote/src/test/scala/serialization/SerializerSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/serialization/SerializerSpec.scala
rename to akka-remote/src/test/scala/serialization/SerializerSpec.scala
diff --git a/akka-core/src/test/scala/serialization/UntypedActorSerializationSpec.scala b/akka-remote/src/test/scala/serialization/UntypedActorSerializationSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/serialization/UntypedActorSerializationSpec.scala
rename to akka-remote/src/test/scala/serialization/UntypedActorSerializationSpec.scala
diff --git a/akka-remote/src/test/scala/ticket/Ticket001Spec.scala b/akka-remote/src/test/scala/ticket/Ticket001Spec.scala
new file mode 100644
index 0000000000..b94796d9a3
--- /dev/null
+++ b/akka-remote/src/test/scala/ticket/Ticket001Spec.scala
@@ -0,0 +1,13 @@
+package se.scalablesolutions.akka.actor.ticket
+
+import org.scalatest.WordSpec
+import org.scalatest.matchers.MustMatchers
+
+class Ticket001Spec extends WordSpec with MustMatchers {
+
+ "An XXX" should {
+ "do YYY" in {
+ 1 must be (1)
+ }
+ }
+}
diff --git a/akka-samples/akka-sample-chat/Buildfile b/akka-samples/akka-sample-chat/Buildfile
index fecc093964..814e6e4149 100644
--- a/akka-samples/akka-sample-chat/Buildfile
+++ b/akka-samples/akka-sample-chat/Buildfile
@@ -7,7 +7,7 @@ repositories.remote << "http://www.ibiblio.org/maven2/"
repositories.remote << "http://www.lag.net/repo"
repositories.remote << "http://multiverse.googlecode.com/svn/maven-repository/releases"
-AKKA = group('akka-core', 'akka-comet', 'akka-util','akka-kernel', 'akka-rest', 'akka-util-java',
+AKKA = group('akka-remote', 'akka-comet', 'akka-util','akka-kernel', 'akka-rest', 'akka-util-java',
'akka-security','akka-persistence-common', 'akka-persistence-redis',
'akka-amqp',
:under=> 'se.scalablesolutions.akka',
diff --git a/akka-sbt-plugin/src/main/scala/AkkaProject.scala b/akka-sbt-plugin/src/main/scala/AkkaProject.scala
index 10b69722e0..2bde073df8 100644
--- a/akka-sbt-plugin/src/main/scala/AkkaProject.scala
+++ b/akka-sbt-plugin/src/main/scala/AkkaProject.scala
@@ -50,6 +50,6 @@ trait AkkaProject extends AkkaBaseProject {
// convenience method
def akkaModule(module: String) = "se.scalablesolutions.akka" %% ("akka-" + module) % akkaVersion
- // akka core dependency by default
- val akkaCore = akkaModule("core")
+ // akka remote dependency by default
+ val akkaRemote = akkaModule("remote")
}
diff --git a/akka-typed-actor/src/main/java/se/scalablesolutions/akka/config/DependencyBinding.java b/akka-typed-actor/src/main/java/se/scalablesolutions/akka/config/DependencyBinding.java
new file mode 100644
index 0000000000..2956e6860f
--- /dev/null
+++ b/akka-typed-actor/src/main/java/se/scalablesolutions/akka/config/DependencyBinding.java
@@ -0,0 +1,24 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package se.scalablesolutions.akka.config;
+
+/**
+ * @author Jonas Bonér
+ */
+public class DependencyBinding {
+ private final Class intf;
+ private final Object target;
+
+ public DependencyBinding(final Class intf, final Object target) {
+ this.intf = intf;
+ this.target = target;
+ }
+ public Class getInterface() {
+ return intf;
+ }
+ public Object getTarget() {
+ return target;
+ }
+}
diff --git a/akka-typed-actor/src/main/java/se/scalablesolutions/akka/config/TypedActorGuiceModule.java b/akka-typed-actor/src/main/java/se/scalablesolutions/akka/config/TypedActorGuiceModule.java
new file mode 100644
index 0000000000..0c2ed11402
--- /dev/null
+++ b/akka-typed-actor/src/main/java/se/scalablesolutions/akka/config/TypedActorGuiceModule.java
@@ -0,0 +1,32 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package se.scalablesolutions.akka.config;
+
+import java.util.List;
+
+import com.google.inject.AbstractModule;
+import com.google.inject.Singleton;
+//import com.google.inject.jsr250.ResourceProviderFactory;
+
+/**
+ * @author Jonas Bonér
+ */
+public class TypedActorGuiceModule extends AbstractModule {
+ private final List bindings;
+
+ public TypedActorGuiceModule(final List bindings) {
+ this.bindings = bindings;
+ }
+
+ protected void configure() {
+ //bind(ResourceProviderFactory.class);
+ for (int i = 0; i < bindings.size(); i++) {
+ final DependencyBinding db = bindings.get(i);
+ //if (db.getInterface() ne null) bind((Class) db.getInterface()).to((Class) db.getTarget()).in(Singleton.class);
+ //else
+ this.bind(db.getInterface()).toInstance(db.getTarget());
+ }
+ }
+}
diff --git a/akka-typed-actor/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java b/akka-typed-actor/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java
new file mode 100644
index 0000000000..0ab1a0aa10
--- /dev/null
+++ b/akka-typed-actor/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java
@@ -0,0 +1,5190 @@
+// Generated by the protocol buffer compiler. DO NOT EDIT!
+// source: RemoteProtocol.proto
+
+package se.scalablesolutions.akka.remote.protocol;
+
+public final class RemoteProtocol {
+ private RemoteProtocol() {}
+ public static void registerAllExtensions(
+ com.google.protobuf.ExtensionRegistry registry) {
+ }
+ public enum ActorType
+ implements com.google.protobuf.ProtocolMessageEnum {
+ SCALA_ACTOR(0, 1),
+ JAVA_ACTOR(1, 2),
+ TYPED_ACTOR(2, 3),
+ ;
+
+
+ public final int getNumber() { return value; }
+
+ public static ActorType valueOf(int value) {
+ switch (value) {
+ case 1: return SCALA_ACTOR;
+ case 2: return JAVA_ACTOR;
+ case 3: return TYPED_ACTOR;
+ default: return null;
+ }
+ }
+
+ public static com.google.protobuf.Internal.EnumLiteMap
+ internalGetValueMap() {
+ return internalValueMap;
+ }
+ private static com.google.protobuf.Internal.EnumLiteMap
+ internalValueMap =
+ new com.google.protobuf.Internal.EnumLiteMap() {
+ public ActorType findValueByNumber(int number) {
+ return ActorType.valueOf(number)
+ ; }
+ };
+
+ public final com.google.protobuf.Descriptors.EnumValueDescriptor
+ getValueDescriptor() {
+ return getDescriptor().getValues().get(index);
+ }
+ public final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptorForType() {
+ return getDescriptor();
+ }
+ public static final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptor() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(0);
+ }
+
+ private static final ActorType[] VALUES = {
+ SCALA_ACTOR, JAVA_ACTOR, TYPED_ACTOR,
+ };
+ public static ActorType valueOf(
+ com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+ if (desc.getType() != getDescriptor()) {
+ throw new java.lang.IllegalArgumentException(
+ "EnumValueDescriptor is not for this type.");
+ }
+ return VALUES[desc.getIndex()];
+ }
+ private final int index;
+ private final int value;
+ private ActorType(int index, int value) {
+ this.index = index;
+ this.value = value;
+ }
+
+ static {
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor();
+ }
+
+ // @@protoc_insertion_point(enum_scope:ActorType)
+ }
+
+ public enum SerializationSchemeType
+ implements com.google.protobuf.ProtocolMessageEnum {
+ JAVA(0, 1),
+ SBINARY(1, 2),
+ SCALA_JSON(2, 3),
+ JAVA_JSON(3, 4),
+ PROTOBUF(4, 5),
+ ;
+
+
+ public final int getNumber() { return value; }
+
+ public static SerializationSchemeType valueOf(int value) {
+ switch (value) {
+ case 1: return JAVA;
+ case 2: return SBINARY;
+ case 3: return SCALA_JSON;
+ case 4: return JAVA_JSON;
+ case 5: return PROTOBUF;
+ default: return null;
+ }
+ }
+
+ public static com.google.protobuf.Internal.EnumLiteMap
+ internalGetValueMap() {
+ return internalValueMap;
+ }
+ private static com.google.protobuf.Internal.EnumLiteMap
+ internalValueMap =
+ new com.google.protobuf.Internal.EnumLiteMap() {
+ public SerializationSchemeType findValueByNumber(int number) {
+ return SerializationSchemeType.valueOf(number)
+ ; }
+ };
+
+ public final com.google.protobuf.Descriptors.EnumValueDescriptor
+ getValueDescriptor() {
+ return getDescriptor().getValues().get(index);
+ }
+ public final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptorForType() {
+ return getDescriptor();
+ }
+ public static final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptor() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(1);
+ }
+
+ private static final SerializationSchemeType[] VALUES = {
+ JAVA, SBINARY, SCALA_JSON, JAVA_JSON, PROTOBUF,
+ };
+ public static SerializationSchemeType valueOf(
+ com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+ if (desc.getType() != getDescriptor()) {
+ throw new java.lang.IllegalArgumentException(
+ "EnumValueDescriptor is not for this type.");
+ }
+ return VALUES[desc.getIndex()];
+ }
+ private final int index;
+ private final int value;
+ private SerializationSchemeType(int index, int value) {
+ this.index = index;
+ this.value = value;
+ }
+
+ static {
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor();
+ }
+
+ // @@protoc_insertion_point(enum_scope:SerializationSchemeType)
+ }
+
+ public enum LifeCycleType
+ implements com.google.protobuf.ProtocolMessageEnum {
+ PERMANENT(0, 1),
+ TEMPORARY(1, 2),
+ ;
+
+
+ public final int getNumber() { return value; }
+
+ public static LifeCycleType valueOf(int value) {
+ switch (value) {
+ case 1: return PERMANENT;
+ case 2: return TEMPORARY;
+ default: return null;
+ }
+ }
+
+ public static com.google.protobuf.Internal.EnumLiteMap
+ internalGetValueMap() {
+ return internalValueMap;
+ }
+ private static com.google.protobuf.Internal.EnumLiteMap
+ internalValueMap =
+ new com.google.protobuf.Internal.EnumLiteMap() {
+ public LifeCycleType findValueByNumber(int number) {
+ return LifeCycleType.valueOf(number)
+ ; }
+ };
+
+ public final com.google.protobuf.Descriptors.EnumValueDescriptor
+ getValueDescriptor() {
+ return getDescriptor().getValues().get(index);
+ }
+ public final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptorForType() {
+ return getDescriptor();
+ }
+ public static final com.google.protobuf.Descriptors.EnumDescriptor
+ getDescriptor() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(2);
+ }
+
+ private static final LifeCycleType[] VALUES = {
+ PERMANENT, TEMPORARY,
+ };
+ public static LifeCycleType valueOf(
+ com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+ if (desc.getType() != getDescriptor()) {
+ throw new java.lang.IllegalArgumentException(
+ "EnumValueDescriptor is not for this type.");
+ }
+ return VALUES[desc.getIndex()];
+ }
+ private final int index;
+ private final int value;
+ private LifeCycleType(int index, int value) {
+ this.index = index;
+ this.value = value;
+ }
+
+ static {
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor();
+ }
+
+ // @@protoc_insertion_point(enum_scope:LifeCycleType)
+ }
+
+ public static final class RemoteActorRefProtocol extends
+ com.google.protobuf.GeneratedMessage {
+ // Use RemoteActorRefProtocol.newBuilder() to construct.
+ private RemoteActorRefProtocol() {
+ initFields();
+ }
+ private RemoteActorRefProtocol(boolean noInit) {}
+
+ private static final RemoteActorRefProtocol defaultInstance;
+ public static RemoteActorRefProtocol getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public RemoteActorRefProtocol getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteActorRefProtocol_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteActorRefProtocol_fieldAccessorTable;
+ }
+
+ // required string uuid = 1;
+ public static final int UUID_FIELD_NUMBER = 1;
+ private boolean hasUuid;
+ private java.lang.String uuid_ = "";
+ public boolean hasUuid() { return hasUuid; }
+ public java.lang.String getUuid() { return uuid_; }
+
+ // required string actorClassname = 2;
+ public static final int ACTORCLASSNAME_FIELD_NUMBER = 2;
+ private boolean hasActorClassname;
+ private java.lang.String actorClassname_ = "";
+ public boolean hasActorClassname() { return hasActorClassname; }
+ public java.lang.String getActorClassname() { return actorClassname_; }
+
+ // required .AddressProtocol homeAddress = 3;
+ public static final int HOMEADDRESS_FIELD_NUMBER = 3;
+ private boolean hasHomeAddress;
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol homeAddress_;
+ public boolean hasHomeAddress() { return hasHomeAddress; }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getHomeAddress() { return homeAddress_; }
+
+ // optional uint64 timeout = 4;
+ public static final int TIMEOUT_FIELD_NUMBER = 4;
+ private boolean hasTimeout;
+ private long timeout_ = 0L;
+ public boolean hasTimeout() { return hasTimeout; }
+ public long getTimeout() { return timeout_; }
+
+ private void initFields() {
+ homeAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance();
+ }
+ public final boolean isInitialized() {
+ if (!hasUuid) return false;
+ if (!hasActorClassname) return false;
+ if (!hasHomeAddress) return false;
+ if (!getHomeAddress().isInitialized()) return false;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasUuid()) {
+ output.writeString(1, getUuid());
+ }
+ if (hasActorClassname()) {
+ output.writeString(2, getActorClassname());
+ }
+ if (hasHomeAddress()) {
+ output.writeMessage(3, getHomeAddress());
+ }
+ if (hasTimeout()) {
+ output.writeUInt64(4, getTimeout());
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasUuid()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(1, getUuid());
+ }
+ if (hasActorClassname()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(2, getActorClassname());
+ }
+ if (hasHomeAddress()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(3, getHomeAddress());
+ }
+ if (hasTimeout()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(4, getTimeout());
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder {
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol result;
+
+ // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol();
+ return builder;
+ }
+
+ protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol internalGetResult() {
+ return result;
+ }
+
+ public Builder clear() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "Cannot call clear() after build().");
+ }
+ result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDescriptor();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getDefaultInstanceForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol) {
+ return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol other) {
+ if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) return this;
+ if (other.hasUuid()) {
+ setUuid(other.getUuid());
+ }
+ if (other.hasActorClassname()) {
+ setActorClassname(other.getActorClassname());
+ }
+ if (other.hasHomeAddress()) {
+ mergeHomeAddress(other.getHomeAddress());
+ }
+ if (other.hasTimeout()) {
+ setTimeout(other.getTimeout());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder(
+ this.getUnknownFields());
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ }
+ break;
+ }
+ case 10: {
+ setUuid(input.readString());
+ break;
+ }
+ case 18: {
+ setActorClassname(input.readString());
+ break;
+ }
+ case 26: {
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder();
+ if (hasHomeAddress()) {
+ subBuilder.mergeFrom(getHomeAddress());
+ }
+ input.readMessage(subBuilder, extensionRegistry);
+ setHomeAddress(subBuilder.buildPartial());
+ break;
+ }
+ case 32: {
+ setTimeout(input.readUInt64());
+ break;
+ }
+ }
+ }
+ }
+
+
+ // required string uuid = 1;
+ public boolean hasUuid() {
+ return result.hasUuid();
+ }
+ public java.lang.String getUuid() {
+ return result.getUuid();
+ }
+ public Builder setUuid(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasUuid = true;
+ result.uuid_ = value;
+ return this;
+ }
+ public Builder clearUuid() {
+ result.hasUuid = false;
+ result.uuid_ = getDefaultInstance().getUuid();
+ return this;
+ }
+
+ // required string actorClassname = 2;
+ public boolean hasActorClassname() {
+ return result.hasActorClassname();
+ }
+ public java.lang.String getActorClassname() {
+ return result.getActorClassname();
+ }
+ public Builder setActorClassname(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasActorClassname = true;
+ result.actorClassname_ = value;
+ return this;
+ }
+ public Builder clearActorClassname() {
+ result.hasActorClassname = false;
+ result.actorClassname_ = getDefaultInstance().getActorClassname();
+ return this;
+ }
+
+ // required .AddressProtocol homeAddress = 3;
+ public boolean hasHomeAddress() {
+ return result.hasHomeAddress();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getHomeAddress() {
+ return result.getHomeAddress();
+ }
+ public Builder setHomeAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasHomeAddress = true;
+ result.homeAddress_ = value;
+ return this;
+ }
+ public Builder setHomeAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder builderForValue) {
+ result.hasHomeAddress = true;
+ result.homeAddress_ = builderForValue.build();
+ return this;
+ }
+ public Builder mergeHomeAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol value) {
+ if (result.hasHomeAddress() &&
+ result.homeAddress_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance()) {
+ result.homeAddress_ =
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder(result.homeAddress_).mergeFrom(value).buildPartial();
+ } else {
+ result.homeAddress_ = value;
+ }
+ result.hasHomeAddress = true;
+ return this;
+ }
+ public Builder clearHomeAddress() {
+ result.hasHomeAddress = false;
+ result.homeAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance();
+ return this;
+ }
+
+ // optional uint64 timeout = 4;
+ public boolean hasTimeout() {
+ return result.hasTimeout();
+ }
+ public long getTimeout() {
+ return result.getTimeout();
+ }
+ public Builder setTimeout(long value) {
+ result.hasTimeout = true;
+ result.timeout_ = value;
+ return this;
+ }
+ public Builder clearTimeout() {
+ result.hasTimeout = false;
+ result.timeout_ = 0L;
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:RemoteActorRefProtocol)
+ }
+
+ static {
+ defaultInstance = new RemoteActorRefProtocol(true);
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:RemoteActorRefProtocol)
+ }
+
+ public static final class SerializedActorRefProtocol extends
+ com.google.protobuf.GeneratedMessage {
+ // Use SerializedActorRefProtocol.newBuilder() to construct.
+ private SerializedActorRefProtocol() {
+ initFields();
+ }
+ private SerializedActorRefProtocol(boolean noInit) {}
+
+ private static final SerializedActorRefProtocol defaultInstance;
+ public static SerializedActorRefProtocol getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public SerializedActorRefProtocol getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_SerializedActorRefProtocol_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_SerializedActorRefProtocol_fieldAccessorTable;
+ }
+
+ // required string uuid = 1;
+ public static final int UUID_FIELD_NUMBER = 1;
+ private boolean hasUuid;
+ private java.lang.String uuid_ = "";
+ public boolean hasUuid() { return hasUuid; }
+ public java.lang.String getUuid() { return uuid_; }
+
+ // required string id = 2;
+ public static final int ID_FIELD_NUMBER = 2;
+ private boolean hasId;
+ private java.lang.String id_ = "";
+ public boolean hasId() { return hasId; }
+ public java.lang.String getId() { return id_; }
+
+ // required string actorClassname = 3;
+ public static final int ACTORCLASSNAME_FIELD_NUMBER = 3;
+ private boolean hasActorClassname;
+ private java.lang.String actorClassname_ = "";
+ public boolean hasActorClassname() { return hasActorClassname; }
+ public java.lang.String getActorClassname() { return actorClassname_; }
+
+ // required .AddressProtocol originalAddress = 4;
+ public static final int ORIGINALADDRESS_FIELD_NUMBER = 4;
+ private boolean hasOriginalAddress;
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol originalAddress_;
+ public boolean hasOriginalAddress() { return hasOriginalAddress; }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getOriginalAddress() { return originalAddress_; }
+
+ // optional bytes actorInstance = 5;
+ public static final int ACTORINSTANCE_FIELD_NUMBER = 5;
+ private boolean hasActorInstance;
+ private com.google.protobuf.ByteString actorInstance_ = com.google.protobuf.ByteString.EMPTY;
+ public boolean hasActorInstance() { return hasActorInstance; }
+ public com.google.protobuf.ByteString getActorInstance() { return actorInstance_; }
+
+ // optional string serializerClassname = 6;
+ public static final int SERIALIZERCLASSNAME_FIELD_NUMBER = 6;
+ private boolean hasSerializerClassname;
+ private java.lang.String serializerClassname_ = "";
+ public boolean hasSerializerClassname() { return hasSerializerClassname; }
+ public java.lang.String getSerializerClassname() { return serializerClassname_; }
+
+ // optional bool isTransactor = 7;
+ public static final int ISTRANSACTOR_FIELD_NUMBER = 7;
+ private boolean hasIsTransactor;
+ private boolean isTransactor_ = false;
+ public boolean hasIsTransactor() { return hasIsTransactor; }
+ public boolean getIsTransactor() { return isTransactor_; }
+
+ // optional uint64 timeout = 8;
+ public static final int TIMEOUT_FIELD_NUMBER = 8;
+ private boolean hasTimeout;
+ private long timeout_ = 0L;
+ public boolean hasTimeout() { return hasTimeout; }
+ public long getTimeout() { return timeout_; }
+
+ // optional uint64 receiveTimeout = 9;
+ public static final int RECEIVETIMEOUT_FIELD_NUMBER = 9;
+ private boolean hasReceiveTimeout;
+ private long receiveTimeout_ = 0L;
+ public boolean hasReceiveTimeout() { return hasReceiveTimeout; }
+ public long getReceiveTimeout() { return receiveTimeout_; }
+
+ // optional .LifeCycleProtocol lifeCycle = 10;
+ public static final int LIFECYCLE_FIELD_NUMBER = 10;
+ private boolean hasLifeCycle;
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol lifeCycle_;
+ public boolean hasLifeCycle() { return hasLifeCycle; }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getLifeCycle() { return lifeCycle_; }
+
+ // optional .RemoteActorRefProtocol supervisor = 11;
+ public static final int SUPERVISOR_FIELD_NUMBER = 11;
+ private boolean hasSupervisor;
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol supervisor_;
+ public boolean hasSupervisor() { return hasSupervisor; }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSupervisor() { return supervisor_; }
+
+ // optional bytes hotswapStack = 12;
+ public static final int HOTSWAPSTACK_FIELD_NUMBER = 12;
+ private boolean hasHotswapStack;
+ private com.google.protobuf.ByteString hotswapStack_ = com.google.protobuf.ByteString.EMPTY;
+ public boolean hasHotswapStack() { return hasHotswapStack; }
+ public com.google.protobuf.ByteString getHotswapStack() { return hotswapStack_; }
+
+ // repeated .RemoteRequestProtocol messages = 13;
+ public static final int MESSAGES_FIELD_NUMBER = 13;
+ private java.util.List messages_ =
+ java.util.Collections.emptyList();
+ public java.util.List getMessagesList() {
+ return messages_;
+ }
+ public int getMessagesCount() { return messages_.size(); }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol getMessages(int index) {
+ return messages_.get(index);
+ }
+
+ private void initFields() {
+ originalAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance();
+ lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance();
+ supervisor_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance();
+ }
+ public final boolean isInitialized() {
+ if (!hasUuid) return false;
+ if (!hasId) return false;
+ if (!hasActorClassname) return false;
+ if (!hasOriginalAddress) return false;
+ if (!getOriginalAddress().isInitialized()) return false;
+ if (hasLifeCycle()) {
+ if (!getLifeCycle().isInitialized()) return false;
+ }
+ if (hasSupervisor()) {
+ if (!getSupervisor().isInitialized()) return false;
+ }
+ for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol element : getMessagesList()) {
+ if (!element.isInitialized()) return false;
+ }
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasUuid()) {
+ output.writeString(1, getUuid());
+ }
+ if (hasId()) {
+ output.writeString(2, getId());
+ }
+ if (hasActorClassname()) {
+ output.writeString(3, getActorClassname());
+ }
+ if (hasOriginalAddress()) {
+ output.writeMessage(4, getOriginalAddress());
+ }
+ if (hasActorInstance()) {
+ output.writeBytes(5, getActorInstance());
+ }
+ if (hasSerializerClassname()) {
+ output.writeString(6, getSerializerClassname());
+ }
+ if (hasIsTransactor()) {
+ output.writeBool(7, getIsTransactor());
+ }
+ if (hasTimeout()) {
+ output.writeUInt64(8, getTimeout());
+ }
+ if (hasReceiveTimeout()) {
+ output.writeUInt64(9, getReceiveTimeout());
+ }
+ if (hasLifeCycle()) {
+ output.writeMessage(10, getLifeCycle());
+ }
+ if (hasSupervisor()) {
+ output.writeMessage(11, getSupervisor());
+ }
+ if (hasHotswapStack()) {
+ output.writeBytes(12, getHotswapStack());
+ }
+ for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol element : getMessagesList()) {
+ output.writeMessage(13, element);
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasUuid()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(1, getUuid());
+ }
+ if (hasId()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(2, getId());
+ }
+ if (hasActorClassname()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(3, getActorClassname());
+ }
+ if (hasOriginalAddress()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(4, getOriginalAddress());
+ }
+ if (hasActorInstance()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBytesSize(5, getActorInstance());
+ }
+ if (hasSerializerClassname()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(6, getSerializerClassname());
+ }
+ if (hasIsTransactor()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBoolSize(7, getIsTransactor());
+ }
+ if (hasTimeout()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(8, getTimeout());
+ }
+ if (hasReceiveTimeout()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(9, getReceiveTimeout());
+ }
+ if (hasLifeCycle()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(10, getLifeCycle());
+ }
+ if (hasSupervisor()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(11, getSupervisor());
+ }
+ if (hasHotswapStack()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBytesSize(12, getHotswapStack());
+ }
+ for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol element : getMessagesList()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(13, element);
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder {
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol result;
+
+ // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol();
+ return builder;
+ }
+
+ protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol internalGetResult() {
+ return result;
+ }
+
+ public Builder clear() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "Cannot call clear() after build().");
+ }
+ result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDescriptor();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol getDefaultInstanceForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ if (result.messages_ != java.util.Collections.EMPTY_LIST) {
+ result.messages_ =
+ java.util.Collections.unmodifiableList(result.messages_);
+ }
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol) {
+ return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol other) {
+ if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance()) return this;
+ if (other.hasUuid()) {
+ setUuid(other.getUuid());
+ }
+ if (other.hasId()) {
+ setId(other.getId());
+ }
+ if (other.hasActorClassname()) {
+ setActorClassname(other.getActorClassname());
+ }
+ if (other.hasOriginalAddress()) {
+ mergeOriginalAddress(other.getOriginalAddress());
+ }
+ if (other.hasActorInstance()) {
+ setActorInstance(other.getActorInstance());
+ }
+ if (other.hasSerializerClassname()) {
+ setSerializerClassname(other.getSerializerClassname());
+ }
+ if (other.hasIsTransactor()) {
+ setIsTransactor(other.getIsTransactor());
+ }
+ if (other.hasTimeout()) {
+ setTimeout(other.getTimeout());
+ }
+ if (other.hasReceiveTimeout()) {
+ setReceiveTimeout(other.getReceiveTimeout());
+ }
+ if (other.hasLifeCycle()) {
+ mergeLifeCycle(other.getLifeCycle());
+ }
+ if (other.hasSupervisor()) {
+ mergeSupervisor(other.getSupervisor());
+ }
+ if (other.hasHotswapStack()) {
+ setHotswapStack(other.getHotswapStack());
+ }
+ if (!other.messages_.isEmpty()) {
+ if (result.messages_.isEmpty()) {
+ result.messages_ = new java.util.ArrayList();
+ }
+ result.messages_.addAll(other.messages_);
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder(
+ this.getUnknownFields());
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ }
+ break;
+ }
+ case 10: {
+ setUuid(input.readString());
+ break;
+ }
+ case 18: {
+ setId(input.readString());
+ break;
+ }
+ case 26: {
+ setActorClassname(input.readString());
+ break;
+ }
+ case 34: {
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder();
+ if (hasOriginalAddress()) {
+ subBuilder.mergeFrom(getOriginalAddress());
+ }
+ input.readMessage(subBuilder, extensionRegistry);
+ setOriginalAddress(subBuilder.buildPartial());
+ break;
+ }
+ case 42: {
+ setActorInstance(input.readBytes());
+ break;
+ }
+ case 50: {
+ setSerializerClassname(input.readString());
+ break;
+ }
+ case 56: {
+ setIsTransactor(input.readBool());
+ break;
+ }
+ case 64: {
+ setTimeout(input.readUInt64());
+ break;
+ }
+ case 72: {
+ setReceiveTimeout(input.readUInt64());
+ break;
+ }
+ case 82: {
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder();
+ if (hasLifeCycle()) {
+ subBuilder.mergeFrom(getLifeCycle());
+ }
+ input.readMessage(subBuilder, extensionRegistry);
+ setLifeCycle(subBuilder.buildPartial());
+ break;
+ }
+ case 90: {
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder();
+ if (hasSupervisor()) {
+ subBuilder.mergeFrom(getSupervisor());
+ }
+ input.readMessage(subBuilder, extensionRegistry);
+ setSupervisor(subBuilder.buildPartial());
+ break;
+ }
+ case 98: {
+ setHotswapStack(input.readBytes());
+ break;
+ }
+ case 106: {
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.newBuilder();
+ input.readMessage(subBuilder, extensionRegistry);
+ addMessages(subBuilder.buildPartial());
+ break;
+ }
+ }
+ }
+ }
+
+
+ // required string uuid = 1;
+ public boolean hasUuid() {
+ return result.hasUuid();
+ }
+ public java.lang.String getUuid() {
+ return result.getUuid();
+ }
+ public Builder setUuid(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasUuid = true;
+ result.uuid_ = value;
+ return this;
+ }
+ public Builder clearUuid() {
+ result.hasUuid = false;
+ result.uuid_ = getDefaultInstance().getUuid();
+ return this;
+ }
+
+ // required string id = 2;
+ public boolean hasId() {
+ return result.hasId();
+ }
+ public java.lang.String getId() {
+ return result.getId();
+ }
+ public Builder setId(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasId = true;
+ result.id_ = value;
+ return this;
+ }
+ public Builder clearId() {
+ result.hasId = false;
+ result.id_ = getDefaultInstance().getId();
+ return this;
+ }
+
+ // required string actorClassname = 3;
+ public boolean hasActorClassname() {
+ return result.hasActorClassname();
+ }
+ public java.lang.String getActorClassname() {
+ return result.getActorClassname();
+ }
+ public Builder setActorClassname(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasActorClassname = true;
+ result.actorClassname_ = value;
+ return this;
+ }
+ public Builder clearActorClassname() {
+ result.hasActorClassname = false;
+ result.actorClassname_ = getDefaultInstance().getActorClassname();
+ return this;
+ }
+
+ // required .AddressProtocol originalAddress = 4;
+ public boolean hasOriginalAddress() {
+ return result.hasOriginalAddress();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getOriginalAddress() {
+ return result.getOriginalAddress();
+ }
+ public Builder setOriginalAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasOriginalAddress = true;
+ result.originalAddress_ = value;
+ return this;
+ }
+ public Builder setOriginalAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder builderForValue) {
+ result.hasOriginalAddress = true;
+ result.originalAddress_ = builderForValue.build();
+ return this;
+ }
+ public Builder mergeOriginalAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol value) {
+ if (result.hasOriginalAddress() &&
+ result.originalAddress_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance()) {
+ result.originalAddress_ =
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder(result.originalAddress_).mergeFrom(value).buildPartial();
+ } else {
+ result.originalAddress_ = value;
+ }
+ result.hasOriginalAddress = true;
+ return this;
+ }
+ public Builder clearOriginalAddress() {
+ result.hasOriginalAddress = false;
+ result.originalAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance();
+ return this;
+ }
+
+ // optional bytes actorInstance = 5;
+ public boolean hasActorInstance() {
+ return result.hasActorInstance();
+ }
+ public com.google.protobuf.ByteString getActorInstance() {
+ return result.getActorInstance();
+ }
+ public Builder setActorInstance(com.google.protobuf.ByteString value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasActorInstance = true;
+ result.actorInstance_ = value;
+ return this;
+ }
+ public Builder clearActorInstance() {
+ result.hasActorInstance = false;
+ result.actorInstance_ = getDefaultInstance().getActorInstance();
+ return this;
+ }
+
+ // optional string serializerClassname = 6;
+ public boolean hasSerializerClassname() {
+ return result.hasSerializerClassname();
+ }
+ public java.lang.String getSerializerClassname() {
+ return result.getSerializerClassname();
+ }
+ public Builder setSerializerClassname(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasSerializerClassname = true;
+ result.serializerClassname_ = value;
+ return this;
+ }
+ public Builder clearSerializerClassname() {
+ result.hasSerializerClassname = false;
+ result.serializerClassname_ = getDefaultInstance().getSerializerClassname();
+ return this;
+ }
+
+ // optional bool isTransactor = 7;
+ public boolean hasIsTransactor() {
+ return result.hasIsTransactor();
+ }
+ public boolean getIsTransactor() {
+ return result.getIsTransactor();
+ }
+ public Builder setIsTransactor(boolean value) {
+ result.hasIsTransactor = true;
+ result.isTransactor_ = value;
+ return this;
+ }
+ public Builder clearIsTransactor() {
+ result.hasIsTransactor = false;
+ result.isTransactor_ = false;
+ return this;
+ }
+
+ // optional uint64 timeout = 8;
+ public boolean hasTimeout() {
+ return result.hasTimeout();
+ }
+ public long getTimeout() {
+ return result.getTimeout();
+ }
+ public Builder setTimeout(long value) {
+ result.hasTimeout = true;
+ result.timeout_ = value;
+ return this;
+ }
+ public Builder clearTimeout() {
+ result.hasTimeout = false;
+ result.timeout_ = 0L;
+ return this;
+ }
+
+ // optional uint64 receiveTimeout = 9;
+ public boolean hasReceiveTimeout() {
+ return result.hasReceiveTimeout();
+ }
+ public long getReceiveTimeout() {
+ return result.getReceiveTimeout();
+ }
+ public Builder setReceiveTimeout(long value) {
+ result.hasReceiveTimeout = true;
+ result.receiveTimeout_ = value;
+ return this;
+ }
+ public Builder clearReceiveTimeout() {
+ result.hasReceiveTimeout = false;
+ result.receiveTimeout_ = 0L;
+ return this;
+ }
+
+ // optional .LifeCycleProtocol lifeCycle = 10;
+ public boolean hasLifeCycle() {
+ return result.hasLifeCycle();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getLifeCycle() {
+ return result.getLifeCycle();
+ }
+ public Builder setLifeCycle(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasLifeCycle = true;
+ result.lifeCycle_ = value;
+ return this;
+ }
+ public Builder setLifeCycle(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder builderForValue) {
+ result.hasLifeCycle = true;
+ result.lifeCycle_ = builderForValue.build();
+ return this;
+ }
+ public Builder mergeLifeCycle(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol value) {
+ if (result.hasLifeCycle() &&
+ result.lifeCycle_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance()) {
+ result.lifeCycle_ =
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder(result.lifeCycle_).mergeFrom(value).buildPartial();
+ } else {
+ result.lifeCycle_ = value;
+ }
+ result.hasLifeCycle = true;
+ return this;
+ }
+ public Builder clearLifeCycle() {
+ result.hasLifeCycle = false;
+ result.lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance();
+ return this;
+ }
+
+ // optional .RemoteActorRefProtocol supervisor = 11;
+ public boolean hasSupervisor() {
+ return result.hasSupervisor();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSupervisor() {
+ return result.getSupervisor();
+ }
+ public Builder setSupervisor(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasSupervisor = true;
+ result.supervisor_ = value;
+ return this;
+ }
+ public Builder setSupervisor(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder builderForValue) {
+ result.hasSupervisor = true;
+ result.supervisor_ = builderForValue.build();
+ return this;
+ }
+ public Builder mergeSupervisor(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) {
+ if (result.hasSupervisor() &&
+ result.supervisor_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) {
+ result.supervisor_ =
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(result.supervisor_).mergeFrom(value).buildPartial();
+ } else {
+ result.supervisor_ = value;
+ }
+ result.hasSupervisor = true;
+ return this;
+ }
+ public Builder clearSupervisor() {
+ result.hasSupervisor = false;
+ result.supervisor_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance();
+ return this;
+ }
+
+ // optional bytes hotswapStack = 12;
+ public boolean hasHotswapStack() {
+ return result.hasHotswapStack();
+ }
+ public com.google.protobuf.ByteString getHotswapStack() {
+ return result.getHotswapStack();
+ }
+ public Builder setHotswapStack(com.google.protobuf.ByteString value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasHotswapStack = true;
+ result.hotswapStack_ = value;
+ return this;
+ }
+ public Builder clearHotswapStack() {
+ result.hasHotswapStack = false;
+ result.hotswapStack_ = getDefaultInstance().getHotswapStack();
+ return this;
+ }
+
+ // repeated .RemoteRequestProtocol messages = 13;
+ public java.util.List getMessagesList() {
+ return java.util.Collections.unmodifiableList(result.messages_);
+ }
+ public int getMessagesCount() {
+ return result.getMessagesCount();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol getMessages(int index) {
+ return result.getMessages(index);
+ }
+ public Builder setMessages(int index, se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.messages_.set(index, value);
+ return this;
+ }
+ public Builder setMessages(int index, se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder builderForValue) {
+ result.messages_.set(index, builderForValue.build());
+ return this;
+ }
+ public Builder addMessages(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ if (result.messages_.isEmpty()) {
+ result.messages_ = new java.util.ArrayList();
+ }
+ result.messages_.add(value);
+ return this;
+ }
+ public Builder addMessages(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder builderForValue) {
+ if (result.messages_.isEmpty()) {
+ result.messages_ = new java.util.ArrayList();
+ }
+ result.messages_.add(builderForValue.build());
+ return this;
+ }
+ public Builder addAllMessages(
+ java.lang.Iterable extends se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol> values) {
+ if (result.messages_.isEmpty()) {
+ result.messages_ = new java.util.ArrayList();
+ }
+ super.addAll(values, result.messages_);
+ return this;
+ }
+ public Builder clearMessages() {
+ result.messages_ = java.util.Collections.emptyList();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:SerializedActorRefProtocol)
+ }
+
+ static {
+ defaultInstance = new SerializedActorRefProtocol(true);
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:SerializedActorRefProtocol)
+ }
+
+ public static final class MessageProtocol extends
+ com.google.protobuf.GeneratedMessage {
+ // Use MessageProtocol.newBuilder() to construct.
+ private MessageProtocol() {
+ initFields();
+ }
+ private MessageProtocol(boolean noInit) {}
+
+ private static final MessageProtocol defaultInstance;
+ public static MessageProtocol getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public MessageProtocol getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_MessageProtocol_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_MessageProtocol_fieldAccessorTable;
+ }
+
+ // required .SerializationSchemeType serializationScheme = 1;
+ public static final int SERIALIZATIONSCHEME_FIELD_NUMBER = 1;
+ private boolean hasSerializationScheme;
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType serializationScheme_;
+ public boolean hasSerializationScheme() { return hasSerializationScheme; }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType getSerializationScheme() { return serializationScheme_; }
+
+ // required bytes message = 2;
+ public static final int MESSAGE_FIELD_NUMBER = 2;
+ private boolean hasMessage;
+ private com.google.protobuf.ByteString message_ = com.google.protobuf.ByteString.EMPTY;
+ public boolean hasMessage() { return hasMessage; }
+ public com.google.protobuf.ByteString getMessage() { return message_; }
+
+ // optional bytes messageManifest = 3;
+ public static final int MESSAGEMANIFEST_FIELD_NUMBER = 3;
+ private boolean hasMessageManifest;
+ private com.google.protobuf.ByteString messageManifest_ = com.google.protobuf.ByteString.EMPTY;
+ public boolean hasMessageManifest() { return hasMessageManifest; }
+ public com.google.protobuf.ByteString getMessageManifest() { return messageManifest_; }
+
+ private void initFields() {
+ serializationScheme_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType.JAVA;
+ }
+ public final boolean isInitialized() {
+ if (!hasSerializationScheme) return false;
+ if (!hasMessage) return false;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasSerializationScheme()) {
+ output.writeEnum(1, getSerializationScheme().getNumber());
+ }
+ if (hasMessage()) {
+ output.writeBytes(2, getMessage());
+ }
+ if (hasMessageManifest()) {
+ output.writeBytes(3, getMessageManifest());
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasSerializationScheme()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeEnumSize(1, getSerializationScheme().getNumber());
+ }
+ if (hasMessage()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBytesSize(2, getMessage());
+ }
+ if (hasMessageManifest()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBytesSize(3, getMessageManifest());
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder {
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol result;
+
+ // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol();
+ return builder;
+ }
+
+ protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol internalGetResult() {
+ return result;
+ }
+
+ public Builder clear() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "Cannot call clear() after build().");
+ }
+ result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDescriptor();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getDefaultInstanceForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol) {
+ return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol other) {
+ if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance()) return this;
+ if (other.hasSerializationScheme()) {
+ setSerializationScheme(other.getSerializationScheme());
+ }
+ if (other.hasMessage()) {
+ setMessage(other.getMessage());
+ }
+ if (other.hasMessageManifest()) {
+ setMessageManifest(other.getMessageManifest());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder(
+ this.getUnknownFields());
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ }
+ break;
+ }
+ case 8: {
+ int rawValue = input.readEnum();
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType value = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType.valueOf(rawValue);
+ if (value == null) {
+ unknownFields.mergeVarintField(1, rawValue);
+ } else {
+ setSerializationScheme(value);
+ }
+ break;
+ }
+ case 18: {
+ setMessage(input.readBytes());
+ break;
+ }
+ case 26: {
+ setMessageManifest(input.readBytes());
+ break;
+ }
+ }
+ }
+ }
+
+
+ // required .SerializationSchemeType serializationScheme = 1;
+ public boolean hasSerializationScheme() {
+ return result.hasSerializationScheme();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType getSerializationScheme() {
+ return result.getSerializationScheme();
+ }
+ public Builder setSerializationScheme(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasSerializationScheme = true;
+ result.serializationScheme_ = value;
+ return this;
+ }
+ public Builder clearSerializationScheme() {
+ result.hasSerializationScheme = false;
+ result.serializationScheme_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType.JAVA;
+ return this;
+ }
+
+ // required bytes message = 2;
+ public boolean hasMessage() {
+ return result.hasMessage();
+ }
+ public com.google.protobuf.ByteString getMessage() {
+ return result.getMessage();
+ }
+ public Builder setMessage(com.google.protobuf.ByteString value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasMessage = true;
+ result.message_ = value;
+ return this;
+ }
+ public Builder clearMessage() {
+ result.hasMessage = false;
+ result.message_ = getDefaultInstance().getMessage();
+ return this;
+ }
+
+ // optional bytes messageManifest = 3;
+ public boolean hasMessageManifest() {
+ return result.hasMessageManifest();
+ }
+ public com.google.protobuf.ByteString getMessageManifest() {
+ return result.getMessageManifest();
+ }
+ public Builder setMessageManifest(com.google.protobuf.ByteString value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasMessageManifest = true;
+ result.messageManifest_ = value;
+ return this;
+ }
+ public Builder clearMessageManifest() {
+ result.hasMessageManifest = false;
+ result.messageManifest_ = getDefaultInstance().getMessageManifest();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:MessageProtocol)
+ }
+
+ static {
+ defaultInstance = new MessageProtocol(true);
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:MessageProtocol)
+ }
+
+ public static final class ActorInfoProtocol extends
+ com.google.protobuf.GeneratedMessage {
+ // Use ActorInfoProtocol.newBuilder() to construct.
+ private ActorInfoProtocol() {
+ initFields();
+ }
+ private ActorInfoProtocol(boolean noInit) {}
+
+ private static final ActorInfoProtocol defaultInstance;
+ public static ActorInfoProtocol getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public ActorInfoProtocol getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ActorInfoProtocol_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ActorInfoProtocol_fieldAccessorTable;
+ }
+
+ // required string uuid = 1;
+ public static final int UUID_FIELD_NUMBER = 1;
+ private boolean hasUuid;
+ private java.lang.String uuid_ = "";
+ public boolean hasUuid() { return hasUuid; }
+ public java.lang.String getUuid() { return uuid_; }
+
+ // required string target = 2;
+ public static final int TARGET_FIELD_NUMBER = 2;
+ private boolean hasTarget;
+ private java.lang.String target_ = "";
+ public boolean hasTarget() { return hasTarget; }
+ public java.lang.String getTarget() { return target_; }
+
+ // required uint64 timeout = 3;
+ public static final int TIMEOUT_FIELD_NUMBER = 3;
+ private boolean hasTimeout;
+ private long timeout_ = 0L;
+ public boolean hasTimeout() { return hasTimeout; }
+ public long getTimeout() { return timeout_; }
+
+ // required .ActorType actorType = 4;
+ public static final int ACTORTYPE_FIELD_NUMBER = 4;
+ private boolean hasActorType;
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType actorType_;
+ public boolean hasActorType() { return hasActorType; }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType getActorType() { return actorType_; }
+
+ // optional .TypedActorInfoProtocol typedActorInfo = 5;
+ public static final int TYPEDACTORINFO_FIELD_NUMBER = 5;
+ private boolean hasTypedActorInfo;
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol typedActorInfo_;
+ public boolean hasTypedActorInfo() { return hasTypedActorInfo; }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol getTypedActorInfo() { return typedActorInfo_; }
+
+ private void initFields() {
+ actorType_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType.SCALA_ACTOR;
+ typedActorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance();
+ }
+ public final boolean isInitialized() {
+ if (!hasUuid) return false;
+ if (!hasTarget) return false;
+ if (!hasTimeout) return false;
+ if (!hasActorType) return false;
+ if (hasTypedActorInfo()) {
+ if (!getTypedActorInfo().isInitialized()) return false;
+ }
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasUuid()) {
+ output.writeString(1, getUuid());
+ }
+ if (hasTarget()) {
+ output.writeString(2, getTarget());
+ }
+ if (hasTimeout()) {
+ output.writeUInt64(3, getTimeout());
+ }
+ if (hasActorType()) {
+ output.writeEnum(4, getActorType().getNumber());
+ }
+ if (hasTypedActorInfo()) {
+ output.writeMessage(5, getTypedActorInfo());
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasUuid()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(1, getUuid());
+ }
+ if (hasTarget()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(2, getTarget());
+ }
+ if (hasTimeout()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(3, getTimeout());
+ }
+ if (hasActorType()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeEnumSize(4, getActorType().getNumber());
+ }
+ if (hasTypedActorInfo()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(5, getTypedActorInfo());
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder {
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol result;
+
+ // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol();
+ return builder;
+ }
+
+ protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol internalGetResult() {
+ return result;
+ }
+
+ public Builder clear() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "Cannot call clear() after build().");
+ }
+ result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDescriptor();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getDefaultInstanceForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol) {
+ return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol other) {
+ if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance()) return this;
+ if (other.hasUuid()) {
+ setUuid(other.getUuid());
+ }
+ if (other.hasTarget()) {
+ setTarget(other.getTarget());
+ }
+ if (other.hasTimeout()) {
+ setTimeout(other.getTimeout());
+ }
+ if (other.hasActorType()) {
+ setActorType(other.getActorType());
+ }
+ if (other.hasTypedActorInfo()) {
+ mergeTypedActorInfo(other.getTypedActorInfo());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder(
+ this.getUnknownFields());
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ }
+ break;
+ }
+ case 10: {
+ setUuid(input.readString());
+ break;
+ }
+ case 18: {
+ setTarget(input.readString());
+ break;
+ }
+ case 24: {
+ setTimeout(input.readUInt64());
+ break;
+ }
+ case 32: {
+ int rawValue = input.readEnum();
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType value = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType.valueOf(rawValue);
+ if (value == null) {
+ unknownFields.mergeVarintField(4, rawValue);
+ } else {
+ setActorType(value);
+ }
+ break;
+ }
+ case 42: {
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.newBuilder();
+ if (hasTypedActorInfo()) {
+ subBuilder.mergeFrom(getTypedActorInfo());
+ }
+ input.readMessage(subBuilder, extensionRegistry);
+ setTypedActorInfo(subBuilder.buildPartial());
+ break;
+ }
+ }
+ }
+ }
+
+
+ // required string uuid = 1;
+ public boolean hasUuid() {
+ return result.hasUuid();
+ }
+ public java.lang.String getUuid() {
+ return result.getUuid();
+ }
+ public Builder setUuid(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasUuid = true;
+ result.uuid_ = value;
+ return this;
+ }
+ public Builder clearUuid() {
+ result.hasUuid = false;
+ result.uuid_ = getDefaultInstance().getUuid();
+ return this;
+ }
+
+ // required string target = 2;
+ public boolean hasTarget() {
+ return result.hasTarget();
+ }
+ public java.lang.String getTarget() {
+ return result.getTarget();
+ }
+ public Builder setTarget(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasTarget = true;
+ result.target_ = value;
+ return this;
+ }
+ public Builder clearTarget() {
+ result.hasTarget = false;
+ result.target_ = getDefaultInstance().getTarget();
+ return this;
+ }
+
+ // required uint64 timeout = 3;
+ public boolean hasTimeout() {
+ return result.hasTimeout();
+ }
+ public long getTimeout() {
+ return result.getTimeout();
+ }
+ public Builder setTimeout(long value) {
+ result.hasTimeout = true;
+ result.timeout_ = value;
+ return this;
+ }
+ public Builder clearTimeout() {
+ result.hasTimeout = false;
+ result.timeout_ = 0L;
+ return this;
+ }
+
+ // required .ActorType actorType = 4;
+ public boolean hasActorType() {
+ return result.hasActorType();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType getActorType() {
+ return result.getActorType();
+ }
+ public Builder setActorType(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasActorType = true;
+ result.actorType_ = value;
+ return this;
+ }
+ public Builder clearActorType() {
+ result.hasActorType = false;
+ result.actorType_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType.SCALA_ACTOR;
+ return this;
+ }
+
+ // optional .TypedActorInfoProtocol typedActorInfo = 5;
+ public boolean hasTypedActorInfo() {
+ return result.hasTypedActorInfo();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol getTypedActorInfo() {
+ return result.getTypedActorInfo();
+ }
+ public Builder setTypedActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasTypedActorInfo = true;
+ result.typedActorInfo_ = value;
+ return this;
+ }
+ public Builder setTypedActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.Builder builderForValue) {
+ result.hasTypedActorInfo = true;
+ result.typedActorInfo_ = builderForValue.build();
+ return this;
+ }
+ public Builder mergeTypedActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol value) {
+ if (result.hasTypedActorInfo() &&
+ result.typedActorInfo_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance()) {
+ result.typedActorInfo_ =
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.newBuilder(result.typedActorInfo_).mergeFrom(value).buildPartial();
+ } else {
+ result.typedActorInfo_ = value;
+ }
+ result.hasTypedActorInfo = true;
+ return this;
+ }
+ public Builder clearTypedActorInfo() {
+ result.hasTypedActorInfo = false;
+ result.typedActorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:ActorInfoProtocol)
+ }
+
+ static {
+ defaultInstance = new ActorInfoProtocol(true);
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:ActorInfoProtocol)
+ }
+
+ public static final class TypedActorInfoProtocol extends
+ com.google.protobuf.GeneratedMessage {
+ // Use TypedActorInfoProtocol.newBuilder() to construct.
+ private TypedActorInfoProtocol() {
+ initFields();
+ }
+ private TypedActorInfoProtocol(boolean noInit) {}
+
+ private static final TypedActorInfoProtocol defaultInstance;
+ public static TypedActorInfoProtocol getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public TypedActorInfoProtocol getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_TypedActorInfoProtocol_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_TypedActorInfoProtocol_fieldAccessorTable;
+ }
+
+ // required string interface = 1;
+ public static final int INTERFACE_FIELD_NUMBER = 1;
+ private boolean hasInterface;
+ private java.lang.String interface_ = "";
+ public boolean hasInterface() { return hasInterface; }
+ public java.lang.String getInterface() { return interface_; }
+
+ // required string method = 2;
+ public static final int METHOD_FIELD_NUMBER = 2;
+ private boolean hasMethod;
+ private java.lang.String method_ = "";
+ public boolean hasMethod() { return hasMethod; }
+ public java.lang.String getMethod() { return method_; }
+
+ private void initFields() {
+ }
+ public final boolean isInitialized() {
+ if (!hasInterface) return false;
+ if (!hasMethod) return false;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasInterface()) {
+ output.writeString(1, getInterface());
+ }
+ if (hasMethod()) {
+ output.writeString(2, getMethod());
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasInterface()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(1, getInterface());
+ }
+ if (hasMethod()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(2, getMethod());
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder {
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol result;
+
+ // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol();
+ return builder;
+ }
+
+ protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol internalGetResult() {
+ return result;
+ }
+
+ public Builder clear() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "Cannot call clear() after build().");
+ }
+ result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDescriptor();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol getDefaultInstanceForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol) {
+ return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol other) {
+ if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance()) return this;
+ if (other.hasInterface()) {
+ setInterface(other.getInterface());
+ }
+ if (other.hasMethod()) {
+ setMethod(other.getMethod());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder(
+ this.getUnknownFields());
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ }
+ break;
+ }
+ case 10: {
+ setInterface(input.readString());
+ break;
+ }
+ case 18: {
+ setMethod(input.readString());
+ break;
+ }
+ }
+ }
+ }
+
+
+ // required string interface = 1;
+ public boolean hasInterface() {
+ return result.hasInterface();
+ }
+ public java.lang.String getInterface() {
+ return result.getInterface();
+ }
+ public Builder setInterface(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasInterface = true;
+ result.interface_ = value;
+ return this;
+ }
+ public Builder clearInterface() {
+ result.hasInterface = false;
+ result.interface_ = getDefaultInstance().getInterface();
+ return this;
+ }
+
+ // required string method = 2;
+ public boolean hasMethod() {
+ return result.hasMethod();
+ }
+ public java.lang.String getMethod() {
+ return result.getMethod();
+ }
+ public Builder setMethod(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasMethod = true;
+ result.method_ = value;
+ return this;
+ }
+ public Builder clearMethod() {
+ result.hasMethod = false;
+ result.method_ = getDefaultInstance().getMethod();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:TypedActorInfoProtocol)
+ }
+
+ static {
+ defaultInstance = new TypedActorInfoProtocol(true);
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:TypedActorInfoProtocol)
+ }
+
+ public static final class RemoteRequestProtocol extends
+ com.google.protobuf.GeneratedMessage {
+ // Use RemoteRequestProtocol.newBuilder() to construct.
+ private RemoteRequestProtocol() {
+ initFields();
+ }
+ private RemoteRequestProtocol(boolean noInit) {}
+
+ private static final RemoteRequestProtocol defaultInstance;
+ public static RemoteRequestProtocol getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public RemoteRequestProtocol getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteRequestProtocol_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteRequestProtocol_fieldAccessorTable;
+ }
+
+ // required uint64 id = 1;
+ public static final int ID_FIELD_NUMBER = 1;
+ private boolean hasId;
+ private long id_ = 0L;
+ public boolean hasId() { return hasId; }
+ public long getId() { return id_; }
+
+ // required .MessageProtocol message = 2;
+ public static final int MESSAGE_FIELD_NUMBER = 2;
+ private boolean hasMessage;
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol message_;
+ public boolean hasMessage() { return hasMessage; }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { return message_; }
+
+ // required .ActorInfoProtocol actorInfo = 3;
+ public static final int ACTORINFO_FIELD_NUMBER = 3;
+ private boolean hasActorInfo;
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol actorInfo_;
+ public boolean hasActorInfo() { return hasActorInfo; }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getActorInfo() { return actorInfo_; }
+
+ // required bool isOneWay = 4;
+ public static final int ISONEWAY_FIELD_NUMBER = 4;
+ private boolean hasIsOneWay;
+ private boolean isOneWay_ = false;
+ public boolean hasIsOneWay() { return hasIsOneWay; }
+ public boolean getIsOneWay() { return isOneWay_; }
+
+ // optional string supervisorUuid = 5;
+ public static final int SUPERVISORUUID_FIELD_NUMBER = 5;
+ private boolean hasSupervisorUuid;
+ private java.lang.String supervisorUuid_ = "";
+ public boolean hasSupervisorUuid() { return hasSupervisorUuid; }
+ public java.lang.String getSupervisorUuid() { return supervisorUuid_; }
+
+ // optional .RemoteActorRefProtocol sender = 6;
+ public static final int SENDER_FIELD_NUMBER = 6;
+ private boolean hasSender;
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol sender_;
+ public boolean hasSender() { return hasSender; }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSender() { return sender_; }
+
+ private void initFields() {
+ message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance();
+ actorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance();
+ sender_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance();
+ }
+ public final boolean isInitialized() {
+ if (!hasId) return false;
+ if (!hasMessage) return false;
+ if (!hasActorInfo) return false;
+ if (!hasIsOneWay) return false;
+ if (!getMessage().isInitialized()) return false;
+ if (!getActorInfo().isInitialized()) return false;
+ if (hasSender()) {
+ if (!getSender().isInitialized()) return false;
+ }
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasId()) {
+ output.writeUInt64(1, getId());
+ }
+ if (hasMessage()) {
+ output.writeMessage(2, getMessage());
+ }
+ if (hasActorInfo()) {
+ output.writeMessage(3, getActorInfo());
+ }
+ if (hasIsOneWay()) {
+ output.writeBool(4, getIsOneWay());
+ }
+ if (hasSupervisorUuid()) {
+ output.writeString(5, getSupervisorUuid());
+ }
+ if (hasSender()) {
+ output.writeMessage(6, getSender());
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasId()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(1, getId());
+ }
+ if (hasMessage()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(2, getMessage());
+ }
+ if (hasActorInfo()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(3, getActorInfo());
+ }
+ if (hasIsOneWay()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBoolSize(4, getIsOneWay());
+ }
+ if (hasSupervisorUuid()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(5, getSupervisorUuid());
+ }
+ if (hasSender()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(6, getSender());
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder {
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol result;
+
+ // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol();
+ return builder;
+ }
+
+ protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol internalGetResult() {
+ return result;
+ }
+
+ public Builder clear() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "Cannot call clear() after build().");
+ }
+ result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.getDescriptor();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol getDefaultInstanceForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol) {
+ return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol other) {
+ if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.getDefaultInstance()) return this;
+ if (other.hasId()) {
+ setId(other.getId());
+ }
+ if (other.hasMessage()) {
+ mergeMessage(other.getMessage());
+ }
+ if (other.hasActorInfo()) {
+ mergeActorInfo(other.getActorInfo());
+ }
+ if (other.hasIsOneWay()) {
+ setIsOneWay(other.getIsOneWay());
+ }
+ if (other.hasSupervisorUuid()) {
+ setSupervisorUuid(other.getSupervisorUuid());
+ }
+ if (other.hasSender()) {
+ mergeSender(other.getSender());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder(
+ this.getUnknownFields());
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ }
+ break;
+ }
+ case 8: {
+ setId(input.readUInt64());
+ break;
+ }
+ case 18: {
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder();
+ if (hasMessage()) {
+ subBuilder.mergeFrom(getMessage());
+ }
+ input.readMessage(subBuilder, extensionRegistry);
+ setMessage(subBuilder.buildPartial());
+ break;
+ }
+ case 26: {
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder();
+ if (hasActorInfo()) {
+ subBuilder.mergeFrom(getActorInfo());
+ }
+ input.readMessage(subBuilder, extensionRegistry);
+ setActorInfo(subBuilder.buildPartial());
+ break;
+ }
+ case 32: {
+ setIsOneWay(input.readBool());
+ break;
+ }
+ case 42: {
+ setSupervisorUuid(input.readString());
+ break;
+ }
+ case 50: {
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder();
+ if (hasSender()) {
+ subBuilder.mergeFrom(getSender());
+ }
+ input.readMessage(subBuilder, extensionRegistry);
+ setSender(subBuilder.buildPartial());
+ break;
+ }
+ }
+ }
+ }
+
+
+ // required uint64 id = 1;
+ public boolean hasId() {
+ return result.hasId();
+ }
+ public long getId() {
+ return result.getId();
+ }
+ public Builder setId(long value) {
+ result.hasId = true;
+ result.id_ = value;
+ return this;
+ }
+ public Builder clearId() {
+ result.hasId = false;
+ result.id_ = 0L;
+ return this;
+ }
+
+ // required .MessageProtocol message = 2;
+ public boolean hasMessage() {
+ return result.hasMessage();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() {
+ return result.getMessage();
+ }
+ public Builder setMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasMessage = true;
+ result.message_ = value;
+ return this;
+ }
+ public Builder setMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder builderForValue) {
+ result.hasMessage = true;
+ result.message_ = builderForValue.build();
+ return this;
+ }
+ public Builder mergeMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol value) {
+ if (result.hasMessage() &&
+ result.message_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance()) {
+ result.message_ =
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder(result.message_).mergeFrom(value).buildPartial();
+ } else {
+ result.message_ = value;
+ }
+ result.hasMessage = true;
+ return this;
+ }
+ public Builder clearMessage() {
+ result.hasMessage = false;
+ result.message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance();
+ return this;
+ }
+
+ // required .ActorInfoProtocol actorInfo = 3;
+ public boolean hasActorInfo() {
+ return result.hasActorInfo();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getActorInfo() {
+ return result.getActorInfo();
+ }
+ public Builder setActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasActorInfo = true;
+ result.actorInfo_ = value;
+ return this;
+ }
+ public Builder setActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder builderForValue) {
+ result.hasActorInfo = true;
+ result.actorInfo_ = builderForValue.build();
+ return this;
+ }
+ public Builder mergeActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol value) {
+ if (result.hasActorInfo() &&
+ result.actorInfo_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance()) {
+ result.actorInfo_ =
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder(result.actorInfo_).mergeFrom(value).buildPartial();
+ } else {
+ result.actorInfo_ = value;
+ }
+ result.hasActorInfo = true;
+ return this;
+ }
+ public Builder clearActorInfo() {
+ result.hasActorInfo = false;
+ result.actorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance();
+ return this;
+ }
+
+ // required bool isOneWay = 4;
+ public boolean hasIsOneWay() {
+ return result.hasIsOneWay();
+ }
+ public boolean getIsOneWay() {
+ return result.getIsOneWay();
+ }
+ public Builder setIsOneWay(boolean value) {
+ result.hasIsOneWay = true;
+ result.isOneWay_ = value;
+ return this;
+ }
+ public Builder clearIsOneWay() {
+ result.hasIsOneWay = false;
+ result.isOneWay_ = false;
+ return this;
+ }
+
+ // optional string supervisorUuid = 5;
+ public boolean hasSupervisorUuid() {
+ return result.hasSupervisorUuid();
+ }
+ public java.lang.String getSupervisorUuid() {
+ return result.getSupervisorUuid();
+ }
+ public Builder setSupervisorUuid(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasSupervisorUuid = true;
+ result.supervisorUuid_ = value;
+ return this;
+ }
+ public Builder clearSupervisorUuid() {
+ result.hasSupervisorUuid = false;
+ result.supervisorUuid_ = getDefaultInstance().getSupervisorUuid();
+ return this;
+ }
+
+ // optional .RemoteActorRefProtocol sender = 6;
+ public boolean hasSender() {
+ return result.hasSender();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSender() {
+ return result.getSender();
+ }
+ public Builder setSender(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasSender = true;
+ result.sender_ = value;
+ return this;
+ }
+ public Builder setSender(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder builderForValue) {
+ result.hasSender = true;
+ result.sender_ = builderForValue.build();
+ return this;
+ }
+ public Builder mergeSender(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) {
+ if (result.hasSender() &&
+ result.sender_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) {
+ result.sender_ =
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(result.sender_).mergeFrom(value).buildPartial();
+ } else {
+ result.sender_ = value;
+ }
+ result.hasSender = true;
+ return this;
+ }
+ public Builder clearSender() {
+ result.hasSender = false;
+ result.sender_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:RemoteRequestProtocol)
+ }
+
+ static {
+ defaultInstance = new RemoteRequestProtocol(true);
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:RemoteRequestProtocol)
+ }
+
+ public static final class RemoteReplyProtocol extends
+ com.google.protobuf.GeneratedMessage {
+ // Use RemoteReplyProtocol.newBuilder() to construct.
+ private RemoteReplyProtocol() {
+ initFields();
+ }
+ private RemoteReplyProtocol(boolean noInit) {}
+
+ private static final RemoteReplyProtocol defaultInstance;
+ public static RemoteReplyProtocol getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public RemoteReplyProtocol getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteReplyProtocol_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteReplyProtocol_fieldAccessorTable;
+ }
+
+ // required uint64 id = 1;
+ public static final int ID_FIELD_NUMBER = 1;
+ private boolean hasId;
+ private long id_ = 0L;
+ public boolean hasId() { return hasId; }
+ public long getId() { return id_; }
+
+ // optional .MessageProtocol message = 2;
+ public static final int MESSAGE_FIELD_NUMBER = 2;
+ private boolean hasMessage;
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol message_;
+ public boolean hasMessage() { return hasMessage; }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { return message_; }
+
+ // optional .ExceptionProtocol exception = 3;
+ public static final int EXCEPTION_FIELD_NUMBER = 3;
+ private boolean hasException;
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol exception_;
+ public boolean hasException() { return hasException; }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol getException() { return exception_; }
+
+ // optional string supervisorUuid = 4;
+ public static final int SUPERVISORUUID_FIELD_NUMBER = 4;
+ private boolean hasSupervisorUuid;
+ private java.lang.String supervisorUuid_ = "";
+ public boolean hasSupervisorUuid() { return hasSupervisorUuid; }
+ public java.lang.String getSupervisorUuid() { return supervisorUuid_; }
+
+ // required bool isActor = 5;
+ public static final int ISACTOR_FIELD_NUMBER = 5;
+ private boolean hasIsActor;
+ private boolean isActor_ = false;
+ public boolean hasIsActor() { return hasIsActor; }
+ public boolean getIsActor() { return isActor_; }
+
+ // required bool isSuccessful = 6;
+ public static final int ISSUCCESSFUL_FIELD_NUMBER = 6;
+ private boolean hasIsSuccessful;
+ private boolean isSuccessful_ = false;
+ public boolean hasIsSuccessful() { return hasIsSuccessful; }
+ public boolean getIsSuccessful() { return isSuccessful_; }
+
+ private void initFields() {
+ message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance();
+ exception_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance();
+ }
+ public final boolean isInitialized() {
+ if (!hasId) return false;
+ if (!hasIsActor) return false;
+ if (!hasIsSuccessful) return false;
+ if (hasMessage()) {
+ if (!getMessage().isInitialized()) return false;
+ }
+ if (hasException()) {
+ if (!getException().isInitialized()) return false;
+ }
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasId()) {
+ output.writeUInt64(1, getId());
+ }
+ if (hasMessage()) {
+ output.writeMessage(2, getMessage());
+ }
+ if (hasException()) {
+ output.writeMessage(3, getException());
+ }
+ if (hasSupervisorUuid()) {
+ output.writeString(4, getSupervisorUuid());
+ }
+ if (hasIsActor()) {
+ output.writeBool(5, getIsActor());
+ }
+ if (hasIsSuccessful()) {
+ output.writeBool(6, getIsSuccessful());
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasId()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(1, getId());
+ }
+ if (hasMessage()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(2, getMessage());
+ }
+ if (hasException()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(3, getException());
+ }
+ if (hasSupervisorUuid()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(4, getSupervisorUuid());
+ }
+ if (hasIsActor()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBoolSize(5, getIsActor());
+ }
+ if (hasIsSuccessful()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBoolSize(6, getIsSuccessful());
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder {
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol result;
+
+ // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol();
+ return builder;
+ }
+
+ protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol internalGetResult() {
+ return result;
+ }
+
+ public Builder clear() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "Cannot call clear() after build().");
+ }
+ result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.getDescriptor();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol getDefaultInstanceForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol) {
+ return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol other) {
+ if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.getDefaultInstance()) return this;
+ if (other.hasId()) {
+ setId(other.getId());
+ }
+ if (other.hasMessage()) {
+ mergeMessage(other.getMessage());
+ }
+ if (other.hasException()) {
+ mergeException(other.getException());
+ }
+ if (other.hasSupervisorUuid()) {
+ setSupervisorUuid(other.getSupervisorUuid());
+ }
+ if (other.hasIsActor()) {
+ setIsActor(other.getIsActor());
+ }
+ if (other.hasIsSuccessful()) {
+ setIsSuccessful(other.getIsSuccessful());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder(
+ this.getUnknownFields());
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ }
+ break;
+ }
+ case 8: {
+ setId(input.readUInt64());
+ break;
+ }
+ case 18: {
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder();
+ if (hasMessage()) {
+ subBuilder.mergeFrom(getMessage());
+ }
+ input.readMessage(subBuilder, extensionRegistry);
+ setMessage(subBuilder.buildPartial());
+ break;
+ }
+ case 26: {
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder();
+ if (hasException()) {
+ subBuilder.mergeFrom(getException());
+ }
+ input.readMessage(subBuilder, extensionRegistry);
+ setException(subBuilder.buildPartial());
+ break;
+ }
+ case 34: {
+ setSupervisorUuid(input.readString());
+ break;
+ }
+ case 40: {
+ setIsActor(input.readBool());
+ break;
+ }
+ case 48: {
+ setIsSuccessful(input.readBool());
+ break;
+ }
+ }
+ }
+ }
+
+
+ // required uint64 id = 1;
+ public boolean hasId() {
+ return result.hasId();
+ }
+ public long getId() {
+ return result.getId();
+ }
+ public Builder setId(long value) {
+ result.hasId = true;
+ result.id_ = value;
+ return this;
+ }
+ public Builder clearId() {
+ result.hasId = false;
+ result.id_ = 0L;
+ return this;
+ }
+
+ // optional .MessageProtocol message = 2;
+ public boolean hasMessage() {
+ return result.hasMessage();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() {
+ return result.getMessage();
+ }
+ public Builder setMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasMessage = true;
+ result.message_ = value;
+ return this;
+ }
+ public Builder setMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder builderForValue) {
+ result.hasMessage = true;
+ result.message_ = builderForValue.build();
+ return this;
+ }
+ public Builder mergeMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol value) {
+ if (result.hasMessage() &&
+ result.message_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance()) {
+ result.message_ =
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder(result.message_).mergeFrom(value).buildPartial();
+ } else {
+ result.message_ = value;
+ }
+ result.hasMessage = true;
+ return this;
+ }
+ public Builder clearMessage() {
+ result.hasMessage = false;
+ result.message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance();
+ return this;
+ }
+
+ // optional .ExceptionProtocol exception = 3;
+ public boolean hasException() {
+ return result.hasException();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol getException() {
+ return result.getException();
+ }
+ public Builder setException(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasException = true;
+ result.exception_ = value;
+ return this;
+ }
+ public Builder setException(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.Builder builderForValue) {
+ result.hasException = true;
+ result.exception_ = builderForValue.build();
+ return this;
+ }
+ public Builder mergeException(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol value) {
+ if (result.hasException() &&
+ result.exception_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance()) {
+ result.exception_ =
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder(result.exception_).mergeFrom(value).buildPartial();
+ } else {
+ result.exception_ = value;
+ }
+ result.hasException = true;
+ return this;
+ }
+ public Builder clearException() {
+ result.hasException = false;
+ result.exception_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance();
+ return this;
+ }
+
+ // optional string supervisorUuid = 4;
+ public boolean hasSupervisorUuid() {
+ return result.hasSupervisorUuid();
+ }
+ public java.lang.String getSupervisorUuid() {
+ return result.getSupervisorUuid();
+ }
+ public Builder setSupervisorUuid(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasSupervisorUuid = true;
+ result.supervisorUuid_ = value;
+ return this;
+ }
+ public Builder clearSupervisorUuid() {
+ result.hasSupervisorUuid = false;
+ result.supervisorUuid_ = getDefaultInstance().getSupervisorUuid();
+ return this;
+ }
+
+ // required bool isActor = 5;
+ public boolean hasIsActor() {
+ return result.hasIsActor();
+ }
+ public boolean getIsActor() {
+ return result.getIsActor();
+ }
+ public Builder setIsActor(boolean value) {
+ result.hasIsActor = true;
+ result.isActor_ = value;
+ return this;
+ }
+ public Builder clearIsActor() {
+ result.hasIsActor = false;
+ result.isActor_ = false;
+ return this;
+ }
+
+ // required bool isSuccessful = 6;
+ public boolean hasIsSuccessful() {
+ return result.hasIsSuccessful();
+ }
+ public boolean getIsSuccessful() {
+ return result.getIsSuccessful();
+ }
+ public Builder setIsSuccessful(boolean value) {
+ result.hasIsSuccessful = true;
+ result.isSuccessful_ = value;
+ return this;
+ }
+ public Builder clearIsSuccessful() {
+ result.hasIsSuccessful = false;
+ result.isSuccessful_ = false;
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:RemoteReplyProtocol)
+ }
+
+ static {
+ defaultInstance = new RemoteReplyProtocol(true);
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:RemoteReplyProtocol)
+ }
+
+ public static final class LifeCycleProtocol extends
+ com.google.protobuf.GeneratedMessage {
+ // Use LifeCycleProtocol.newBuilder() to construct.
+ private LifeCycleProtocol() {
+ initFields();
+ }
+ private LifeCycleProtocol(boolean noInit) {}
+
+ private static final LifeCycleProtocol defaultInstance;
+ public static LifeCycleProtocol getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public LifeCycleProtocol getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_LifeCycleProtocol_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_LifeCycleProtocol_fieldAccessorTable;
+ }
+
+ // required .LifeCycleType lifeCycle = 1;
+ public static final int LIFECYCLE_FIELD_NUMBER = 1;
+ private boolean hasLifeCycle;
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType lifeCycle_;
+ public boolean hasLifeCycle() { return hasLifeCycle; }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType getLifeCycle() { return lifeCycle_; }
+
+ // optional string preRestart = 2;
+ public static final int PRERESTART_FIELD_NUMBER = 2;
+ private boolean hasPreRestart;
+ private java.lang.String preRestart_ = "";
+ public boolean hasPreRestart() { return hasPreRestart; }
+ public java.lang.String getPreRestart() { return preRestart_; }
+
+ // optional string postRestart = 3;
+ public static final int POSTRESTART_FIELD_NUMBER = 3;
+ private boolean hasPostRestart;
+ private java.lang.String postRestart_ = "";
+ public boolean hasPostRestart() { return hasPostRestart; }
+ public java.lang.String getPostRestart() { return postRestart_; }
+
+ // optional string init = 4;
+ public static final int INIT_FIELD_NUMBER = 4;
+ private boolean hasInit;
+ private java.lang.String init_ = "";
+ public boolean hasInit() { return hasInit; }
+ public java.lang.String getInit() { return init_; }
+
+ // optional string shutdown = 5;
+ public static final int SHUTDOWN_FIELD_NUMBER = 5;
+ private boolean hasShutdown;
+ private java.lang.String shutdown_ = "";
+ public boolean hasShutdown() { return hasShutdown; }
+ public java.lang.String getShutdown() { return shutdown_; }
+
+ private void initFields() {
+ lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType.PERMANENT;
+ }
+ public final boolean isInitialized() {
+ if (!hasLifeCycle) return false;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasLifeCycle()) {
+ output.writeEnum(1, getLifeCycle().getNumber());
+ }
+ if (hasPreRestart()) {
+ output.writeString(2, getPreRestart());
+ }
+ if (hasPostRestart()) {
+ output.writeString(3, getPostRestart());
+ }
+ if (hasInit()) {
+ output.writeString(4, getInit());
+ }
+ if (hasShutdown()) {
+ output.writeString(5, getShutdown());
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasLifeCycle()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeEnumSize(1, getLifeCycle().getNumber());
+ }
+ if (hasPreRestart()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(2, getPreRestart());
+ }
+ if (hasPostRestart()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(3, getPostRestart());
+ }
+ if (hasInit()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(4, getInit());
+ }
+ if (hasShutdown()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(5, getShutdown());
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder {
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol result;
+
+ // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol();
+ return builder;
+ }
+
+ protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol internalGetResult() {
+ return result;
+ }
+
+ public Builder clear() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "Cannot call clear() after build().");
+ }
+ result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDescriptor();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getDefaultInstanceForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol) {
+ return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol other) {
+ if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance()) return this;
+ if (other.hasLifeCycle()) {
+ setLifeCycle(other.getLifeCycle());
+ }
+ if (other.hasPreRestart()) {
+ setPreRestart(other.getPreRestart());
+ }
+ if (other.hasPostRestart()) {
+ setPostRestart(other.getPostRestart());
+ }
+ if (other.hasInit()) {
+ setInit(other.getInit());
+ }
+ if (other.hasShutdown()) {
+ setShutdown(other.getShutdown());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder(
+ this.getUnknownFields());
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ }
+ break;
+ }
+ case 8: {
+ int rawValue = input.readEnum();
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType value = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType.valueOf(rawValue);
+ if (value == null) {
+ unknownFields.mergeVarintField(1, rawValue);
+ } else {
+ setLifeCycle(value);
+ }
+ break;
+ }
+ case 18: {
+ setPreRestart(input.readString());
+ break;
+ }
+ case 26: {
+ setPostRestart(input.readString());
+ break;
+ }
+ case 34: {
+ setInit(input.readString());
+ break;
+ }
+ case 42: {
+ setShutdown(input.readString());
+ break;
+ }
+ }
+ }
+ }
+
+
+ // required .LifeCycleType lifeCycle = 1;
+ public boolean hasLifeCycle() {
+ return result.hasLifeCycle();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType getLifeCycle() {
+ return result.getLifeCycle();
+ }
+ public Builder setLifeCycle(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasLifeCycle = true;
+ result.lifeCycle_ = value;
+ return this;
+ }
+ public Builder clearLifeCycle() {
+ result.hasLifeCycle = false;
+ result.lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType.PERMANENT;
+ return this;
+ }
+
+ // optional string preRestart = 2;
+ public boolean hasPreRestart() {
+ return result.hasPreRestart();
+ }
+ public java.lang.String getPreRestart() {
+ return result.getPreRestart();
+ }
+ public Builder setPreRestart(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasPreRestart = true;
+ result.preRestart_ = value;
+ return this;
+ }
+ public Builder clearPreRestart() {
+ result.hasPreRestart = false;
+ result.preRestart_ = getDefaultInstance().getPreRestart();
+ return this;
+ }
+
+ // optional string postRestart = 3;
+ public boolean hasPostRestart() {
+ return result.hasPostRestart();
+ }
+ public java.lang.String getPostRestart() {
+ return result.getPostRestart();
+ }
+ public Builder setPostRestart(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasPostRestart = true;
+ result.postRestart_ = value;
+ return this;
+ }
+ public Builder clearPostRestart() {
+ result.hasPostRestart = false;
+ result.postRestart_ = getDefaultInstance().getPostRestart();
+ return this;
+ }
+
+ // optional string init = 4;
+ public boolean hasInit() {
+ return result.hasInit();
+ }
+ public java.lang.String getInit() {
+ return result.getInit();
+ }
+ public Builder setInit(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasInit = true;
+ result.init_ = value;
+ return this;
+ }
+ public Builder clearInit() {
+ result.hasInit = false;
+ result.init_ = getDefaultInstance().getInit();
+ return this;
+ }
+
+ // optional string shutdown = 5;
+ public boolean hasShutdown() {
+ return result.hasShutdown();
+ }
+ public java.lang.String getShutdown() {
+ return result.getShutdown();
+ }
+ public Builder setShutdown(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasShutdown = true;
+ result.shutdown_ = value;
+ return this;
+ }
+ public Builder clearShutdown() {
+ result.hasShutdown = false;
+ result.shutdown_ = getDefaultInstance().getShutdown();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:LifeCycleProtocol)
+ }
+
+ static {
+ defaultInstance = new LifeCycleProtocol(true);
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:LifeCycleProtocol)
+ }
+
+ public static final class AddressProtocol extends
+ com.google.protobuf.GeneratedMessage {
+ // Use AddressProtocol.newBuilder() to construct.
+ private AddressProtocol() {
+ initFields();
+ }
+ private AddressProtocol(boolean noInit) {}
+
+ private static final AddressProtocol defaultInstance;
+ public static AddressProtocol getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public AddressProtocol getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_AddressProtocol_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_AddressProtocol_fieldAccessorTable;
+ }
+
+ // required string hostname = 1;
+ public static final int HOSTNAME_FIELD_NUMBER = 1;
+ private boolean hasHostname;
+ private java.lang.String hostname_ = "";
+ public boolean hasHostname() { return hasHostname; }
+ public java.lang.String getHostname() { return hostname_; }
+
+ // required uint32 port = 2;
+ public static final int PORT_FIELD_NUMBER = 2;
+ private boolean hasPort;
+ private int port_ = 0;
+ public boolean hasPort() { return hasPort; }
+ public int getPort() { return port_; }
+
+ private void initFields() {
+ }
+ public final boolean isInitialized() {
+ if (!hasHostname) return false;
+ if (!hasPort) return false;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasHostname()) {
+ output.writeString(1, getHostname());
+ }
+ if (hasPort()) {
+ output.writeUInt32(2, getPort());
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasHostname()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(1, getHostname());
+ }
+ if (hasPort()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt32Size(2, getPort());
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder {
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol result;
+
+ // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol();
+ return builder;
+ }
+
+ protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol internalGetResult() {
+ return result;
+ }
+
+ public Builder clear() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "Cannot call clear() after build().");
+ }
+ result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDescriptor();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getDefaultInstanceForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol) {
+ return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol other) {
+ if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance()) return this;
+ if (other.hasHostname()) {
+ setHostname(other.getHostname());
+ }
+ if (other.hasPort()) {
+ setPort(other.getPort());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder(
+ this.getUnknownFields());
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ }
+ break;
+ }
+ case 10: {
+ setHostname(input.readString());
+ break;
+ }
+ case 16: {
+ setPort(input.readUInt32());
+ break;
+ }
+ }
+ }
+ }
+
+
+ // required string hostname = 1;
+ public boolean hasHostname() {
+ return result.hasHostname();
+ }
+ public java.lang.String getHostname() {
+ return result.getHostname();
+ }
+ public Builder setHostname(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasHostname = true;
+ result.hostname_ = value;
+ return this;
+ }
+ public Builder clearHostname() {
+ result.hasHostname = false;
+ result.hostname_ = getDefaultInstance().getHostname();
+ return this;
+ }
+
+ // required uint32 port = 2;
+ public boolean hasPort() {
+ return result.hasPort();
+ }
+ public int getPort() {
+ return result.getPort();
+ }
+ public Builder setPort(int value) {
+ result.hasPort = true;
+ result.port_ = value;
+ return this;
+ }
+ public Builder clearPort() {
+ result.hasPort = false;
+ result.port_ = 0;
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:AddressProtocol)
+ }
+
+ static {
+ defaultInstance = new AddressProtocol(true);
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:AddressProtocol)
+ }
+
+ public static final class ExceptionProtocol extends
+ com.google.protobuf.GeneratedMessage {
+ // Use ExceptionProtocol.newBuilder() to construct.
+ private ExceptionProtocol() {
+ initFields();
+ }
+ private ExceptionProtocol(boolean noInit) {}
+
+ private static final ExceptionProtocol defaultInstance;
+ public static ExceptionProtocol getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public ExceptionProtocol getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ExceptionProtocol_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ExceptionProtocol_fieldAccessorTable;
+ }
+
+ // required string classname = 1;
+ public static final int CLASSNAME_FIELD_NUMBER = 1;
+ private boolean hasClassname;
+ private java.lang.String classname_ = "";
+ public boolean hasClassname() { return hasClassname; }
+ public java.lang.String getClassname() { return classname_; }
+
+ // required string message = 2;
+ public static final int MESSAGE_FIELD_NUMBER = 2;
+ private boolean hasMessage;
+ private java.lang.String message_ = "";
+ public boolean hasMessage() { return hasMessage; }
+ public java.lang.String getMessage() { return message_; }
+
+ private void initFields() {
+ }
+ public final boolean isInitialized() {
+ if (!hasClassname) return false;
+ if (!hasMessage) return false;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasClassname()) {
+ output.writeString(1, getClassname());
+ }
+ if (hasMessage()) {
+ output.writeString(2, getMessage());
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasClassname()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(1, getClassname());
+ }
+ if (hasMessage()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(2, getMessage());
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder {
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol result;
+
+ // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol();
+ return builder;
+ }
+
+ protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol internalGetResult() {
+ return result;
+ }
+
+ public Builder clear() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "Cannot call clear() after build().");
+ }
+ result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDescriptor();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol getDefaultInstanceForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol) {
+ return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol other) {
+ if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance()) return this;
+ if (other.hasClassname()) {
+ setClassname(other.getClassname());
+ }
+ if (other.hasMessage()) {
+ setMessage(other.getMessage());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder(
+ this.getUnknownFields());
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ }
+ break;
+ }
+ case 10: {
+ setClassname(input.readString());
+ break;
+ }
+ case 18: {
+ setMessage(input.readString());
+ break;
+ }
+ }
+ }
+ }
+
+
+ // required string classname = 1;
+ public boolean hasClassname() {
+ return result.hasClassname();
+ }
+ public java.lang.String getClassname() {
+ return result.getClassname();
+ }
+ public Builder setClassname(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasClassname = true;
+ result.classname_ = value;
+ return this;
+ }
+ public Builder clearClassname() {
+ result.hasClassname = false;
+ result.classname_ = getDefaultInstance().getClassname();
+ return this;
+ }
+
+ // required string message = 2;
+ public boolean hasMessage() {
+ return result.hasMessage();
+ }
+ public java.lang.String getMessage() {
+ return result.getMessage();
+ }
+ public Builder setMessage(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasMessage = true;
+ result.message_ = value;
+ return this;
+ }
+ public Builder clearMessage() {
+ result.hasMessage = false;
+ result.message_ = getDefaultInstance().getMessage();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:ExceptionProtocol)
+ }
+
+ static {
+ defaultInstance = new ExceptionProtocol(true);
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:ExceptionProtocol)
+ }
+
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_RemoteActorRefProtocol_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_RemoteActorRefProtocol_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_SerializedActorRefProtocol_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_SerializedActorRefProtocol_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_MessageProtocol_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_MessageProtocol_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_ActorInfoProtocol_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_ActorInfoProtocol_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_TypedActorInfoProtocol_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_TypedActorInfoProtocol_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_RemoteRequestProtocol_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_RemoteRequestProtocol_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_RemoteReplyProtocol_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_RemoteReplyProtocol_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_LifeCycleProtocol_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_LifeCycleProtocol_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_AddressProtocol_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_AddressProtocol_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_ExceptionProtocol_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_ExceptionProtocol_fieldAccessorTable;
+
+ public static com.google.protobuf.Descriptors.FileDescriptor
+ getDescriptor() {
+ return descriptor;
+ }
+ private static com.google.protobuf.Descriptors.FileDescriptor
+ descriptor;
+ static {
+ java.lang.String[] descriptorData = {
+ "\n\024RemoteProtocol.proto\"v\n\026RemoteActorRef" +
+ "Protocol\022\014\n\004uuid\030\001 \002(\t\022\026\n\016actorClassname" +
+ "\030\002 \002(\t\022%\n\013homeAddress\030\003 \002(\0132\020.AddressPro" +
+ "tocol\022\017\n\007timeout\030\004 \001(\004\"\200\003\n\032SerializedAct" +
+ "orRefProtocol\022\014\n\004uuid\030\001 \002(\t\022\n\n\002id\030\002 \002(\t\022" +
+ "\026\n\016actorClassname\030\003 \002(\t\022)\n\017originalAddre" +
+ "ss\030\004 \002(\0132\020.AddressProtocol\022\025\n\ractorInsta" +
+ "nce\030\005 \001(\014\022\033\n\023serializerClassname\030\006 \001(\t\022\024" +
+ "\n\014isTransactor\030\007 \001(\010\022\017\n\007timeout\030\010 \001(\004\022\026\n" +
+ "\016receiveTimeout\030\t \001(\004\022%\n\tlifeCycle\030\n \001(\013",
+ "2\022.LifeCycleProtocol\022+\n\nsupervisor\030\013 \001(\013" +
+ "2\027.RemoteActorRefProtocol\022\024\n\014hotswapStac" +
+ "k\030\014 \001(\014\022(\n\010messages\030\r \003(\0132\026.RemoteReques" +
+ "tProtocol\"r\n\017MessageProtocol\0225\n\023serializ" +
+ "ationScheme\030\001 \002(\0162\030.SerializationSchemeT" +
+ "ype\022\017\n\007message\030\002 \002(\014\022\027\n\017messageManifest\030" +
+ "\003 \001(\014\"\222\001\n\021ActorInfoProtocol\022\014\n\004uuid\030\001 \002(" +
+ "\t\022\016\n\006target\030\002 \002(\t\022\017\n\007timeout\030\003 \002(\004\022\035\n\tac" +
+ "torType\030\004 \002(\0162\n.ActorType\022/\n\016typedActorI" +
+ "nfo\030\005 \001(\0132\027.TypedActorInfoProtocol\";\n\026Ty",
+ "pedActorInfoProtocol\022\021\n\tinterface\030\001 \002(\t\022" +
+ "\016\n\006method\030\002 \002(\t\"\300\001\n\025RemoteRequestProtoco" +
+ "l\022\n\n\002id\030\001 \002(\004\022!\n\007message\030\002 \002(\0132\020.Message" +
+ "Protocol\022%\n\tactorInfo\030\003 \002(\0132\022.ActorInfoP" +
+ "rotocol\022\020\n\010isOneWay\030\004 \002(\010\022\026\n\016supervisorU" +
+ "uid\030\005 \001(\t\022\'\n\006sender\030\006 \001(\0132\027.RemoteActorR" +
+ "efProtocol\"\252\001\n\023RemoteReplyProtocol\022\n\n\002id" +
+ "\030\001 \002(\004\022!\n\007message\030\002 \001(\0132\020.MessageProtoco" +
+ "l\022%\n\texception\030\003 \001(\0132\022.ExceptionProtocol" +
+ "\022\026\n\016supervisorUuid\030\004 \001(\t\022\017\n\007isActor\030\005 \002(",
+ "\010\022\024\n\014isSuccessful\030\006 \002(\010\"\177\n\021LifeCycleProt" +
+ "ocol\022!\n\tlifeCycle\030\001 \002(\0162\016.LifeCycleType\022" +
+ "\022\n\npreRestart\030\002 \001(\t\022\023\n\013postRestart\030\003 \001(\t" +
+ "\022\014\n\004init\030\004 \001(\t\022\020\n\010shutdown\030\005 \001(\t\"1\n\017Addr" +
+ "essProtocol\022\020\n\010hostname\030\001 \002(\t\022\014\n\004port\030\002 " +
+ "\002(\r\"7\n\021ExceptionProtocol\022\021\n\tclassname\030\001 " +
+ "\002(\t\022\017\n\007message\030\002 \002(\t*=\n\tActorType\022\017\n\013SCA" +
+ "LA_ACTOR\020\001\022\016\n\nJAVA_ACTOR\020\002\022\017\n\013TYPED_ACTO" +
+ "R\020\003*]\n\027SerializationSchemeType\022\010\n\004JAVA\020\001" +
+ "\022\013\n\007SBINARY\020\002\022\016\n\nSCALA_JSON\020\003\022\r\n\tJAVA_JS",
+ "ON\020\004\022\014\n\010PROTOBUF\020\005*-\n\rLifeCycleType\022\r\n\tP" +
+ "ERMANENT\020\001\022\r\n\tTEMPORARY\020\002B-\n)se.scalable" +
+ "solutions.akka.remote.protocolH\001"
+ };
+ com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+ new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+ public com.google.protobuf.ExtensionRegistry assignDescriptors(
+ com.google.protobuf.Descriptors.FileDescriptor root) {
+ descriptor = root;
+ internal_static_RemoteActorRefProtocol_descriptor =
+ getDescriptor().getMessageTypes().get(0);
+ internal_static_RemoteActorRefProtocol_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_RemoteActorRefProtocol_descriptor,
+ new java.lang.String[] { "Uuid", "ActorClassname", "HomeAddress", "Timeout", },
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.class,
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder.class);
+ internal_static_SerializedActorRefProtocol_descriptor =
+ getDescriptor().getMessageTypes().get(1);
+ internal_static_SerializedActorRefProtocol_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_SerializedActorRefProtocol_descriptor,
+ new java.lang.String[] { "Uuid", "Id", "ActorClassname", "OriginalAddress", "ActorInstance", "SerializerClassname", "IsTransactor", "Timeout", "ReceiveTimeout", "LifeCycle", "Supervisor", "HotswapStack", "Messages", },
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.class,
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.Builder.class);
+ internal_static_MessageProtocol_descriptor =
+ getDescriptor().getMessageTypes().get(2);
+ internal_static_MessageProtocol_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_MessageProtocol_descriptor,
+ new java.lang.String[] { "SerializationScheme", "Message", "MessageManifest", },
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.class,
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder.class);
+ internal_static_ActorInfoProtocol_descriptor =
+ getDescriptor().getMessageTypes().get(3);
+ internal_static_ActorInfoProtocol_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_ActorInfoProtocol_descriptor,
+ new java.lang.String[] { "Uuid", "Target", "Timeout", "ActorType", "TypedActorInfo", },
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.class,
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder.class);
+ internal_static_TypedActorInfoProtocol_descriptor =
+ getDescriptor().getMessageTypes().get(4);
+ internal_static_TypedActorInfoProtocol_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_TypedActorInfoProtocol_descriptor,
+ new java.lang.String[] { "Interface", "Method", },
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.class,
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.Builder.class);
+ internal_static_RemoteRequestProtocol_descriptor =
+ getDescriptor().getMessageTypes().get(5);
+ internal_static_RemoteRequestProtocol_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_RemoteRequestProtocol_descriptor,
+ new java.lang.String[] { "Id", "Message", "ActorInfo", "IsOneWay", "SupervisorUuid", "Sender", },
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.class,
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder.class);
+ internal_static_RemoteReplyProtocol_descriptor =
+ getDescriptor().getMessageTypes().get(6);
+ internal_static_RemoteReplyProtocol_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_RemoteReplyProtocol_descriptor,
+ new java.lang.String[] { "Id", "Message", "Exception", "SupervisorUuid", "IsActor", "IsSuccessful", },
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.class,
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.Builder.class);
+ internal_static_LifeCycleProtocol_descriptor =
+ getDescriptor().getMessageTypes().get(7);
+ internal_static_LifeCycleProtocol_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_LifeCycleProtocol_descriptor,
+ new java.lang.String[] { "LifeCycle", "PreRestart", "PostRestart", "Init", "Shutdown", },
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.class,
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder.class);
+ internal_static_AddressProtocol_descriptor =
+ getDescriptor().getMessageTypes().get(8);
+ internal_static_AddressProtocol_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_AddressProtocol_descriptor,
+ new java.lang.String[] { "Hostname", "Port", },
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.class,
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder.class);
+ internal_static_ExceptionProtocol_descriptor =
+ getDescriptor().getMessageTypes().get(9);
+ internal_static_ExceptionProtocol_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_ExceptionProtocol_descriptor,
+ new java.lang.String[] { "Classname", "Message", },
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.class,
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.Builder.class);
+ return null;
+ }
+ };
+ com.google.protobuf.Descriptors.FileDescriptor
+ .internalBuildGeneratedFileFrom(descriptorData,
+ new com.google.protobuf.Descriptors.FileDescriptor[] {
+ }, assigner);
+ }
+
+ public static void internalForceInit() {}
+
+ // @@protoc_insertion_point(outer_class_scope)
+}
diff --git a/akka-core/src/main/scala/actor/TypedActor.scala b/akka-typed-actor/src/main/scala/actor/TypedActor.scala
similarity index 94%
rename from akka-core/src/main/scala/actor/TypedActor.scala
rename to akka-typed-actor/src/main/scala/actor/TypedActor.scala
index 77473fe4d1..b27f5b4b4d 100644
--- a/akka-core/src/main/scala/actor/TypedActor.scala
+++ b/akka-typed-actor/src/main/scala/actor/TypedActor.scala
@@ -6,12 +6,10 @@ package se.scalablesolutions.akka.actor
import Actor._
import se.scalablesolutions.akka.config.FaultHandlingStrategy
-import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._
-import se.scalablesolutions.akka.remote.{MessageSerializer, RemoteClient, RemoteRequestProtocolIdFactory}
import se.scalablesolutions.akka.dispatch.{MessageDispatcher, Future, CompletableFuture, Dispatchers}
import se.scalablesolutions.akka.config.ScalaConfig._
-import se.scalablesolutions.akka.serialization.Serializer
import se.scalablesolutions.akka.util._
+import ReflectiveAccess._
import org.codehaus.aspectwerkz.joinpoint.{MethodRtti, JoinPoint}
import org.codehaus.aspectwerkz.proxy.Proxy
@@ -109,7 +107,7 @@ import scala.reflect.BeanProperty
*
* @author Jonas Bonér
*/
-abstract class TypedActor extends Actor {
+abstract class TypedActor extends Actor with Proxyable {
val DELEGATE_FIELD_NAME = "DELEGATE_0".intern
@volatile private[actor] var proxy: AnyRef = _
@@ -192,7 +190,7 @@ abstract class TypedActor extends Actor {
/**
* Rewrite target instance in AspectWerkz Proxy.
*/
- private[actor] def swapInstanceInProxy(newInstance: Actor) = proxyDelegate.set(proxy, newInstance)
+ private[actor] def swapProxiedActor(newInstance: Actor) = proxyDelegate.set(proxy, newInstance)
private[akka] def initialize(typedActorProxy: AnyRef) = {
proxy = typedActorProxy
@@ -226,8 +224,11 @@ abstract class TypedActor extends Actor {
if (arg.getClass.getName.contains(TypedActor.AW_PROXY_PREFIX)) unserializable = true
}
if (!unserializable && hasMutableArgument) {
- val copyOfArgs = Serializer.Java.deepClone(args)
- joinPoint.getRtti.asInstanceOf[MethodRtti].setParameterValues(copyOfArgs.asInstanceOf[Array[AnyRef]])
+
+ //FIXME serializeArguments
+ // val copyOfArgs = Serializer.Java.deepClone(args)
+ // joinPoint.getRtti.asInstanceOf[MethodRtti].setParameterValues(copyOfArgs.asInstanceOf[Array[AnyRef]])
+ joinPoint
}
}
}
@@ -537,6 +538,12 @@ object TypedActor extends Logging {
private[akka] def supervise(restartStrategy: RestartStrategy, components: List[Supervise]): Supervisor =
Supervisor(SupervisorConfig(restartStrategy, components))
+
+ def isJoinPointAndOneWay(message: Any): Boolean = if (isJoinPoint(message))
+ isOneWay(message.asInstanceOf[JoinPoint].getRtti.asInstanceOf[MethodRtti])
+ else false
+
+ private[akka] def isJoinPoint(message: Any): Boolean = message.isInstanceOf[JoinPoint]
}
/**
@@ -598,33 +605,14 @@ private[akka] sealed class TypedActorAspect {
private def remoteDispatch(joinPoint: JoinPoint): AnyRef = {
val methodRtti = joinPoint.getRtti.asInstanceOf[MethodRtti]
val isOneWay = TypedActor.isOneWay(methodRtti)
+
val (message: Array[AnyRef], isEscaped) = escapeArguments(methodRtti.getParameterValues)
- val typedActorInfo = TypedActorInfoProtocol.newBuilder
- .setInterface(interfaceClass.getName)
- .setMethod(methodRtti.getMethod.getName)
- .build
-
- val actorInfo = ActorInfoProtocol.newBuilder
- .setUuid(uuid)
- .setTarget(typedActor.getClass.getName)
- .setTimeout(timeout)
- .setActorType(ActorType.TYPED_ACTOR)
- .setTypedActorInfo(typedActorInfo)
- .build
-
- val requestBuilder = RemoteRequestProtocol.newBuilder
- .setId(RemoteRequestProtocolIdFactory.nextId)
- .setMessage(MessageSerializer.serialize(message))
- .setActorInfo(actorInfo)
- .setIsOneWay(isOneWay)
-
- val id = actorRef.registerSupervisorAsRemoteActor
- if (id.isDefined) requestBuilder.setSupervisorUuid(id.get)
-
- val remoteMessage = requestBuilder.build
-
- val future = RemoteClient.clientFor(remoteAddress.get).send(remoteMessage, None)
+ val future = RemoteClientModule.send[AnyRef](
+ message, None, None, remoteAddress.get,
+ timeout, isOneWay, actorRef,
+ Some((interfaceClass.getName, methodRtti.getMethod.getName)),
+ ActorType.TypedActor)
if (isOneWay) null // for void methods
else {
diff --git a/akka-core/src/main/scala/config/TypedActorConfigurator.scala b/akka-typed-actor/src/main/scala/config/TypedActorConfigurator.scala
similarity index 100%
rename from akka-core/src/main/scala/config/TypedActorConfigurator.scala
rename to akka-typed-actor/src/main/scala/config/TypedActorConfigurator.scala
diff --git a/akka-core/src/main/scala/config/TypedActorGuiceConfigurator.scala b/akka-typed-actor/src/main/scala/config/TypedActorGuiceConfigurator.scala
similarity index 97%
rename from akka-core/src/main/scala/config/TypedActorGuiceConfigurator.scala
rename to akka-typed-actor/src/main/scala/config/TypedActorGuiceConfigurator.scala
index 718050655c..339c4d297d 100644
--- a/akka-core/src/main/scala/config/TypedActorGuiceConfigurator.scala
+++ b/akka-typed-actor/src/main/scala/config/TypedActorGuiceConfigurator.scala
@@ -6,8 +6,8 @@ package se.scalablesolutions.akka.config
import se.scalablesolutions.akka.actor._
import se.scalablesolutions.akka.config.ScalaConfig._
-import se.scalablesolutions.akka.remote.RemoteServer
-import se.scalablesolutions.akka.util.Logging
+import se.scalablesolutions.akka.util._
+import ReflectiveAccess._
import org.codehaus.aspectwerkz.proxy.Proxy
@@ -122,7 +122,7 @@ private[akka] class TypedActorGuiceConfigurator extends TypedActorConfiguratorBa
remoteAddress.foreach { address =>
actorRef.makeRemote(remoteAddress.get)
- RemoteServer.registerTypedActor(address, implementationClass.getName, proxy)
+ RemoteServerModule.registerTypedActor(address, implementationClass.getName, proxy)
}
AspectInitRegistry.register(
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/Bar.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/Bar.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/Bar.java
rename to akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/Bar.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/BarImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/BarImpl.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/BarImpl.java
rename to akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/BarImpl.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/Ext.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/Ext.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/Ext.java
rename to akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/Ext.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/ExtImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/ExtImpl.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/ExtImpl.java
rename to akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/ExtImpl.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/Foo.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/Foo.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/Foo.java
rename to akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/Foo.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/FooImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/FooImpl.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/FooImpl.java
rename to akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/FooImpl.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActor.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActor.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActor.java
rename to akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActor.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActorImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActorImpl.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActorImpl.java
rename to akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActorImpl.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/SamplePojo.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SamplePojo.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/SamplePojo.java
rename to akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SamplePojo.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/SamplePojoImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SamplePojoImpl.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/SamplePojoImpl.java
rename to akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SamplePojoImpl.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojo.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojo.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojo.java
rename to akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojo.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoCaller.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoCaller.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoCaller.java
rename to akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoCaller.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoCallerImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoCallerImpl.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoCallerImpl.java
rename to akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoCallerImpl.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoImpl.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoImpl.java
rename to akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojoImpl.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActor.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActor.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActor.java
rename to akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActor.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActorImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActorImpl.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActorImpl.java
rename to akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActorImpl.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/TypedActorFailer.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TypedActorFailer.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/TypedActorFailer.java
rename to akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TypedActorFailer.java
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/TypedActorFailerImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TypedActorFailerImpl.java
similarity index 100%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/TypedActorFailerImpl.java
rename to akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TypedActorFailerImpl.java
diff --git a/akka-core/src/test/resources/META-INF/aop.xml b/akka-typed-actor/src/test/resources/META-INF/aop.xml
similarity index 100%
rename from akka-core/src/test/resources/META-INF/aop.xml
rename to akka-typed-actor/src/test/resources/META-INF/aop.xml
diff --git a/akka-core/src/test/scala/actor/typed-actor/NestedTransactionalTypedActorSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/NestedTransactionalTypedActorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/actor/typed-actor/NestedTransactionalTypedActorSpec.scala
rename to akka-typed-actor/src/test/scala/actor/typed-actor/NestedTransactionalTypedActorSpec.scala
diff --git a/akka-core/src/test/scala/actor/typed-actor/RestartNestedTransactionalTypedActorSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/RestartNestedTransactionalTypedActorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/actor/typed-actor/RestartNestedTransactionalTypedActorSpec.scala
rename to akka-typed-actor/src/test/scala/actor/typed-actor/RestartNestedTransactionalTypedActorSpec.scala
diff --git a/akka-core/src/test/scala/actor/typed-actor/RestartTransactionalTypedActorSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/RestartTransactionalTypedActorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/actor/typed-actor/RestartTransactionalTypedActorSpec.scala
rename to akka-typed-actor/src/test/scala/actor/typed-actor/RestartTransactionalTypedActorSpec.scala
diff --git a/akka-core/src/test/scala/actor/typed-actor/TransactionalTypedActorSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/TransactionalTypedActorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/actor/typed-actor/TransactionalTypedActorSpec.scala
rename to akka-typed-actor/src/test/scala/actor/typed-actor/TransactionalTypedActorSpec.scala
diff --git a/akka-core/src/test/scala/actor/typed-actor/TypedActorContextSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorContextSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/actor/typed-actor/TypedActorContextSpec.scala
rename to akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorContextSpec.scala
diff --git a/akka-core/src/test/scala/actor/typed-actor/TypedActorGuiceConfiguratorSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorGuiceConfiguratorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/actor/typed-actor/TypedActorGuiceConfiguratorSpec.scala
rename to akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorGuiceConfiguratorSpec.scala
diff --git a/akka-core/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala
rename to akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala
diff --git a/akka-core/src/test/scala/actor/typed-actor/TypedActorSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/actor/typed-actor/TypedActorSpec.scala
rename to akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorSpec.scala
diff --git a/akka-core/src/test/scala/actor/typed-actor/TypedActorUtilFunctionsSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorUtilFunctionsSpec.scala
similarity index 100%
rename from akka-core/src/test/scala/actor/typed-actor/TypedActorUtilFunctionsSpec.scala
rename to akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorUtilFunctionsSpec.scala
diff --git a/config/storage-conf.xml b/config/cassandra-akka-storage-conf.xml
similarity index 100%
rename from config/storage-conf.xml
rename to config/cassandra-akka-storage-conf.xml
diff --git a/config/logback.xml b/config/logback.xml
index 1ace0bfd8f..3e6ba75548 100644
--- a/config/logback.xml
+++ b/config/logback.xml
@@ -22,7 +22,7 @@
./logs/akka.log.%d{yyyy-MM-dd-HH}
-
+
diff --git a/config/multiverse-properties-reference.txt b/config/multiverse-properties-reference.txt
deleted file mode 100644
index 7503cec167..0000000000
--- a/config/multiverse-properties-reference.txt
+++ /dev/null
@@ -1,38 +0,0 @@
-# ============================================
-# ===== Multiverse JVM Options Reference =====
-# ============================================
-
-# All these properties can be set on the commandline using '-D='
-
-# a flag that is used to enable sanity checks.
-# default=true
-org.multiverse.MuliverseConstants.sanityChecks=
-
-# a flag that enables to dump the bytecode of the instrumented classes to the tmp directory
-# This is a very interesting feature for debugging of the instrumentation
-# default=false
-org.multiverse.stms.alpha.instrumentation.MultiverseJavaAgent.dumpBytecode=
-
-# a string containing the full path to a static no-arg factory method that is used to create the global stm.
-# default = org.multiverse.stms.alpha.AlphaStm.createDebug
-org.multiverse.api.GlobalStmInstance.factorymethod=
-
-# a flag that enables the reuse of the FailedToObtainLocksException exception instance
-# default = true
-org.multiverse.api.exceptions.FailedToObtainLocksException.reuse=
-
-# a flag that enables the reuse of the LoadLockedException exception instance
-# default = true
-org.multiverse.api.exceptions.LoadLockedException.reuse=
-
-# a flag that enables the reuse of the LoadLockedException exception instance
-# default = true
-org.multiverse.api.exceptions.LoadTooOldVersionException.reuse=
-
-# a flag that enables the reuse of the RetryError exception instance
-# default = true
-org.multiverse.api.exceptions.RetryError.reuse=
-
-# a flag that enables the reuse of the WriteConflictException exception instance
-# default = true
-org.multiverse.api.exceptions.WriteConflictException.reuse=
diff --git a/project/build/AkkaProject.scala b/project/build/AkkaProject.scala
index 4503357db6..a5dbc5bd93 100644
--- a/project/build/AkkaProject.scala
+++ b/project/build/AkkaProject.scala
@@ -216,15 +216,17 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
// Subprojects
// -------------------------------------------------------------------------------------------------------------------
- lazy val akka_core = project("akka-core", "akka-core", new AkkaCoreProject(_))
- lazy val akka_amqp = project("akka-amqp", "akka-amqp", new AkkaAMQPProject(_), akka_core)
- lazy val akka_http = project("akka-http", "akka-http", new AkkaHttpProject(_), akka_core, akka_camel)
- lazy val akka_camel = project("akka-camel", "akka-camel", new AkkaCamelProject(_), akka_core)
+ lazy val akka_actor = project("akka-actor", "akka-actor", new AkkaActorProject(_))
+ lazy val akka_typed_actor = project("akka-typed-actor", "akka-typed-actor", new AkkaTypedActorProject(_), akka_actor)
+ lazy val akka_remote = project("akka-remote", "akka-remote", new AkkaRemoteProject(_), akka_typed_actor)
+ lazy val akka_amqp = project("akka-amqp", "akka-amqp", new AkkaAMQPProject(_), akka_remote)
+ lazy val akka_http = project("akka-http", "akka-http", new AkkaHttpProject(_), akka_remote, akka_camel)
+ lazy val akka_camel = project("akka-camel", "akka-camel", new AkkaCamelProject(_), akka_remote)
lazy val akka_persistence = project("akka-persistence", "akka-persistence", new AkkaPersistenceParentProject(_))
- lazy val akka_spring = project("akka-spring", "akka-spring", new AkkaSpringProject(_), akka_core, akka_camel)
- lazy val akka_jta = project("akka-jta", "akka-jta", new AkkaJTAProject(_), akka_core)
+ lazy val akka_spring = project("akka-spring", "akka-spring", new AkkaSpringProject(_), akka_remote, akka_camel)
+ lazy val akka_jta = project("akka-jta", "akka-jta", new AkkaJTAProject(_), akka_remote)
lazy val akka_kernel = project("akka-kernel", "akka-kernel", new AkkaKernelProject(_),
- akka_core, akka_http, akka_spring, akka_camel, akka_persistence, akka_amqp)
+ akka_remote, akka_http, akka_spring, akka_camel, akka_persistence, akka_amqp)
lazy val akka_osgi = project("akka-osgi", "akka-osgi", new AkkaOSGiParentProject(_))
lazy val akka_samples = project("akka-samples", "akka-samples", new AkkaSamplesParentProject(_))
@@ -250,8 +252,11 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
.filter(!_.getName.contains("scala-library"))
.map("lib_managed/scala_%s/compile/".format(buildScalaVersion) + _.getName)
.mkString(" ") +
+ " config/" +
" scala-library.jar" +
- " dist/akka-core_%s-%s.jar".format(buildScalaVersion, version) +
+ " dist/akka-actor_%s-%s.jar".format(buildScalaVersion, version) +
+ " dist/akka-typed-actor_%s-%s.jar".format(buildScalaVersion, version) +
+ " dist/akka-remote_%s-%s.jar".format(buildScalaVersion, version) +
" dist/akka-http_%s-%s.jar".format(buildScalaVersion, version) +
" dist/akka-camel_%s-%s.jar".format(buildScalaVersion, version) +
" dist/akka-amqp_%s-%s.jar".format(buildScalaVersion, version) +
@@ -271,10 +276,9 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
}
override def mainResources = super.mainResources +++
- descendents(info.projectPath / "config", "*") ---
- (super.mainResources ** "logback-test.xml")
+ (info.projectPath / "config").descendentsExcept("*", "logback-test.xml")
- override def testResources = super.testResources --- (super.testResources ** "logback-test.xml")
+ override def runClasspath = super.runClasspath +++ "config"
// ------------------------------------------------------------
// publishing
@@ -307,53 +311,75 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
- // publish to local mvn
- import Process._
- lazy val publishLocalMvn = runMvnInstall
- def runMvnInstall = task {
- for (absPath <- akkaArtifacts.getPaths) {
- val artifactRE = """(.*)/dist/(.*)-(.*).jar""".r
- val artifactRE(path, artifactId, artifactVersion) = absPath
- val command = "mvn install:install-file" +
- " -Dfile=" + absPath +
- " -DgroupId=se.scalablesolutions.akka" +
- " -DartifactId=" + artifactId +
- " -Dversion=" + version +
- " -Dpackaging=jar -DgeneratePom=true"
- command ! log
- }
- None
- } dependsOn(dist) describedAs("Run mvn install for artifacts in dist.")
+ // publish to local mvn
+ import Process._
+ lazy val publishLocalMvn = runMvnInstall
+ def runMvnInstall = task {
+ for (absPath <- akkaArtifacts.getPaths) {
+ val artifactRE = """(.*)/dist/(.*)-(.*).jar""".r
+ val artifactRE(path, artifactId, artifactVersion) = absPath
+ val command = "mvn install:install-file" +
+ " -Dfile=" + absPath +
+ " -DgroupId=se.scalablesolutions.akka" +
+ " -DartifactId=" + artifactId +
+ " -Dversion=" + version +
+ " -Dpackaging=jar -DgeneratePom=true"
+ command ! log
+ }
+ None
+ } dependsOn(dist) describedAs("Run mvn install for artifacts in dist.")
// -------------------------------------------------------------------------------------------------------------------
- // akka-core subproject
+ // akka-actor subproject
// -------------------------------------------------------------------------------------------------------------------
- class AkkaCoreProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) {
+ class AkkaActorProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) {
+ val configgy = Dependencies.configgy
+ val hawtdispatch = Dependencies.hawtdispatch
+ val multiverse = Dependencies.multiverse
+ val jsr166x = Dependencies.jsr166x
+ val slf4j = Dependencies.slf4j
+ val logback = Dependencies.logback
+ val logback_core = Dependencies.logback_core
+
+ // testing
+ val junit = Dependencies.junit
+ val scalatest = Dependencies.scalatest
+ }
+
+ // -------------------------------------------------------------------------------------------------------------------
+ // akka-typed-actor subproject
+ // -------------------------------------------------------------------------------------------------------------------
+
+ class AkkaTypedActorProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) {
val aopalliance = Dependencies.aopalliance
+ val werkz = Dependencies.werkz
+ val werkz_core = Dependencies.werkz_core
+
+ // testing
+ val junit = Dependencies.junit
+ val scalatest = Dependencies.scalatest
+ }
+
+ // -------------------------------------------------------------------------------------------------------------------
+ // akka-remote subproject
+ // -------------------------------------------------------------------------------------------------------------------
+
+ class AkkaRemoteProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) {
val commons_codec = Dependencies.commons_codec
val commons_io = Dependencies.commons_io
- val configgy = Dependencies.configgy
val dispatch_http = Dependencies.dispatch_http
val dispatch_json = Dependencies.dispatch_json
val guicey = Dependencies.guicey
val h2_lzf = Dependencies.h2_lzf
- val hawtdispatch = Dependencies.hawtdispatch
val jackson = Dependencies.jackson
val jackson_core = Dependencies.jackson_core
val jgroups = Dependencies.jgroups
- val jsr166x = Dependencies.jsr166x
val jta_1_1 = Dependencies.jta_1_1
- val multiverse = Dependencies.multiverse
val netty = Dependencies.netty
val protobuf = Dependencies.protobuf
val sbinary = Dependencies.sbinary
val sjson = Dependencies.sjson
- val werkz = Dependencies.werkz
- val werkz_core = Dependencies.werkz_core
- val slf4j = Dependencies.slf4j
- val logback = Dependencies.logback
- val logback_core = Dependencies.logback_core
// testing
val junit = Dependencies.junit
@@ -419,7 +445,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
class AkkaPersistenceParentProject(info: ProjectInfo) extends ParentProject(info) {
lazy val akka_persistence_common = project("akka-persistence-common", "akka-persistence-common",
- new AkkaPersistenceCommonProject(_), akka_core)
+ new AkkaPersistenceCommonProject(_), akka_remote)
lazy val akka_persistence_redis = project("akka-persistence-redis", "akka-persistence-redis",
new AkkaRedisProject(_), akka_persistence_common)
lazy val akka_persistence_mongo = project("akka-persistence-mongo", "akka-persistence-mongo",
@@ -514,7 +540,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val akka_osgi_dependencies_bundle = project("akka-osgi-dependencies-bundle", "akka-osgi-dependencies-bundle",
new AkkaOSGiDependenciesBundleProject(_), akka_kernel, akka_jta) // akka_kernel does not depend on akka_jta (why?) therefore we list akka_jta here
lazy val akka_osgi_assembly = project("akka-osgi-assembly", "akka-osgi-assembly",
- new AkkaOSGiAssemblyProject(_), akka_osgi_dependencies_bundle, akka_core, akka_amqp, akka_http,
+ new AkkaOSGiAssemblyProject(_), akka_osgi_dependencies_bundle, akka_remote, akka_amqp, akka_http,
akka_camel, akka_spring, akka_jta, akka_persistence.akka_persistence_common,
akka_persistence.akka_persistence_redis, akka_persistence.akka_persistence_mongo,
akka_persistence.akka_persistence_cassandra)
@@ -675,7 +701,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
class AkkaSamplesParentProject(info: ProjectInfo) extends ParentProject(info) {
lazy val akka_sample_ants = project("akka-sample-ants", "akka-sample-ants",
- new AkkaSampleAntsProject(_), akka_core)
+ new AkkaSampleAntsProject(_), akka_remote)
lazy val akka_sample_chat = project("akka-sample-chat", "akka-sample-chat",
new AkkaSampleChatProject(_), akka_kernel)
lazy val akka_sample_pubsub = project("akka-sample-pubsub", "akka-sample-pubsub",
@@ -695,7 +721,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val akka_sample_remote = project("akka-sample-remote", "akka-sample-remote",
new AkkaSampleRemoteProject(_), akka_kernel)
lazy val akka_sample_osgi = project("akka-sample-osgi", "akka-sample-osgi",
- new AkkaSampleOSGiProject(_), akka_core)
+ new AkkaSampleOSGiProject(_), akka_remote)
}
// -------------------------------------------------------------------------------------------------------------------
@@ -730,6 +756,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
// ------------------------------------------------------------
class AkkaDefaultProject(info: ProjectInfo, val deployPath: Path) extends DefaultProject(info) with DeployProject with OSGiProject {
+ override def runClasspath = super.runClasspath +++ (AkkaParentProject.this.info.projectPath / "config")
+ override def testClasspath = super.testClasspath +++ (AkkaParentProject.this.info.projectPath / "config")
override def packageDocsJar = this.defaultJarPath("-docs.jar")
override def packageSrcJar = this.defaultJarPath("-sources.jar")
}