diff --git a/.gitignore b/.gitignore index 0d810187b9..02f7ff993d 100755 --- a/.gitignore +++ b/.gitignore @@ -1,4 +1,6 @@ *# +*.log +*.orig *.jfr *.iml *.ipr diff --git a/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala index 0625e9ce73..7d14b28d6d 100644 --- a/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala @@ -5,7 +5,6 @@ package akka.serialization import language.postfixOps - import akka.testkit.{ AkkaSpec, EventFilter } import akka.actor._ import akka.dispatch.sysmsg._ @@ -17,6 +16,8 @@ import scala.beans.BeanInfo import com.typesafe.config._ import akka.pattern.ask import org.apache.commons.codec.binary.Hex.encodeHex +import java.nio.ByteOrder +import java.nio.ByteBuffer import akka.actor.NoSerializationVerificationNeeded import test.akka.serialization.NoVerification @@ -249,7 +250,25 @@ class SerializeSpec extends AkkaSpec(SerializationTests.serializeConf) { intercept[IllegalArgumentException] { byteSerializer.toBinary("pigdog") - }.getMessage should ===("ByteArraySerializer only serializes byte arrays, not [pigdog]") + }.getMessage should ===(s"${classOf[ByteArraySerializer].getName} only serializes byte arrays, not [java.lang.String]") + } + + "support ByteBuffer serialization for byte arrays" in { + val byteSerializer = ser.serializerFor(classOf[Array[Byte]]).asInstanceOf[ByteBufferSerializer] + + val byteBuffer = ByteBuffer.allocate(128).order(ByteOrder.LITTLE_ENDIAN) + val str = "abcdef" + val payload = str.getBytes("UTF-8") + byteSerializer.toBinary(payload, byteBuffer) + byteBuffer.position() should ===(payload.length) + byteBuffer.flip() + val deserialized = byteSerializer.fromBinary(byteBuffer, "").asInstanceOf[Array[Byte]] + byteBuffer.remaining() should ===(0) + new String(deserialized, "UTF-8") should ===(str) + + intercept[IllegalArgumentException] { + byteSerializer.toBinary("pigdog", byteBuffer) + }.getMessage should ===(s"${classOf[ByteArraySerializer].getName} only serializes byte arrays, not [java.lang.String]") } } } diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index 131f9a83ea..10ff0eee8d 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -83,9 +83,11 @@ akka { actor { + # Either one of "local", "remote" or "cluster" or the # FQCN of the ActorRefProvider to be used; the below is the built-in default, - # another one is akka.remote.RemoteActorRefProvider in the akka-remote bundle. - provider = "akka.actor.LocalActorRefProvider" + # note that "remote" and "cluster" requires the akka-remote and akka-cluster + # artifacts to be on the classpath. + provider = "local" # The guardian "/user" will use this class to obtain its supervisorStrategy. # It needs to be a subclass of akka.actor.SupervisorStrategyConfigurator. @@ -583,6 +585,19 @@ akka { "[B" = bytes "java.io.Serializable" = java } + + # Set this to on to enable serialization-bindings define in + # additional-serialization-bindings. Those are by default not included + # for backwards compatibility reasons. They are enabled by default if + # akka.remote.artery.enabled=on. + enable-additional-serialization-bindings = off + + # Additional serialization-bindings that are replacing Java serialization are + # defined in this section and not included by default for backwards compatibility + # reasons. They can be enabled with enable-additional-serialization-bindings=on. + # They are enabled by default if akka.remote.artery.enabled=on. + additional-serialization-bindings { + } # Log warnings when the default Java serialization is used to serialize messages. # The default serializer uses Java serialization which is not very performant and should not @@ -604,7 +619,7 @@ akka { # Identifier values from 0 to 16 are reserved for Akka internal usage. serialization-identifiers { "akka.serialization.JavaSerializer" = 1 - "akka.serialization.ByteArraySerializer" = 4 + "akka.serialization.ByteArraySerializer" = 4 } # Configuration items which are used by the akka.actor.ActorDSL._ methods diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index c0789e1481..bd9afb83a2 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -166,7 +166,15 @@ object ActorSystem { import config._ final val ConfigVersion: String = getString("akka.version") - final val ProviderClass: String = getString("akka.actor.provider") + final val ProviderClass: String = + getString("akka.actor.provider") match { + case "local" ⇒ classOf[LocalActorRefProvider].getName + // these two cannot be referenced by class as they may not be on the classpath + case "remote" ⇒ "akka.remote.RemoteActorRefProvider" + case "cluster" ⇒ "akka.cluster.ClusterActorRefProvider" + case fqcn ⇒ fqcn + } + final val SupervisorStrategyClass: String = getString("akka.actor.guardian-supervisor-strategy") final val CreationTimeout: Timeout = Timeout(config.getMillisDuration("akka.actor.creation-timeout")) final val UnstartedPushTimeout: Timeout = Timeout(config.getMillisDuration("akka.actor.unstarted-push-timeout")) @@ -831,6 +839,7 @@ private[akka] class ActorSystemImpl( /** * Adds a Runnable that will be executed on ActorSystem termination. * Note that callbacks are executed in reverse order of insertion. + * * @param r The callback to be executed on ActorSystem termination * Throws RejectedExecutionException if called after ActorSystem has been terminated. */ diff --git a/akka-actor/src/main/scala/akka/actor/Address.scala b/akka-actor/src/main/scala/akka/actor/Address.scala index f5ba1fb781..ac5f90d32d 100644 --- a/akka-actor/src/main/scala/akka/actor/Address.scala +++ b/akka-actor/src/main/scala/akka/actor/Address.scala @@ -76,6 +76,18 @@ object Address { * Constructs a new Address with the specified protocol, system name, host and port */ def apply(protocol: String, system: String, host: String, port: Int) = new Address(protocol, system, Some(host), Some(port)) + + /** + * `Address` ordering type class, sorts addresses by protocol, name, host and port. + */ + implicit val addressOrdering: Ordering[Address] = Ordering.fromLessThan[Address] { (a, b) ⇒ + if (a eq b) false + else if (a.protocol != b.protocol) a.system.compareTo(b.protocol) < 0 + else if (a.system != b.system) a.system.compareTo(b.system) < 0 + else if (a.host != b.host) a.host.getOrElse("").compareTo(b.host.getOrElse("")) < 0 + else if (a.port != b.port) a.port.getOrElse(0) < b.port.getOrElse(0) + else false + } } private[akka] trait PathUtils { diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index b2dbae6ea3..8e527bd1e6 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -396,6 +396,17 @@ object Logging { n.substring(i + 1) } + /** + * Class name representation of a message. + * `ActorSelectionMessage` representation includes class name of + * wrapped message. + */ + def messageClassName(message: Any): String = message match { + case null ⇒ "null" + case ActorSelectionMessage(m, _, _) ⇒ s"ActorSelectionMessage(${m.getClass.getName})" + case m ⇒ m.getClass.getName + } + /** * INTERNAL API */ diff --git a/akka-actor/src/main/scala/akka/io/DirectByteBufferPool.scala b/akka-actor/src/main/scala/akka/io/DirectByteBufferPool.scala index fdde031dda..5c03de0603 100644 --- a/akka-actor/src/main/scala/akka/io/DirectByteBufferPool.scala +++ b/akka-actor/src/main/scala/akka/io/DirectByteBufferPool.scala @@ -71,6 +71,29 @@ private[akka] class DirectByteBufferPool(defaultBufferSize: Int, maxPoolEntries: tryCleanDirectByteBuffer(buf) } + private final def tryCleanDirectByteBuffer(toBeDestroyed: ByteBuffer): Unit = DirectByteBufferPool.tryCleanDirectByteBuffer(toBeDestroyed) +} + +/** INTERNAL API */ +private[akka] object DirectByteBufferPool { + private val CleanDirectBuffer: ByteBuffer ⇒ Unit = + try { + val cleanerMethod = Class.forName("java.nio.DirectByteBuffer").getMethod("cleaner") + cleanerMethod.setAccessible(true) + + val cleanMethod = Class.forName("sun.misc.Cleaner").getMethod("clean") + cleanMethod.setAccessible(true) + + { (bb: ByteBuffer) ⇒ + try + if (bb.isDirect) { + val cleaner = cleanerMethod.invoke(bb) + cleanerMethod.invoke(cleaner) + } + catch { case NonFatal(e) ⇒ /* ok, best effort attempt to cleanup failed */ } + } + } catch { case NonFatal(e) ⇒ _ ⇒ () /* reflection failed, use no-op fallback */ } + /** * DirectByteBuffers are garbage collected by using a phantom reference and a * reference queue. Every once a while, the JVM checks the reference queue and @@ -81,16 +104,5 @@ private[akka] class DirectByteBufferPool(defaultBufferSize: Int, maxPoolEntries: * * Utilizes reflection to avoid dependency to `sun.misc.Cleaner`. */ - private final def tryCleanDirectByteBuffer(toBeDestroyed: ByteBuffer): Unit = try { - if (toBeDestroyed.isDirect) { - val cleanerMethod = toBeDestroyed.getClass().getMethod("cleaner") - cleanerMethod.setAccessible(true) - val cleaner = cleanerMethod.invoke(toBeDestroyed) - val cleanMethod = cleaner.getClass().getMethod("clean") - cleanMethod.setAccessible(true) - cleanMethod.invoke(cleaner) - } - } catch { - case NonFatal(_) ⇒ // attempt failed, ok - } -} + def tryCleanDirectByteBuffer(byteBuffer: ByteBuffer): Unit = CleanDirectBuffer(byteBuffer) +} \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/routing/MurmurHash.scala b/akka-actor/src/main/scala/akka/routing/MurmurHash.scala index fc67613a5d..f4fb81edb5 100644 --- a/akka-actor/src/main/scala/akka/routing/MurmurHash.scala +++ b/akka-actor/src/main/scala/akka/routing/MurmurHash.scala @@ -68,11 +68,11 @@ object MurmurHash { /** * Incorporates a new value into an existing hash. * - * @param hash the prior hash value - * @param value the new value to incorporate - * @param magicA a magic integer from the stream - * @param magicB a magic integer from a different stream - * @return the updated hash value + * @param hash the prior hash value + * @param value the new value to incorporate + * @param magicA a magic integer from the stream + * @param magicB a magic integer from a different stream + * @return the updated hash value */ def extendHash(hash: Int, value: Int, magicA: Int, magicB: Int): Int = (hash ^ rotl(value * magicA, 11) * magicB) * 3 + visibleMixer diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index bd1fb22fd0..7dfdf6eff1 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -14,6 +14,10 @@ import scala.util.{ Try, DynamicVariable, Failure } import scala.collection.immutable import scala.util.control.NonFatal import scala.util.Success +import java.nio.ByteBuffer +import java.util.concurrent.atomic.AtomicReference +import scala.annotation.tailrec +import java.util.NoSuchElementException object Serialization { @@ -30,12 +34,20 @@ object Serialization { private[akka] val currentTransportInformation = new DynamicVariable[Information](null) class Settings(val config: Config) { - val Serializers: Map[String, String] = configToMap("akka.actor.serializers") - val SerializationBindings: Map[String, String] = configToMap("akka.actor.serialization-bindings") + val Serializers: Map[String, String] = configToMap(config.getConfig("akka.actor.serializers")) + val SerializationBindings: Map[String, String] = { + val defaultBindings = config.getConfig("akka.actor.serialization-bindings") + val bindings = + if (config.getBoolean("akka.actor.enable-additional-serialization-bindings") || + config.getBoolean("akka.remote.artery.enabled")) + defaultBindings.withFallback(config.getConfig("akka.actor.additional-serialization-bindings")) + else defaultBindings + configToMap(bindings) + } - private final def configToMap(path: String): Map[String, String] = { + private final def configToMap(cfg: Config): Map[String, String] = { import scala.collection.JavaConverters._ - config.getConfig(path).root.unwrapped.asScala.toMap map { case (k, v) ⇒ (k → v.toString) } + cfg.root.unwrapped.asScala.toMap map { case (k, v) ⇒ (k → v.toString) } } } @@ -83,6 +95,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { val settings = new Settings(system.settings.config) val log = Logging(system, getClass.getName) + private val manifestCache = new AtomicReference[Map[String, Option[Class[_]]]](Map.empty[String, Option[Class[_]]]) /** * Serializes the given AnyRef/java.lang.Object according to the Serialization configuration @@ -97,7 +110,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { */ def deserialize[T](bytes: Array[Byte], serializerId: Int, clazz: Option[Class[_ <: T]]): Try[T] = Try { - val serializer = try serializerByIdentity(serializerId) catch { + val serializer = try getSerializerById(serializerId) catch { case _: NoSuchElementException ⇒ throw new NotSerializableException( s"Cannot find serializer with id [$serializerId]. The most probable reason is that the configuration entry " + "akka.actor.serializers is not in synch between the two systems.") @@ -112,28 +125,66 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { */ def deserialize(bytes: Array[Byte], serializerId: Int, manifest: String): Try[AnyRef] = Try { - val serializer = try serializerByIdentity(serializerId) catch { + val serializer = try getSerializerById(serializerId) catch { case _: NoSuchElementException ⇒ throw new NotSerializableException( s"Cannot find serializer with id [$serializerId]. The most probable reason is that the configuration entry " + "akka.actor.serializers is not in synch between the two systems.") } - serializer match { - case s2: SerializerWithStringManifest ⇒ s2.fromBinary(bytes, manifest) - case s1 ⇒ - if (manifest == "") - s1.fromBinary(bytes, None) - else { - system.dynamicAccess.getClassFor[AnyRef](manifest) match { - case Success(classManifest) ⇒ - s1.fromBinary(bytes, Some(classManifest)) - case Failure(e) ⇒ - throw new NotSerializableException( - s"Cannot find manifest class [$manifest] for serializer with id [$serializerId].") - } - } - } + deserializeByteArray(bytes, serializer, manifest) } + private def deserializeByteArray(bytes: Array[Byte], serializer: Serializer, manifest: String): AnyRef = { + + @tailrec def updateCache(cache: Map[String, Option[Class[_]]], key: String, value: Option[Class[_]]): Boolean = { + manifestCache.compareAndSet(cache, cache.updated(key, value)) || + updateCache(manifestCache.get, key, value) // recursive, try again + } + + serializer match { + case s2: SerializerWithStringManifest ⇒ s2.fromBinary(bytes, manifest) + case s1 ⇒ + if (manifest == "") + s1.fromBinary(bytes, None) + else { + val cache = manifestCache.get + cache.get(manifest) match { + case Some(cachedClassManifest) ⇒ s1.fromBinary(bytes, cachedClassManifest) + case None ⇒ + system.dynamicAccess.getClassFor[AnyRef](manifest) match { + case Success(classManifest) ⇒ + val classManifestOption: Option[Class[_]] = Some(classManifest) + updateCache(cache, manifest, classManifestOption) + s1.fromBinary(bytes, classManifestOption) + case Failure(e) ⇒ + throw new NotSerializableException( + s"Cannot find manifest class [$manifest] for serializer with id [${serializer.identifier}].") + } + } + } + } + } + + /** + * Deserializes the given ByteBuffer of bytes using the specified serializer id, + * using the optional type hint to the Serializer. + * Returns either the resulting object or throws an exception if deserialization fails. + */ + def deserializeByteBuffer(buf: ByteBuffer, serializerId: Int, manifest: String): AnyRef = { + val serializer = try getSerializerById(serializerId) catch { + case _: NoSuchElementException ⇒ throw new NotSerializableException( + s"Cannot find serializer with id [$serializerId]. The most probable reason is that the configuration entry " + + "akka.actor.serializers is not in synch between the two systems.") + } + serializer match { + case ser: ByteBufferSerializer ⇒ + ser.fromBinary(buf, manifest) + case _ ⇒ + val bytes = Array.ofDim[Byte](buf.remaining()) + buf.get(bytes) + deserializeByteArray(bytes, serializer, manifest) + } + } + /** * Deserializes the given array of bytes using the specified type to look up what Serializer should be used. * Returns either the resulting object or an Exception if one was thrown. @@ -246,6 +297,31 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { val serializerByIdentity: Map[Int, Serializer] = Map(NullSerializer.identifier → NullSerializer) ++ serializers map { case (_, v) ⇒ (v.identifier, v) } + /** + * Serializers with id 0 - 1023 are stored in an array for quick allocation free access + */ + private val quickSerializerByIdentity: Array[Serializer] = { + val size = 1024 + val table = Array.ofDim[Serializer](size) + serializerByIdentity.foreach { + case (id, ser) ⇒ if (0 <= id && id < size) table(id) = ser + } + table + } + + /** + * @throws `NoSuchElementException` if no serializer with given `id` + */ + private def getSerializerById(id: Int): Serializer = { + if (0 <= id && id < quickSerializerByIdentity.length) { + quickSerializerByIdentity(id) match { + case null ⇒ throw new NoSuchElementException(s"key not found: $id") + case ser ⇒ ser + } + } else + serializerByIdentity(id) + } + private val isJavaSerializationWarningEnabled = settings.config.getBoolean("akka.actor.warn-about-java-serializer-usage") private val isWarningOnNoVerificationEnabled = settings.config.getBoolean("akka.actor.warn-on-no-serialization-verification") diff --git a/akka-actor/src/main/scala/akka/serialization/Serializer.scala b/akka-actor/src/main/scala/akka/serialization/Serializer.scala index 51bcc545ad..3c7c742da6 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serializer.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serializer.scala @@ -5,6 +5,7 @@ package akka.serialization */ import java.io.{ ObjectOutputStream, ByteArrayOutputStream, ByteArrayInputStream } +import java.nio.ByteBuffer import java.util.concurrent.Callable import akka.util.ClassLoaderObjectInputStream import akka.actor.ExtendedActorSystem @@ -132,6 +133,56 @@ abstract class SerializerWithStringManifest extends Serializer { } +/** + * Serializer between an object and a `ByteBuffer` representing that object. + * + * Implementations should typically extend [[SerializerWithStringManifest]] and + * in addition to the `ByteBuffer` based `toBinary` and `fromBinary` methods also + * implement the array based `toBinary` and `fromBinary` methods. The array based + * methods will be used when `ByteBuffer` is not used, e.g. in Akka Persistence. + * + * Note that the array based methods can for example be implemented by delegation + * like this: + * {{{ + * // you need to know the maximum size in bytes of the serialized messages + * val pool = new akka.io.DirectByteBufferPool(defaultBufferSize = 1024 * 1024, maxPoolEntries = 10) + * + * + * // Implement this method for compatibility with `SerializerWithStringManifest`. + * override def toBinary(o: AnyRef): Array[Byte] = { + * val buf = pool.acquire() + * try { + * toBinary(o, buf) + * buf.flip() + * val bytes = Array.ofDim[Byte](buf.remaining) + * buf.get(bytes) + * bytes + * } finally { + * pool.release(buf) + * } + * } + * + * // Implement this method for compatibility with `SerializerWithStringManifest`. + * override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef = + * fromBinary(ByteBuffer.wrap(bytes), manifest) + * + * }}} + */ +trait ByteBufferSerializer { + + /** + * Serializes the given object into the `ByteBuffer`. + */ + def toBinary(o: AnyRef, buf: ByteBuffer): Unit + + /** + * Produces an object from a `ByteBuffer`, with an optional type-hint; + * the class should be loaded using ActorSystem.dynamicAccess. + */ + def fromBinary(buf: ByteBuffer, manifest: String): AnyRef + +} + /** * Base serializer trait with serialization identifiers configuration contract, * when globally unique serialization identifier is configured in the `reference.conf`. @@ -252,7 +303,7 @@ class NullSerializer extends Serializer { val nullAsBytes = Array[Byte]() def includeManifest: Boolean = false def identifier = 0 - def toBinary(o: AnyRef) = nullAsBytes + def toBinary(o: AnyRef): Array[Byte] = nullAsBytes def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = null } @@ -260,7 +311,7 @@ class NullSerializer extends Serializer { * This is a special Serializer that Serializes and deserializes byte arrays only, * (just returns the byte array unchanged/uncopied) */ -class ByteArraySerializer(val system: ExtendedActorSystem) extends BaseSerializer { +class ByteArraySerializer(val system: ExtendedActorSystem) extends BaseSerializer with ByteBufferSerializer { @deprecated("Use constructor with ExtendedActorSystem", "2.4") def this() = this(null) @@ -271,10 +322,25 @@ class ByteArraySerializer(val system: ExtendedActorSystem) extends BaseSerialize else identifierFromConfig def includeManifest: Boolean = false - def toBinary(o: AnyRef) = o match { + def toBinary(o: AnyRef): Array[Byte] = o match { case null ⇒ null case o: Array[Byte] ⇒ o - case other ⇒ throw new IllegalArgumentException("ByteArraySerializer only serializes byte arrays, not [" + other + "]") + case other ⇒ throw new IllegalArgumentException( + s"${getClass.getName} only serializes byte arrays, not [${other.getClass.getName}]") } def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = bytes + + override def toBinary(o: AnyRef, buf: ByteBuffer): Unit = + o match { + case null ⇒ + case bytes: Array[Byte] ⇒ buf.put(bytes) + case other ⇒ throw new IllegalArgumentException( + s"${getClass.getName} only serializes byte arrays, not [${other.getClass.getName}]") + } + + override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = { + val bytes = Array.ofDim[Byte](buf.remaining()) + buf.get(bytes) + bytes + } } diff --git a/akka-actor/src/main/scala/akka/util/ByteString.scala b/akka-actor/src/main/scala/akka/util/ByteString.scala index 8196495d7c..6588a9917c 100644 --- a/akka-actor/src/main/scala/akka/util/ByteString.scala +++ b/akka-actor/src/main/scala/akka/util/ByteString.scala @@ -178,11 +178,15 @@ object ByteString { val copyLength = Math.min(buffer.remaining, offset + length) if (copyLength > 0) { buffer.put(bytes, offset, copyLength) - drop(copyLength) } copyLength } + /** INTERNAL API: Specialized for internal use, appending ByteString1C to a ByteStringBuilder. */ + private[akka] def appendToBuilder(buffer: ByteStringBuilder) = { + buffer.putByteArrayUnsafe(bytes) + } + } /** INTERNAL API: ByteString backed by exactly one array, with start / end markers */ diff --git a/akka-actor/src/main/scala/akka/util/PrettyByteString.scala b/akka-actor/src/main/scala/akka/util/PrettyByteString.scala new file mode 100644 index 0000000000..85b6a38005 --- /dev/null +++ b/akka-actor/src/main/scala/akka/util/PrettyByteString.scala @@ -0,0 +1,44 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.util + +/** + * INTERNAL API: ByteString pretty printer, based on Johanes Rudolph's implementation from: + * https://github.com/jrudolph/akka/commit/c889dddf37c8635c365a79a391eb18a709f36773#diff-947cbf07996eeb823cb9850cc2e81126R19 + */ +private[akka] object PrettyByteString { + private val indentDepth = 2 + private val indent = " " * (indentDepth + 1) + + implicit class asPretty(bs: ByteString) { + def prettyPrint(maxBytes: Int = 16 * 5): String = formatBytes(bs, maxBytes).mkString("\n") + } + + def formatBytes(bs: ByteString, maxBytes: Int = 16 * 5): Iterator[String] = { + def asHex(b: Byte): String = b formatted "%02X" + def asASCII(b: Byte): Char = + if (b >= 0x20 && b < 0x7f) b.toChar + else '.' + + def formatLine(bs: ByteString): String = { + val data = bs.toSeq + val hex = data.map(asHex).mkString(" ") + val ascii = data.map(asASCII).mkString + f"$indent%s $hex%-48s | $ascii" + } + def formatBytes(bs: ByteString): String = + bs.grouped(16).map(formatLine).mkString("\n") + + val prefix = s"${indent}ByteString(${bs.size} bytes)" + + if (bs.size <= maxBytes) Iterator(prefix + "\n", formatBytes(bs)) + else + Iterator( + s"$prefix first + last $maxBytes:\n", + formatBytes(bs.take(maxBytes)), + s"\n$indent ... [${bs.size - maxBytes} bytes omitted] ...\n", + formatBytes(bs.takeRight(maxBytes))) + } + +} diff --git a/akka-actor/src/main/scala/akka/util/PrettyDuration.scala b/akka-actor/src/main/scala/akka/util/PrettyDuration.scala index 9471151864..0f9694b8b9 100644 --- a/akka-actor/src/main/scala/akka/util/PrettyDuration.scala +++ b/akka-actor/src/main/scala/akka/util/PrettyDuration.scala @@ -7,7 +7,8 @@ import java.util.Locale import scala.concurrent.duration._ -object PrettyDuration { +/** INTERNAL API */ +private[akka] object PrettyDuration { /** * JAVA API diff --git a/akka-bench-jmh/src/main/scala/akka/remote/artery/BenchTestSource.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/BenchTestSource.scala new file mode 100644 index 0000000000..dc4f12f1bb --- /dev/null +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/BenchTestSource.scala @@ -0,0 +1,63 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.stream.Attributes +import akka.stream.Outlet +import akka.stream.SourceShape +import akka.stream.stage.GraphStage +import akka.stream.stage.GraphStageLogic +import akka.stream.stage.OutHandler + +/** + * Emits integers from 1 to the given `elementCount`. The `java.lang.Integer` + * objects are allocated in the constructor of the stage, so it should be created + * before the benchmark is started. + */ +class BenchTestSource(elementCount: Int) extends GraphStage[SourceShape[java.lang.Integer]] { + + private val elements = Array.ofDim[java.lang.Integer](elementCount) + (1 to elementCount).map(n => elements(n - 1) = n) + + val out: Outlet[java.lang.Integer] = Outlet("BenchTestSource") + override val shape: SourceShape[java.lang.Integer] = SourceShape(out) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new GraphStageLogic(shape) with OutHandler { + + var n = 0 + + override def onPull(): Unit = { + n += 1 + if (n > elementCount) + complete(out) + else + push(out, elements(n - 1)) + } + + setHandler(out, this) + } +} + +class BenchTestSourceSameElement[T](elements: Int, elem: T) extends GraphStage[SourceShape[T]] { + + val out: Outlet[T] = Outlet("BenchTestSourceSameElement") + override val shape: SourceShape[T] = SourceShape(out) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new GraphStageLogic(shape) with OutHandler { + + var n = 0 + + override def onPull(): Unit = { + n += 1 + if (n > elements) + complete(out) + else + push(out, elem) + } + + setHandler(out, this) + } +} diff --git a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala new file mode 100644 index 0000000000..6203e053e2 --- /dev/null +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -0,0 +1,229 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.io.File +import java.nio.ByteBuffer +import java.nio.ByteOrder +import java.nio.channels.FileChannel +import java.util.concurrent.CountDownLatch +import java.util.concurrent.TimeUnit + +import akka.remote.artery.compress._ +import akka.stream.impl.ConstantFun +import org.openjdk.jmh.annotations.Scope + +import scala.concurrent.Await +import scala.concurrent.duration._ +import akka.NotUsed +import akka.actor._ +import akka.remote.AddressUidExtension +import akka.remote.RARP +import akka.remote.RemoteActorRef +import akka.remote.UniqueAddress +import akka.stream.ActorMaterializer +import akka.stream.ActorMaterializerSettings +import akka.stream.scaladsl._ +import com.typesafe.config.ConfigFactory +import org.openjdk.jmh.annotations._ +import akka.util.OptionVal +import akka.actor.Address +import scala.concurrent.Future +import akka.Done + +@State(Scope.Benchmark) +@OutputTimeUnit(TimeUnit.MILLISECONDS) +@BenchmarkMode(Array(Mode.Throughput)) +@Fork(2) +@Warmup(iterations = 4) +@Measurement(iterations = 5) +class CodecBenchmark { + + val config = ConfigFactory.parseString( + """ + akka { + loglevel = WARNING + actor.provider = remote + remote.artery.enabled = on + remote.artery.hostname = localhost + remote.artery.port = 0 + } + """ + ) + + implicit val system = ActorSystem("CodecBenchmark", config) + val systemB = ActorSystem("systemB", system.settings.config) + + private val envelopePool = new EnvelopeBufferPool(1024 * 1024, 128) + private val inboundEnvelopePool = ReusableInboundEnvelope.createObjectPool(capacity = 16) + private val outboundEnvelopePool = ReusableOutboundEnvelope.createObjectPool(capacity = 16) + + val headerIn = HeaderBuilder.in(NoInboundCompressions) + val envelopeTemplateBuffer = ByteBuffer.allocate(1024 * 1024).order(ByteOrder.LITTLE_ENDIAN) + + val uniqueLocalAddress = UniqueAddress( + system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress, + AddressUidExtension(system).longAddressUid + ) + val payload = Array.ofDim[Byte](1000) + + private val inboundContext: InboundContext = new InboundContext { + override def localAddress: UniqueAddress = uniqueLocalAddress + override def association(uid: Long): OptionVal[OutboundContext] = OptionVal.None + // the following methods are not used by in this test + override def sendControl(to: Address, message: ControlMessage): Unit = ??? + override def association(remoteAddress: Address): OutboundContext = ??? + override def completeHandshake(peer: UniqueAddress): Future[Done] = ??? + override lazy val settings: ArterySettings = + ArterySettings(ConfigFactory.load().getConfig("akka.remote.artery")) + } + + private var materializer: ActorMaterializer = _ + private var remoteRefB: RemoteActorRef = _ + private var resolvedRef: InternalActorRef = _ + private var senderStringA: String = _ + private var recipientStringB: String = _ + + @Setup + def setup(): Unit = { + val settings = ActorMaterializerSettings(system) + materializer = ActorMaterializer(settings) + + val actorOnSystemA = system.actorOf(Props.empty, "a") + senderStringA = actorOnSystemA.path.toSerializationFormatWithAddress(uniqueLocalAddress.address) + + val actorOnSystemB = systemB.actorOf(Props.empty, "b") + val addressB = systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + val rootB = RootActorPath(addressB) + remoteRefB = + Await.result(system.actorSelection(rootB / "user" / "b").resolveOne(5.seconds), 5.seconds) + .asInstanceOf[RemoteActorRef] + resolvedRef = actorOnSystemA.asInstanceOf[InternalActorRef] + recipientStringB = remoteRefB.path.toSerializationFormatWithAddress(addressB) + + val envelope = new EnvelopeBuffer(envelopeTemplateBuffer) + headerIn setVersion 1 + headerIn setUid 42 + headerIn setSerializer 4 + headerIn setSenderActorRef actorOnSystemA + headerIn setRecipientActorRef remoteRefB + headerIn setManifest "" + envelope.writeHeader(headerIn) + envelope.byteBuffer.put(payload) + envelope.byteBuffer.flip() + } + + @TearDown + def shutdown(): Unit = { + Await.result(system.terminate(), 5.seconds) + Await.result(systemB.terminate(), 5.seconds) + } + + @Benchmark + @OperationsPerInvocation(100000) + def reference(): Unit = { + val latch = new CountDownLatch(1) + val N = 100000 + + Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) + .runWith(new LatchSink(N, latch))(materializer) + + if (!latch.await(30, TimeUnit.SECONDS)) + throw new RuntimeException("Latch didn't complete in time") + } + + @Benchmark + @OperationsPerInvocation(100000) + def encode(): Unit = { + val latch = new CountDownLatch(1) + val N = 100000 + + val encoder: Flow[OutboundEnvelope, EnvelopeBuffer, Encoder.ChangeOutboundCompression] = + Flow.fromGraph(new Encoder(uniqueLocalAddress, system.asInstanceOf[ExtendedActorSystem], outboundEnvelopePool, envelopePool, false)) + + Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) + .map(msg ⇒ outboundEnvelopePool.acquire().init(OptionVal.None, payload, OptionVal.Some(remoteRefB))) + .via(encoder) + .map(envelope => envelopePool.release(envelope)) + .runWith(new LatchSink(N, latch))(materializer) + + if (!latch.await(30, TimeUnit.SECONDS)) + throw new RuntimeException("Latch didn't complete in time") + } + + @Benchmark + @OperationsPerInvocation(100000) + def decode(): Unit = { + val latch = new CountDownLatch(1) + val N = 100000 + + val localRecipient = resolvedRef.path.toSerializationFormatWithAddress(uniqueLocalAddress.address) + val provider = RARP(system).provider + val resolveActorRefWithLocalAddress: String ⇒ InternalActorRef = { + recipient ⇒ + // juggling with the refs, since we don't run the real thing + val resolved = provider.resolveActorRefWithLocalAddress(localRecipient, uniqueLocalAddress.address) + resolved + } + + val decoder: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = + Flow.fromGraph(new Decoder(inboundContext, system.asInstanceOf[ExtendedActorSystem], + uniqueLocalAddress, NoInboundCompressions, envelopePool, inboundEnvelopePool)) + + Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) + .map { _ => + val envelope = envelopePool.acquire() + envelopeTemplateBuffer.rewind() + envelope.byteBuffer.put(envelopeTemplateBuffer) + envelope.byteBuffer.flip() + envelope + } + .via(decoder) + .map { + case env: ReusableInboundEnvelope => inboundEnvelopePool.release(env) + case _ => + } + .runWith(new LatchSink(N, latch))(materializer) + + if (!latch.await(30, TimeUnit.SECONDS)) + throw new RuntimeException("Latch didn't complete in time") + } + + @Benchmark + @OperationsPerInvocation(100000) + def encode_decode(): Unit = { + val latch = new CountDownLatch(1) + val N = 100000 + + val encoder: Flow[OutboundEnvelope, EnvelopeBuffer, Encoder.ChangeOutboundCompression] = + Flow.fromGraph(new Encoder(uniqueLocalAddress, system.asInstanceOf[ExtendedActorSystem], outboundEnvelopePool, envelopePool, false)) + + val localRecipient = resolvedRef.path.toSerializationFormatWithAddress(uniqueLocalAddress.address) + val provider = RARP(system).provider + val resolveActorRefWithLocalAddress: String ⇒ InternalActorRef = { + recipient ⇒ + // juggling with the refs, since we don't run the real thing + val resolved = provider.resolveActorRefWithLocalAddress(localRecipient, uniqueLocalAddress.address) + resolved + } + + val decoder: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = + Flow.fromGraph(new Decoder(inboundContext, system.asInstanceOf[ExtendedActorSystem], + uniqueLocalAddress, NoInboundCompressions, envelopePool, inboundEnvelopePool)) + + Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) + .map(msg ⇒ outboundEnvelopePool.acquire().init(OptionVal.None, payload, OptionVal.Some(remoteRefB))) + .via(encoder) + .via(decoder) + .map { + case env: ReusableInboundEnvelope => inboundEnvelopePool.release(env) + case _ => + } + .runWith(new LatchSink(N, latch))(materializer) + + if (!latch.await(30, TimeUnit.SECONDS)) + throw new RuntimeException("Latch didn't complete in time") + } + +} diff --git a/akka-bench-jmh/src/main/scala/akka/remote/artery/FlightRecorderBench.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/FlightRecorderBench.scala new file mode 100644 index 0000000000..9660405291 --- /dev/null +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/FlightRecorderBench.scala @@ -0,0 +1,65 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.io.File +import java.nio.channels.FileChannel +import java.nio.file.StandardOpenOption +import java.util.concurrent.{ CountDownLatch, TimeUnit } +import java.util.concurrent.TimeUnit + +import org.openjdk.jmh.annotations.{ OperationsPerInvocation, _ } + +@State(Scope.Benchmark) +@OutputTimeUnit(TimeUnit.MICROSECONDS) +@BenchmarkMode(Array(Mode.Throughput)) +class FlightRecorderBench { + + @Param(Array("1", "5", "10")) + var writers: Int = 0 + + val Writes = 10000000 + + private var file: File = _ + private var fileChannel: FileChannel = _ + private var recorder: FlightRecorder = _ + + @Setup + def setup(): Unit = { + file = File.createTempFile("akka-flightrecorder", "dat") + file.deleteOnExit() + fileChannel = FileChannel.open(file.toPath, StandardOpenOption.CREATE, StandardOpenOption.WRITE, StandardOpenOption.READ) + recorder = new FlightRecorder(fileChannel) + } + + @TearDown + def shutdown(): Unit = { + fileChannel.force(false) + recorder.close() + fileChannel.close() + file.delete() + } + + @Benchmark + @OperationsPerInvocation(10000000) + def flight_recorder_writes(): Unit = { + val latch = new CountDownLatch(writers) + (1 to writers).foreach { _ => + val sink = recorder.createEventSink() + new Thread { + override def run(): Unit = { + var i = Writes + while (i > 0) { + sink.hiFreq(16, 16) + i -= 1 + } + latch.countDown() + } + }.run() + } + + latch.await() + } + +} diff --git a/akka-bench-jmh/src/main/scala/akka/remote/artery/LatchSink.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/LatchSink.scala new file mode 100644 index 0000000000..6f1d2d539b --- /dev/null +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/LatchSink.scala @@ -0,0 +1,63 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.concurrent.CountDownLatch +import java.util.concurrent.CyclicBarrier + +import akka.stream.Attributes +import akka.stream.Inlet +import akka.stream.SinkShape +import akka.stream.stage.GraphStage +import akka.stream.stage.GraphStageLogic +import akka.stream.stage.InHandler + +class LatchSink(countDownAfter: Int, latch: CountDownLatch) extends GraphStage[SinkShape[Any]] { + val in: Inlet[Any] = Inlet("LatchSink") + override val shape: SinkShape[Any] = SinkShape(in) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new GraphStageLogic(shape) with InHandler { + + var n = 0 + + override def preStart(): Unit = pull(in) + + override def onPush(): Unit = { + n += 1 + if (n == countDownAfter) + latch.countDown() + grab(in) + pull(in) + } + + setHandler(in, this) + } +} + +class BarrierSink(countDownAfter: Int, latch: CountDownLatch, barrierAfter: Int, barrier: CyclicBarrier) + extends GraphStage[SinkShape[Any]] { + val in: Inlet[Any] = Inlet("BarrierSink") + override val shape: SinkShape[Any] = SinkShape(in) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new GraphStageLogic(shape) with InHandler { + + var n = 0 + + override def preStart(): Unit = pull(in) + + override def onPush(): Unit = { + n += 1 + grab(in) + if (n == countDownAfter) + latch.countDown() + else if (n % barrierAfter == 0) + barrier.await() + pull(in) + } + + setHandler(in, this) + } +} diff --git a/akka-bench-jmh/src/main/scala/akka/remote/artery/LiteralEncodingBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/LiteralEncodingBenchmark.scala new file mode 100644 index 0000000000..1cba3a4fa6 --- /dev/null +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/LiteralEncodingBenchmark.scala @@ -0,0 +1,107 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.nio.ByteBuffer +import java.nio.ByteOrder +import java.nio.charset.Charset +import java.util.concurrent.TimeUnit +import org.openjdk.jmh.annotations._ + +@State(Scope.Benchmark) +@OutputTimeUnit(TimeUnit.MILLISECONDS) +@BenchmarkMode(Array(Mode.Throughput)) +@Fork(2) +@Warmup(iterations = 5) +@Measurement(iterations = 10) +class LiteralEncodingBenchmark { + + private val UsAscii = Charset.forName("US-ASCII") + private val str = "akka://SomeSystem@host12:1234/user/foo" + private val buffer = ByteBuffer.allocate(128).order(ByteOrder.LITTLE_ENDIAN) + private val literalChars = Array.ofDim[Char](64) + private val literalBytes = Array.ofDim[Byte](64) + private val unsafe = akka.util.Unsafe.instance + private val stringValueFieldOffset = unsafe.objectFieldOffset(classOf[String].getDeclaredField("value")) + + @Benchmark + def getBytesNewArray(): String = { + val length = str.length() + // write + buffer.clear() + val bytes = str.getBytes(UsAscii) + buffer.put(bytes) + buffer.flip() + + // read + val bytes2 = Array.ofDim[Byte](length) + buffer.get(bytes2) + new String(bytes2, UsAscii) + } + + @Benchmark + def getBytesReuseArray(): String = { + val length = str.length() + // write + buffer.clear() + val bytes = str.getBytes(UsAscii) + buffer.put(bytes) + buffer.flip() + + // read + buffer.get(literalBytes, 0, length) + new String(literalBytes, UsAscii) + } + + @Benchmark + def getChars(): String = { + val length = str.length() + // write + buffer.clear() + str.getChars(0, length, literalChars, 0) + var i = 0 + while (i < length) { + literalBytes(i) = literalChars(i).asInstanceOf[Byte] + i += 1 + } + buffer.put(literalBytes, 0, length) + buffer.flip() + + // read + buffer.get(literalBytes, 0, length) + i = 0 + while (i < length) { + // UsAscii + literalChars(i) = literalBytes(i).asInstanceOf[Char] + i += 1 + } + String.valueOf(literalChars, 0, length) + } + + @Benchmark + def getCharsUnsafe(): String = { + val length = str.length() + // write + buffer.clear() + val chars = unsafe.getObject(str, stringValueFieldOffset).asInstanceOf[Array[Char]] + var i = 0 + while (i < length) { + literalBytes(i) = chars(i).asInstanceOf[Byte] + i += 1 + } + buffer.put(literalBytes, 0, length) + buffer.flip() + + // read + buffer.get(literalBytes, 0, length) + i = 0 + while (i < length) { + // UsAscii + literalChars(i) = literalBytes(i).asInstanceOf[Char] + i += 1 + } + String.valueOf(literalChars, 0, length) + } + +} diff --git a/akka-bench-jmh/src/main/scala/akka/remote/artery/SendQueueBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/SendQueueBenchmark.scala new file mode 100644 index 0000000000..2b131923f3 --- /dev/null +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/SendQueueBenchmark.scala @@ -0,0 +1,139 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.concurrent.TimeUnit +import akka.NotUsed +import akka.actor.ActorSystem +import akka.stream.scaladsl._ +import com.typesafe.config.ConfigFactory +import org.openjdk.jmh.annotations._ +import scala.concurrent.Lock +import scala.util.Success +import akka.stream.impl.fusing.GraphStages +import org.reactivestreams._ +import scala.concurrent.Await +import scala.concurrent.duration._ +import akka.stream.ActorMaterializer +import akka.stream.ActorMaterializerSettings +import java.util.concurrent.Semaphore +import akka.stream.OverflowStrategy +import java.util.concurrent.CyclicBarrier +import java.util.concurrent.CountDownLatch +import akka.stream.KillSwitches +import org.agrona.concurrent.ManyToOneConcurrentArrayQueue + +@State(Scope.Benchmark) +@OutputTimeUnit(TimeUnit.MILLISECONDS) +@BenchmarkMode(Array(Mode.Throughput)) +@Fork(2) +@Warmup(iterations = 4) +@Measurement(iterations = 10) +class SendQueueBenchmark { + + val config = ConfigFactory.parseString( + """ + """ + ) + + implicit val system = ActorSystem("SendQueueBenchmark", config) + + var materializer: ActorMaterializer = _ + + @Setup + def setup(): Unit = { + val settings = ActorMaterializerSettings(system) + materializer = ActorMaterializer(settings) + } + + @TearDown + def shutdown(): Unit = { + Await.result(system.terminate(), 5.seconds) + } + + @Benchmark + @OperationsPerInvocation(100000) + def queue(): Unit = { + val latch = new CountDownLatch(1) + val barrier = new CyclicBarrier(2) + val N = 100000 + val burstSize = 1000 + + val source = Source.queue[Int](1024, OverflowStrategy.dropBuffer) + + val (queue, killSwitch) = source.viaMat(KillSwitches.single)(Keep.both) + .toMat(new BarrierSink(N, latch, burstSize, barrier))(Keep.left).run()(materializer) + + var n = 1 + while (n <= N) { + queue.offer(n) + if (n % burstSize == 0 && n < N) { + barrier.await() + } + n += 1 + } + + if (!latch.await(30, TimeUnit.SECONDS)) + throw new RuntimeException("Latch didn't complete in time") + killSwitch.shutdown() + } + + @Benchmark + @OperationsPerInvocation(100000) + def actorRef(): Unit = { + val latch = new CountDownLatch(1) + val barrier = new CyclicBarrier(2) + val N = 100000 + val burstSize = 1000 + + val source = Source.actorRef(1024, OverflowStrategy.dropBuffer) + + val (ref, killSwitch) = source.viaMat(KillSwitches.single)(Keep.both) + .toMat(new BarrierSink(N, latch, burstSize, barrier))(Keep.left).run()(materializer) + + var n = 1 + while (n <= N) { + ref ! n + if (n % burstSize == 0 && n < N) { + barrier.await() + } + n += 1 + } + + if (!latch.await(30, TimeUnit.SECONDS)) + throw new RuntimeException("Latch didn't complete in time") + killSwitch.shutdown() + } + + @Benchmark + @OperationsPerInvocation(100000) + def sendQueue(): Unit = { + val latch = new CountDownLatch(1) + val barrier = new CyclicBarrier(2) + val N = 100000 + val burstSize = 1000 + + val queue = new ManyToOneConcurrentArrayQueue[Int](1024) + val source = Source.fromGraph(new SendQueue[Int]) + + val (sendQueue, killSwitch) = source.viaMat(KillSwitches.single)(Keep.both) + .toMat(new BarrierSink(N, latch, burstSize, barrier))(Keep.left).run()(materializer) + sendQueue.inject(queue) + + var n = 1 + while (n <= N) { + if (!sendQueue.offer(n)) + println(s"offer failed $n") // should not happen + if (n % burstSize == 0 && n < N) { + barrier.await() + } + n += 1 + } + + if (!latch.await(30, TimeUnit.SECONDS)) + throw new RuntimeException("Latch didn't complete in time") + killSwitch.shutdown() + } + +} diff --git a/akka-bench-jmh/src/main/scala/akka/remote/artery/compress/CountMinSketchBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/compress/CountMinSketchBenchmark.scala new file mode 100644 index 0000000000..c8be6e628f --- /dev/null +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/compress/CountMinSketchBenchmark.scala @@ -0,0 +1,50 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery.compress + +import java.util.Random + +import org.openjdk.jmh.annotations._ +import org.openjdk.jmh.infra.Blackhole + +@State(Scope.Benchmark) +@BenchmarkMode(Array(Mode.Throughput)) +@Fork(2) +class CountMinSketchBenchmark { + + // @Param(Array("4", "8", "12", "16")) + @Param(Array("16", "256", "4096", "65536")) + var w: Int = _ + @Param(Array("16", "128", "1024")) + var d: Int = _ + + private val seed: Int = 20160726 + + val rand = new Random(seed) + + val preallocateIds = Array.ofDim[Int](8192) + val preallocateValues = Array.ofDim[Long](8192) + + var countMinSketch: CountMinSketch = _ + + @Setup + def init(): Unit = { + countMinSketch = new CountMinSketch(d, w, seed) + (0 to 8191).foreach { index => + preallocateIds(index) = rand.nextInt() + preallocateValues(index) = Math.abs(rand.nextInt()) + } + } + + @Benchmark + @OperationsPerInvocation(8192) + def updateRandomNumbers(blackhole: Blackhole): Unit = { + var i: Int = 0; + while (i < 8192) { + blackhole.consume(countMinSketch.addObjectAndEstimateCount(preallocateIds(i), preallocateValues(i))) + i += 1 + } + } + +} diff --git a/akka-bench-jmh/src/main/scala/akka/remote/artery/compress/InvertCompressionTableBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/compress/InvertCompressionTableBenchmark.scala new file mode 100644 index 0000000000..561ed83751 --- /dev/null +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/compress/InvertCompressionTableBenchmark.scala @@ -0,0 +1,27 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import java.util.concurrent.ThreadLocalRandom + +import org.openjdk.jmh.annotations._ + +@Fork(1) +@State(Scope.Benchmark) +class InvertCompressionTableBenchmark { + + /* + TODO: Possibly specialise the inversion, it's not in hot path so not doing it for now + a.r.artery.compress.CompressionTableBenchmark.invert_comp_to_decomp_1024 N/A thrpt 20 5828.963 ± 281.631 ops/s + a.r.artery.compress.CompressionTableBenchmark.invert_comp_to_decomp_256 N/A thrpt 20 29040.889 ± 345.425 ops/s + */ + + def randomName = ThreadLocalRandom.current().nextInt(1000).toString + val compTable_256 = CompressionTable(17L, 2, Map(Vector.fill[String](256)(randomName).zipWithIndex: _*)) + val compTable_1024 = CompressionTable(17L, 3, Map(Vector.fill[String](1024)(randomName).zipWithIndex: _*)) + + @Benchmark def invert_comp_to_decomp_256 = compTable_256.invert + @Benchmark def invert_comp_to_decomp_1024 = compTable_1024.invert +} diff --git a/akka-bench-jmh/src/main/scala/akka/remote/compress/HeavyHittersBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/remote/compress/HeavyHittersBenchmark.scala new file mode 100644 index 0000000000..6db11656b7 --- /dev/null +++ b/akka-bench-jmh/src/main/scala/akka/remote/compress/HeavyHittersBenchmark.scala @@ -0,0 +1,110 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.compress + +import java.util.Random + +import akka.remote.artery.compress.TopHeavyHitters +import org.openjdk.jmh.annotations._ +import org.openjdk.jmh.infra.Blackhole + +/** + * On Macbook pro: + * [info] Benchmark (n) Mode Cnt Score Error Units + * [info] HeavyHittersBenchmark.updateHitter 8192 thrpt 40 357 405.512 ± 3329.008 ops/s + * [info] HeavyHittersBenchmark.updateNotHitter 8192 thrpt 40 259 032 711.743 ± 7199514.142 ops/s + * [info] HeavyHittersBenchmark.updateRandomHitter 8192 thrpt 40 2 105 102.088 ± 18214.624 ops/s + * + * === + * on our benchmarking box: + * ubuntu@ip-172-31-43-199:~/akka-ktoso$ lscpu + * Architecture: x86_64 + * CPU op-mode(s): 32-bit, 64-bit + * Byte Order: Little Endian + * CPU(s): 2 + * Thread(s) per core: 2 + * CPU MHz: 2494.068 + * Hypervisor vendor: Xen + * Virtualization type: full + * L1d cache: 32K + * L1i cache: 32K + * L2 cache: 256K + * L3 cache: 25600K + * + * ubuntu@ip-172-31-43-199:~/akka-ktoso$ cpuid | grep nm + * (simple synth) = Intel Core i9-4000 / Xeon E5-1600/E5-2600 v2 (Ivy Bridge-EP C1/M1/S1), 22nm + * + * [info] Benchmark (n) Mode Cnt Score Error Units + * [info] HeavyHittersBenchmark.updateHitter 8192 thrpt 40 309 512.584 ± 153.248 ops/s + * [info] HeavyHittersBenchmark.updateNotHitter 8192 thrpt 40 248 170 545.577 ± 1244986.765 ops/s + * [info] HeavyHittersBenchmark.updateRandomHitter 8192 thrpt 40 1 207 521.674 ± 912.676 ops/s + */ +@State(Scope.Benchmark) +@BenchmarkMode(Array(Mode.Throughput)) +@Fork(2) +class HeavyHittersBenchmark { + + // @Param(Array("512", "8192")) + @Param(Array("8192")) + var n: Int = 0 + + private var topN: TopHeavyHitters[String] = _ + + val rand = new Random(1001021) + + val preallocatedNums: Array[Long] = Array.ofDim(8192) + val preallocatedStrings: Array[String] = Array.ofDim(8192) + + @Setup + def init(): Unit = { + topN = new TopHeavyHitters(n) + var i = 0 + while (i < n) { + topN.update(i.toString, i) + preallocatedNums(i) = rand.nextLong() + preallocatedStrings(i) = i.toString + i += 1 + } + } + + @Benchmark + @OperationsPerInvocation(8192) + def updateNotHitter(blackhole: Blackhole): Unit = { + var i = 0 + while (i < 8192) { + blackhole.consume(topN.update("NOT", 1)) // definitely not a heavy hitter + i += 1 + } + } + + @Benchmark + @OperationsPerInvocation(8192) + def updateExistingHitter(blackhole: Blackhole): Unit = { + var i: Int = 0 + while (i < 8192) { + blackhole.consume(topN.update(preallocatedStrings(i % 16), Long.MaxValue)) // definitely a heavy hitter + i += 1 + } + } + + @Benchmark + def updateNewHitter(blackhole: Blackhole): Unit = { + var i = 0 + while (i < 8192) { + blackhole.consume(topN.update(preallocatedStrings(i), Long.MaxValue)) + i += 1 + } + } + + @Benchmark + @OperationsPerInvocation(8192) + def updateRandomHitter(blackhole: Blackhole): Unit = { + var i = 0 + while (i < 8192) { + blackhole.consume(topN.update(preallocatedStrings(i), preallocatedNums(i))) // maybe a heavy hitter + i += 1 + } + } + +} diff --git a/akka-bench-jmh/src/main/scala/akka/stream/FlowMapBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/stream/FlowMapBenchmark.scala index 44f97b3375..ea00cd9fc9 100644 --- a/akka-bench-jmh/src/main/scala/akka/stream/FlowMapBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/stream/FlowMapBenchmark.scala @@ -18,7 +18,7 @@ import scala.concurrent.Await import scala.concurrent.duration._ @State(Scope.Benchmark) -@OutputTimeUnit(TimeUnit.MILLISECONDS) +@OutputTimeUnit(TimeUnit.SECONDS) @BenchmarkMode(Array(Mode.Throughput)) class FlowMapBenchmark { diff --git a/akka-bench-jmh/src/main/scala/akka/util/LruBoundedCacheBench.scala b/akka-bench-jmh/src/main/scala/akka/util/LruBoundedCacheBench.scala new file mode 100644 index 0000000000..f3325d4996 --- /dev/null +++ b/akka-bench-jmh/src/main/scala/akka/util/LruBoundedCacheBench.scala @@ -0,0 +1,78 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.util + +import java.util +import java.util.concurrent.TimeUnit + +import akka.remote.artery.LruBoundedCache +import org.openjdk.jmh.annotations.{ Param, _ } + +import scala.util.Random + +@State(Scope.Benchmark) +@Measurement(timeUnit = TimeUnit.MICROSECONDS) +class LruBoundedCacheBench { + + var javaHashMap: java.util.HashMap[String, String] = _ + + @Param(Array("1024", "8192")) + var count = 0 + + @Param(Array("128", "256")) + var stringSize = 0 + private var lruCache: LruBoundedCache[String, String] = _ + + @Param(Array("90", "99")) + var loadFactor: Int = _ + + var toAdd: String = _ + var toRemove: String = _ + var toGet: String = _ + + @Setup + def setup(): Unit = { + val loadF: Double = loadFactor / 100.0 + val threshold = (loadF * count).toInt + + val random = Random + javaHashMap = new util.HashMap[String, String](count) + lruCache = new LruBoundedCache[String, String](count, threshold) { + override protected def compute(k: String): String = k + override protected def hash(k: String): Int = k.hashCode + override protected def isCacheable(v: String): Boolean = true + } + + // Loading + for (i <- 1 to threshold) { + val value = random.nextString(stringSize) + if (i == 1) toGet = value + toRemove = value + javaHashMap.put(value, value) + lruCache.get(value) + } + + toAdd = random.nextString(stringSize) + + } + + @Benchmark + def addOne_lruCache(): String = { + lruCache.getOrCompute(toAdd) + } + + @Benchmark + def addOne_hashMap(): String = { + javaHashMap.put(toAdd, toAdd) + javaHashMap.get(toAdd) + } + + @Benchmark + def addOne_hashMap_remove_put_get(): String = { + javaHashMap.remove(toRemove) + javaHashMap.put(toAdd, toAdd) + javaHashMap.get(toAdd) + } + +} diff --git a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/TestUtil.scala b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/TestUtil.scala index 6f6689b3e2..656ac42ffd 100644 --- a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/TestUtil.scala +++ b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/TestUtil.scala @@ -6,11 +6,13 @@ package akka.cluster.metrics import scala.language.postfixOps import java.util.logging.LogManager + import org.slf4j.bridge.SLF4JBridgeHandler import akka.testkit.AkkaSpec import akka.actor.ExtendedActorSystem import akka.actor.Address import java.io.Closeable + import akka.actor.ActorRef import akka.actor.Props import akka.actor.Actor @@ -22,6 +24,7 @@ import akka.actor.ActorLogging import org.scalatest.mock.MockitoSugar import akka.actor.ActorSystem import akka.dispatch.Dispatchers +import akka.remote.RARP /** * Redirect different logging sources to SLF4J. @@ -132,7 +135,7 @@ trait MetricsCollectorFactory { this: AkkaSpec ⇒ */ class MockitoSigarMetricsCollector(system: ActorSystem) extends SigarMetricsCollector( - Address("akka.tcp", system.name), + Address(if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka" else "akka.tcp", system.name), MetricsConfig.defaultDecayFactor, MockitoSigarProvider().createSigarInstance) { } @@ -153,7 +156,7 @@ object MetricsConfig { gossip-interval = 1s } } - akka.actor.provider = "akka.remote.RemoteActorRefProvider" + akka.actor.provider = remote """ /** Test w/o cluster, with collection disabled. */ @@ -163,7 +166,7 @@ object MetricsConfig { enabled = off } } - akka.actor.provider = "akka.remote.RemoteActorRefProvider" + akka.actor.provider = remote """ /** Test in cluster, with manual collection activation, collector mock, fast. */ @@ -178,7 +181,7 @@ object MetricsConfig { fallback = false } } - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" """ } diff --git a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/protobuf/MessageSerializerSpec.scala b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/protobuf/MessageSerializerSpec.scala index ffa044a193..5ae834d543 100644 --- a/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/protobuf/MessageSerializerSpec.scala +++ b/akka-cluster-metrics/src/test/scala/akka/cluster/metrics/protobuf/MessageSerializerSpec.scala @@ -15,7 +15,7 @@ import akka.cluster.TestMember import akka.cluster.metrics.MetricsGossipEnvelope class MessageSerializerSpec extends AkkaSpec( - "akka.actor.provider = akka.cluster.ClusterActorRefProvider") { + "akka.actor.provider = cluster") { val serializer = new MessageSerializer(system.asInstanceOf[ExtendedActorSystem]) diff --git a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/protobuf/ClusterShardingMessageSerializer.scala b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/protobuf/ClusterShardingMessageSerializer.scala index 48d297de0a..3e629ee2d9 100644 --- a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/protobuf/ClusterShardingMessageSerializer.scala +++ b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/protobuf/ClusterShardingMessageSerializer.scala @@ -31,8 +31,6 @@ private[akka] class ClusterShardingMessageSerializer(val system: ExtendedActorSy import Shard.{ GetShardStats, ShardStats } import Shard.{ State ⇒ EntityState, EntityStarted, EntityStopped } - private lazy val serialization = SerializationExtension(system) - private final val BufferSize = 1024 * 4 private val CoordinatorStateManifest = "AA" diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingCustomShardAllocationSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingCustomShardAllocationSpec.scala index ba1c05e08d..864c2ad949 100644 --- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingCustomShardAllocationSpec.scala +++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingCustomShardAllocationSpec.scala @@ -85,7 +85,7 @@ abstract class ClusterShardingCustomShardAllocationSpecConfig(val mode: String) commonConfig(ConfigFactory.parseString(s""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.log-remote-lifecycle-events = off akka.persistence.journal.plugin = "akka.persistence.journal.leveldb-shared" akka.persistence.journal.leveldb-shared { diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingFailureSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingFailureSpec.scala index 887b82d7bc..2086ee0e75 100644 --- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingFailureSpec.scala +++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingFailureSpec.scala @@ -55,7 +55,7 @@ abstract class ClusterShardingFailureSpecConfig(val mode: String) extends MultiN commonConfig(ConfigFactory.parseString(s""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.log-remote-lifecycle-events = off akka.cluster.auto-down-unreachable-after = 0s akka.cluster.roles = ["backend"] diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGetStateSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGetStateSpec.scala index 2cea3817f1..baf9f7c3ea 100644 --- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGetStateSpec.scala +++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGetStateSpec.scala @@ -46,7 +46,7 @@ object ClusterShardingGetStateSpecConfig extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.log-remote-lifecycle-events = off akka.cluster.metrics.enabled = off akka.cluster.auto-down-unreachable-after = 0s diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGetStatsSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGetStatsSpec.scala index 149577b3e0..2d55b5c555 100644 --- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGetStatsSpec.scala +++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGetStatsSpec.scala @@ -47,7 +47,7 @@ object ClusterShardingGetStatsSpecConfig extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.log-remote-lifecycle-events = off akka.log-dead-letters-during-shutdown = off akka.cluster.metrics.enabled = off diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGracefulShutdownSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGracefulShutdownSpec.scala index 00c3ff0400..2433a53a3d 100644 --- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGracefulShutdownSpec.scala +++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingGracefulShutdownSpec.scala @@ -72,7 +72,7 @@ abstract class ClusterShardingGracefulShutdownSpecConfig(val mode: String) exten commonConfig(ConfigFactory.parseString(s""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.log-remote-lifecycle-events = off akka.persistence.journal.plugin = "akka.persistence.journal.leveldb-shared" akka.persistence.journal.leveldb-shared { diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingLeavingSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingLeavingSpec.scala index fb88783782..56beb82211 100644 --- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingLeavingSpec.scala +++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingLeavingSpec.scala @@ -63,7 +63,7 @@ abstract class ClusterShardingLeavingSpecConfig(val mode: String) extends MultiN commonConfig(ConfigFactory.parseString(s""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.log-remote-lifecycle-events = off akka.cluster.auto-down-unreachable-after = 0s akka.persistence.journal.plugin = "akka.persistence.journal.leveldb-shared" diff --git a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingSpec.scala b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingSpec.scala index 93aa967374..36a251628d 100644 --- a/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingSpec.scala +++ b/akka-cluster-sharding/src/multi-jvm/scala/akka/cluster/sharding/ClusterShardingSpec.scala @@ -129,7 +129,7 @@ abstract class ClusterShardingSpecConfig( commonConfig(ConfigFactory.parseString(s""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.log-remote-lifecycle-events = off akka.cluster.auto-down-unreachable-after = 0s akka.cluster.roles = ["backend"] diff --git a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/RemoveInternalClusterShardingDataSpec.scala b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/RemoveInternalClusterShardingDataSpec.scala index 817c49a4c1..b772d60d01 100644 --- a/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/RemoveInternalClusterShardingDataSpec.scala +++ b/akka-cluster-sharding/src/test/scala/akka/cluster/sharding/RemoveInternalClusterShardingDataSpec.scala @@ -27,7 +27,7 @@ import org.apache.commons.io.FileUtils object RemoveInternalClusterShardingDataSpec { val config = """ akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.netty.tcp.port = 0 akka.persistence.journal.plugin = "akka.persistence.journal.leveldb" akka.persistence.journal.leveldb { diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientSpec.scala index f55cf493ee..c0d04d110a 100644 --- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientSpec.scala +++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientSpec.scala @@ -16,6 +16,7 @@ import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.STMultiNodeSpec import akka.testkit._ import akka.cluster.pubsub._ +import akka.remote.RARP import akka.remote.transport.ThrottlerTransportAdapter.Direction import akka.util.Timeout @@ -30,7 +31,7 @@ object ClusterClientSpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.log-remote-lifecycle-events = off akka.cluster.auto-down-unreachable-after = 0s akka.cluster.client.heartbeat-interval = 1s @@ -430,10 +431,13 @@ class ClusterClientSpec extends MultiNodeSpec(ClusterClientSpec) with STMultiNod runOn(remainingServerRoleNames.toSeq: _*) { Await.ready(system.whenTerminated, 20.seconds) // start new system on same port + val port = Cluster(system).selfAddress.port.get val sys2 = ActorSystem( system.name, - ConfigFactory.parseString("akka.remote.netty.tcp.port=" + Cluster(system).selfAddress.port.get) - .withFallback(system.settings.config)) + ConfigFactory.parseString( + if (RARP(system).provider.remoteSettings.Artery.Enabled) s"akka.remote.artery.canonical.port=$port" + else s"akka.remote.netty.tcp.port=$port" + ).withFallback(system.settings.config)) Cluster(sys2).join(Cluster(sys2).selfAddress) val service2 = sys2.actorOf(Props(classOf[TestService], testActor), "service2") ClusterClientReceptionist(sys2).registerService(service2) diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientStopSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientStopSpec.scala index f5112076b2..95949577a1 100644 --- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientStopSpec.scala +++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/client/ClusterClientStopSpec.scala @@ -19,7 +19,7 @@ object ClusterClientStopSpec extends MultiNodeConfig { val second = role("second") commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.log-remote-lifecycle-events = off akka.cluster.client { heartbeat-interval = 1s diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubMediatorSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubMediatorSpec.scala index e9e10d9fdb..05659b1abf 100644 --- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubMediatorSpec.scala +++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubMediatorSpec.scala @@ -28,7 +28,7 @@ object DistributedPubSubMediatorSpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.log-remote-lifecycle-events = off akka.cluster.auto-down-unreachable-after = 0s akka.cluster.pub-sub.max-delta-elements = 500 diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubRestartSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubRestartSpec.scala index 67b12594b9..d7450cfd2c 100644 --- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubRestartSpec.scala +++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/pubsub/DistributedPubSubRestartSpec.scala @@ -21,10 +21,12 @@ import akka.actor.ActorLogging import akka.cluster.pubsub.DistributedPubSubMediator.Internal.Status import akka.cluster.pubsub.DistributedPubSubMediator.Internal.Delta import akka.actor.ActorSystem + import scala.concurrent.Await import akka.actor.Identify import akka.actor.RootActorPath import akka.actor.ActorIdentity +import akka.remote.RARP object DistributedPubSubRestartSpec extends MultiNodeConfig { val first = role("first") @@ -136,10 +138,16 @@ class DistributedPubSubRestartSpec extends MultiNodeSpec(DistributedPubSubRestar runOn(third) { Await.result(system.whenTerminated, 10.seconds) - val newSystem = ActorSystem( - system.name, - ConfigFactory.parseString(s"akka.remote.netty.tcp.port=${Cluster(system).selfAddress.port.get}").withFallback( - system.settings.config)) + val newSystem = { + val port = Cluster(system).selfAddress.port.get + val config = ConfigFactory.parseString( + if (RARP(system).provider.remoteSettings.Artery.Enabled) s"akka.remote.artery.canonical.port=$port" + else s"akka.remote.netty.tcp.port=$port" + ).withFallback(system.settings.config) + + ActorSystem(system.name, config) + } + try { // don't join the old cluster Cluster(newSystem).join(Cluster(newSystem).selfAddress) diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerChaosSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerChaosSpec.scala index c3fd74b90a..122348fb63 100644 --- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerChaosSpec.scala +++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerChaosSpec.scala @@ -38,7 +38,7 @@ object ClusterSingletonManagerChaosSpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.log-remote-lifecycle-events = off akka.cluster.auto-down-unreachable-after = 0s """)) diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerLeaveSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerLeaveSpec.scala index 71652eee0f..b66fe09871 100644 --- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerLeaveSpec.scala +++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerLeaveSpec.scala @@ -35,7 +35,7 @@ object ClusterSingletonManagerLeaveSpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.log-remote-lifecycle-events = off akka.cluster.auto-down-unreachable-after = off """)) diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerSpec.scala index fe5ceab877..2901929d3a 100644 --- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerSpec.scala +++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerSpec.scala @@ -41,7 +41,7 @@ object ClusterSingletonManagerSpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.log-remote-lifecycle-events = off akka.cluster.auto-down-unreachable-after = 0s """)) diff --git a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerStartupSpec.scala b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerStartupSpec.scala index 0b0b42b7a8..3e6ab7f6df 100644 --- a/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerStartupSpec.scala +++ b/akka-cluster-tools/src/multi-jvm/scala/akka/cluster/singleton/ClusterSingletonManagerStartupSpec.scala @@ -35,7 +35,7 @@ object ClusterSingletonManagerStartupSpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.log-remote-lifecycle-events = off akka.cluster.auto-down-unreachable-after = 0s """)) diff --git a/akka-cluster-tools/src/test/java/akka/cluster/client/ClusterClientTest.java b/akka-cluster-tools/src/test/java/akka/cluster/client/ClusterClientTest.java index 71117b1629..88d4614ac1 100644 --- a/akka-cluster-tools/src/test/java/akka/cluster/client/ClusterClientTest.java +++ b/akka-cluster-tools/src/test/java/akka/cluster/client/ClusterClientTest.java @@ -21,7 +21,7 @@ public class ClusterClientTest extends JUnitSuite { public static AkkaJUnitActorSystemResource actorSystemResource = new AkkaJUnitActorSystemResource("DistributedPubSubMediatorTest", ConfigFactory.parseString( - "akka.actor.provider = \"akka.cluster.ClusterActorRefProvider\"\n" + + "akka.actor.provider = \"cluster\"\n" + "akka.remote.netty.tcp.port=0")); private final ActorSystem system = actorSystemResource.getSystem(); diff --git a/akka-cluster-tools/src/test/java/akka/cluster/pubsub/DistributedPubSubMediatorTest.java b/akka-cluster-tools/src/test/java/akka/cluster/pubsub/DistributedPubSubMediatorTest.java index c1c9311bd6..91fb51d33e 100644 --- a/akka-cluster-tools/src/test/java/akka/cluster/pubsub/DistributedPubSubMediatorTest.java +++ b/akka-cluster-tools/src/test/java/akka/cluster/pubsub/DistributedPubSubMediatorTest.java @@ -25,7 +25,7 @@ public class DistributedPubSubMediatorTest extends JUnitSuite { public static AkkaJUnitActorSystemResource actorSystemResource = new AkkaJUnitActorSystemResource("DistributedPubSubMediatorTest", ConfigFactory.parseString( - "akka.actor.provider = \"akka.cluster.ClusterActorRefProvider\"\n" + + "akka.actor.provider = \"cluster\"\n" + "akka.remote.netty.tcp.port=0")); private final ActorSystem system = actorSystemResource.getSystem(); diff --git a/akka-cluster-tools/src/test/scala/akka/cluster/pubsub/DistributedPubSubMediatorRouterSpec.scala b/akka-cluster-tools/src/test/scala/akka/cluster/pubsub/DistributedPubSubMediatorRouterSpec.scala index f1c05a2752..52bf506cfb 100644 --- a/akka-cluster-tools/src/test/scala/akka/cluster/pubsub/DistributedPubSubMediatorRouterSpec.scala +++ b/akka-cluster-tools/src/test/scala/akka/cluster/pubsub/DistributedPubSubMediatorRouterSpec.scala @@ -15,7 +15,7 @@ case class UnwrappedMessage(msg: String) object DistributedPubSubMediatorRouterSpec { def config(routingLogic: String) = s""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.netty.tcp.port=0 akka.remote.log-remote-lifecycle-events = off akka.cluster.pub-sub.routing-logic = $routingLogic diff --git a/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonProxySpec.scala b/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonProxySpec.scala index 005f7b608d..f803615094 100644 --- a/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonProxySpec.scala +++ b/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonProxySpec.scala @@ -58,27 +58,27 @@ object ClusterSingletonProxySpec { } } - val cfg = """akka { - - loglevel = INFO - - cluster { - auto-down-unreachable-after = 10s - - min-nr-of-members = 2 - } - - actor.provider = "akka.cluster.ClusterActorRefProvider" - - remote { - log-remote-lifecycle-events = off - netty.tcp { - hostname = "127.0.0.1" - port = 0 - } - } - } - """ + val cfg = """ + akka { + loglevel = INFO + cluster { + auto-down-unreachable-after = 10s + min-nr-of-members = 2 + } + actor.provider = "cluster" + remote { + log-remote-lifecycle-events = off + netty.tcp { + hostname = "127.0.0.1" + port = 0 + } + artery.canonical { + hostname = "127.0.0.1" + port = 0 + } + } + } + """ class Singleton extends Actor with ActorLogging { diff --git a/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonRestartSpec.scala b/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonRestartSpec.scala index e8c2f6d8b1..2b5b3a13c2 100644 --- a/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonRestartSpec.scala +++ b/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonRestartSpec.scala @@ -4,11 +4,11 @@ package akka.cluster.singleton import scala.concurrent.duration._ - import akka.actor.ActorSystem import akka.actor.PoisonPill import akka.cluster.Cluster import akka.cluster.MemberStatus +import akka.remote.RARP import akka.testkit.AkkaSpec import akka.testkit.TestActors import akka.testkit.TestProbe @@ -22,6 +22,10 @@ class ClusterSingletonRestartSpec extends AkkaSpec(""" hostname = "127.0.0.1" port = 0 } + artery.canonical { + hostname = "127.0.0.1" + port = 0 + } } """) { @@ -64,10 +68,17 @@ class ClusterSingletonRestartSpec extends AkkaSpec(""" shutdown(sys1) // it will be downed by the join attempts of the new incarnation - sys3 = ActorSystem( - system.name, - ConfigFactory.parseString(s"akka.remote.netty.tcp.port=${Cluster(sys1).selfAddress.port.get}").withFallback( - system.settings.config)) + sys3 = { + val sys1port = Cluster(sys1).selfAddress.port.get + + val sys3Config = + ConfigFactory.parseString( + if (RARP(sys1).provider.remoteSettings.Artery.Enabled) s"akka.remote.artery.canonical.port=$sys1port" + else s"akka.remote.netty.tcp.port=$sys1port" + ).withFallback(system.settings.config) + + ActorSystem(system.name, sys3Config) + } join(sys3, sys2) within(5.seconds) { diff --git a/akka-cluster/src/main/java/akka/cluster/protobuf/msg/ClusterMessages.java b/akka-cluster/src/main/java/akka/cluster/protobuf/msg/ClusterMessages.java index 9793fa2616..15db086207 100644 --- a/akka-cluster/src/main/java/akka/cluster/protobuf/msg/ClusterMessages.java +++ b/akka-cluster/src/main/java/akka/cluster/protobuf/msg/ClusterMessages.java @@ -16041,6 +16041,24 @@ public final class ClusterMessages { * required uint32 uid = 2; */ int getUid(); + + // optional uint32 uid2 = 3; + /** + * optional uint32 uid2 = 3; + * + *
+     * 64 bit uids but with backward wire compatibility
+     * 
+ */ + boolean hasUid2(); + /** + * optional uint32 uid2 = 3; + * + *
+     * 64 bit uids but with backward wire compatibility
+     * 
+ */ + int getUid2(); } /** * Protobuf type {@code UniqueAddress} @@ -16116,6 +16134,11 @@ public final class ClusterMessages { uid_ = input.readUInt32(); break; } + case 24: { + bitField0_ |= 0x00000004; + uid2_ = input.readUInt32(); + break; + } } } } catch (akka.protobuf.InvalidProtocolBufferException e) { @@ -16194,9 +16217,34 @@ public final class ClusterMessages { return uid_; } + // optional uint32 uid2 = 3; + public static final int UID2_FIELD_NUMBER = 3; + private int uid2_; + /** + * optional uint32 uid2 = 3; + * + *
+     * 64 bit uids but with backward wire compatibility
+     * 
+ */ + public boolean hasUid2() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint32 uid2 = 3; + * + *
+     * 64 bit uids but with backward wire compatibility
+     * 
+ */ + public int getUid2() { + return uid2_; + } + private void initFields() { address_ = akka.cluster.protobuf.msg.ClusterMessages.Address.getDefaultInstance(); uid_ = 0; + uid2_ = 0; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -16228,6 +16276,9 @@ public final class ClusterMessages { if (((bitField0_ & 0x00000002) == 0x00000002)) { output.writeUInt32(2, uid_); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt32(3, uid2_); + } getUnknownFields().writeTo(output); } @@ -16245,6 +16296,10 @@ public final class ClusterMessages { size += akka.protobuf.CodedOutputStream .computeUInt32Size(2, uid_); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += akka.protobuf.CodedOutputStream + .computeUInt32Size(3, uid2_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -16375,6 +16430,8 @@ public final class ClusterMessages { bitField0_ = (bitField0_ & ~0x00000001); uid_ = 0; bitField0_ = (bitField0_ & ~0x00000002); + uid2_ = 0; + bitField0_ = (bitField0_ & ~0x00000004); return this; } @@ -16415,6 +16472,10 @@ public final class ClusterMessages { to_bitField0_ |= 0x00000002; } result.uid_ = uid_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.uid2_ = uid2_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -16437,6 +16498,9 @@ public final class ClusterMessages { if (other.hasUid()) { setUid(other.getUid()); } + if (other.hasUid2()) { + setUid2(other.getUid2()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -16626,6 +16690,55 @@ public final class ClusterMessages { return this; } + // optional uint32 uid2 = 3; + private int uid2_ ; + /** + * optional uint32 uid2 = 3; + * + *
+       * 64 bit uids but with backward wire compatibility
+       * 
+ */ + public boolean hasUid2() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional uint32 uid2 = 3; + * + *
+       * 64 bit uids but with backward wire compatibility
+       * 
+ */ + public int getUid2() { + return uid2_; + } + /** + * optional uint32 uid2 = 3; + * + *
+       * 64 bit uids but with backward wire compatibility
+       * 
+ */ + public Builder setUid2(int value) { + bitField0_ |= 0x00000004; + uid2_ = value; + onChanged(); + return this; + } + /** + * optional uint32 uid2 = 3; + * + *
+       * 64 bit uids but with backward wire compatibility
+       * 
+ */ + public Builder clearUid2() { + bitField0_ = (bitField0_ & ~0x00000004); + uid2_ = 0; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:UniqueAddress) } @@ -16789,14 +16902,14 @@ public final class ClusterMessages { "\016\n\nSerialized\020\000\022\n\n\006Double\020\001\022\t\n\005Float\020\002\022\013" + "\n\007Integer\020\003\022\010\n\004Long\020\004\"\007\n\005Empty\"K\n\007Addres" + "s\022\016\n\006system\030\001 \002(\t\022\020\n\010hostname\030\002 \002(\t\022\014\n\004p" + - "ort\030\003 \002(\r\022\020\n\010protocol\030\004 \001(\t\"7\n\rUniqueAdd" + + "ort\030\003 \002(\r\022\020\n\010protocol\030\004 \001(\t\"E\n\rUniqueAdd" + "ress\022\031\n\007address\030\001 \002(\0132\010.Address\022\013\n\003uid\030\002" + - " \002(\r*D\n\022ReachabilityStatus\022\r\n\tReachable\020" + - "\000\022\017\n\013Unreachable\020\001\022\016\n\nTerminated\020\002*b\n\014Me" + - "mberStatus\022\013\n\007Joining\020\000\022\006\n\002Up\020\001\022\013\n\007Leavi" + - "ng\020\002\022\013\n\007Exiting\020\003\022\010\n\004Down\020\004\022\013\n\007Removed\020\005" + - "\022\014\n\010WeaklyUp\020\006B\035\n\031akka.cluster.protobuf.", - "msgH\001" + " \002(\r\022\014\n\004uid2\030\003 \001(\r*D\n\022ReachabilityStatus" + + "\022\r\n\tReachable\020\000\022\017\n\013Unreachable\020\001\022\016\n\nTerm" + + "inated\020\002*b\n\014MemberStatus\022\013\n\007Joining\020\000\022\006\n" + + "\002Up\020\001\022\013\n\007Leaving\020\002\022\013\n\007Exiting\020\003\022\010\n\004Down\020" + + "\004\022\013\n\007Removed\020\005\022\014\n\010WeaklyUp\020\006B\035\n\031akka.clu", + "ster.protobuf.msgH\001" }; akka.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new akka.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -16922,7 +17035,7 @@ public final class ClusterMessages { internal_static_UniqueAddress_fieldAccessorTable = new akka.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_UniqueAddress_descriptor, - new java.lang.String[] { "Address", "Uid", }); + new java.lang.String[] { "Address", "Uid", "Uid2", }); return null; } }; diff --git a/akka-cluster/src/main/protobuf/ClusterMessages.proto b/akka-cluster/src/main/protobuf/ClusterMessages.proto index 8b280d421f..988899c73d 100644 --- a/akka-cluster/src/main/protobuf/ClusterMessages.proto +++ b/akka-cluster/src/main/protobuf/ClusterMessages.proto @@ -252,4 +252,6 @@ message Address { message UniqueAddress { required Address address = 1; required uint32 uid = 2; + // 64 bit uids but with backward wire compatibility + optional uint32 uid2 = 3; } diff --git a/akka-cluster/src/main/resources/reference.conf b/akka-cluster/src/main/resources/reference.conf index 978438b4be..a24dd0faa9 100644 --- a/akka-cluster/src/main/resources/reference.conf +++ b/akka-cluster/src/main/resources/reference.conf @@ -54,6 +54,11 @@ akka { # `akka.cluster.DowningProvider` having a public one argument constructor accepting an `ActorSystem` downing-provider-class = "" + # Artery only setting + # When a node has been gracefully removed, let this time pass (to allow for example + # cluster singleton handover to complete) and then quarantine the removed node. + quarantine-removed-node-after=30s + # By default, the leader will not move 'Joining' members to 'Up' during a network # split. This feature allows the leader to accept 'Joining' members to be 'WeaklyUp' # so they become part of the cluster even during a network split. The leader will diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index d8ec2cd930..582f9b82a9 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -67,7 +67,7 @@ class Cluster(val system: ExtendedActorSystem) extends Extension { */ val selfUniqueAddress: UniqueAddress = system.provider match { case c: ClusterActorRefProvider ⇒ - UniqueAddress(c.transport.defaultAddress, AddressUidExtension(system).addressUid) + UniqueAddress(c.transport.defaultAddress, AddressUidExtension(system).longAddressUid) case other ⇒ throw new ConfigurationException( s"ActorSystem [${system}] needs to have a 'ClusterActorRefProvider' enabled in the configuration, currently uses [${other.getClass.getName}]") } diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala index a65ca8cb24..afb87d0253 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala @@ -250,7 +250,7 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with val NumberOfGossipsBeforeShutdownWhenLeaderExits = 3 val MaxGossipsBeforeShuttingDownMyself = 5 - def vclockName(node: UniqueAddress): String = node.address + "-" + node.uid + def vclockName(node: UniqueAddress): String = s"${node.address}-${node.longUid}" val vclockNode = VectorClock.Node(vclockName(selfUniqueAddress)) // note that self is not initially member, diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterHeartbeat.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterHeartbeat.scala index 6279e44928..160e5615f9 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterHeartbeat.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterHeartbeat.scala @@ -8,7 +8,7 @@ import scala.collection.immutable import akka.actor.{ ActorLogging, ActorSelection, Address, Actor, RootActorPath } import akka.cluster.ClusterEvent._ import akka.remote.FailureDetectorRegistry -import akka.remote.PriorityMessage +import akka.remote.HeartbeatMessage import akka.actor.DeadLetterSuppression /** @@ -36,12 +36,12 @@ private[cluster] object ClusterHeartbeatSender { /** * Sent at regular intervals for failure detection. */ - final case class Heartbeat(from: Address) extends ClusterMessage with PriorityMessage with DeadLetterSuppression + final case class Heartbeat(from: Address) extends ClusterMessage with HeartbeatMessage with DeadLetterSuppression /** * Sent as reply to [[Heartbeat]] messages. */ - final case class HeartbeatRsp(from: UniqueAddress) extends ClusterMessage with PriorityMessage with DeadLetterSuppression + final case class HeartbeatRsp(from: UniqueAddress) extends ClusterMessage with HeartbeatMessage with DeadLetterSuppression // sent to self only case object HeartbeatTick diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala index 8fb729930a..af6d3cba2f 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala @@ -12,6 +12,7 @@ import akka.cluster.ClusterEvent.MemberRemoved import akka.cluster.ClusterEvent.MemberWeaklyUp import akka.remote.FailureDetectorRegistry import akka.remote.RemoteWatcher +import akka.remote.RARP /** * INTERNAL API @@ -51,9 +52,12 @@ private[cluster] class ClusterRemoteWatcher( unreachableReaperInterval, heartbeatExpectedResponseAfter) { + private val arteryEnabled = RARP(context.system).provider.remoteSettings.Artery.Enabled val cluster = Cluster(context.system) import cluster.selfAddress + private final case class DelayedQuarantine(m: Member, previousStatus: MemberStatus) extends NoSerializationVerificationNeeded + var clusterNodes: Set[Address] = Set.empty override def preStart(): Unit = { @@ -73,10 +77,11 @@ private[cluster] class ClusterRemoteWatcher( clusterNodes = state.members.collect { case m if m.address != selfAddress ⇒ m.address } clusterNodes foreach takeOverResponsibility unreachable = unreachable diff clusterNodes - case MemberUp(m) ⇒ memberUp(m) - case MemberWeaklyUp(m) ⇒ memberUp(m) - case MemberRemoved(m, previousStatus) ⇒ memberRemoved(m, previousStatus) - case _: MemberEvent ⇒ // not interesting + case MemberUp(m) ⇒ memberUp(m) + case MemberWeaklyUp(m) ⇒ memberUp(m) + case MemberRemoved(m, previousStatus) ⇒ memberRemoved(m, previousStatus) + case _: MemberEvent ⇒ // not interesting + case DelayedQuarantine(m, previousStatus) ⇒ delayedQuarantine(m, previousStatus) } def memberUp(m: Member): Unit = @@ -89,12 +94,22 @@ private[cluster] class ClusterRemoteWatcher( def memberRemoved(m: Member, previousStatus: MemberStatus): Unit = if (m.address != selfAddress) { clusterNodes -= m.address + if (previousStatus == MemberStatus.Down) { - quarantine(m.address, Some(m.uniqueAddress.uid)) + quarantine(m.address, Some(m.uniqueAddress.uid), s"Cluster member removed, previous status [$previousStatus]") + } else if (arteryEnabled) { + // don't quarantine gracefully removed members (leaving) directly, + // give Cluster Singleton some time to exchange TakeOver/HandOver messages. + import context.dispatcher + context.system.scheduler.scheduleOnce(cluster.settings.QuarantineRemovedNodeAfter, self, DelayedQuarantine(m, previousStatus)) } + publishAddressTerminated(m.address) } + def delayedQuarantine(m: Member, previousStatus: MemberStatus): Unit = + quarantine(m.address, Some(m.uniqueAddress.longUid), s"Cluster member removed, previous status [$previousStatus]") + override def watchNode(watchee: InternalActorRef) = if (!clusterNodes(watchee.path.address)) super.watchNode(watchee) diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala index 682a7cb849..462a8ca01a 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala @@ -88,6 +88,9 @@ final class ClusterSettings(val config: Config, val systemName: String) { else classOf[NoDowning].getName } + val QuarantineRemovedNodeAfter: FiniteDuration = + cc.getMillisDuration("quarantine-removed-node-after") requiring (_ > Duration.Zero, "quarantine-removed-node-after must be > 0") + val AllowWeaklyUpMembers = cc.getBoolean("allow-weakly-up-members") val Roles: Set[String] = immutableSeq(cc.getStringList("roles")).toSet diff --git a/akka-cluster/src/main/scala/akka/cluster/Member.scala b/akka-cluster/src/main/scala/akka/cluster/Member.scala index bd3f817587..37671623d7 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Member.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Member.scala @@ -7,6 +7,8 @@ package akka.cluster import akka.actor.Address import MemberStatus._ +import scala.runtime.AbstractFunction2 + /** * Represents the address, current status, and roles of a cluster member node. * @@ -243,18 +245,43 @@ object MemberStatus { Removed → Set.empty[MemberStatus]) } +object UniqueAddress extends AbstractFunction2[Address, Int, UniqueAddress] { + + // for binary compatibility + @deprecated("Use Long UID apply instead", since = "2.4.11") + def apply(address: Address, uid: Int) = new UniqueAddress(address, uid.toLong) + +} + /** * Member identifier consisting of address and random `uid`. * The `uid` is needed to be able to distinguish different * incarnations of a member with same hostname and port. */ @SerialVersionUID(1L) -final case class UniqueAddress(address: Address, uid: Int) extends Ordered[UniqueAddress] { - override def hashCode = uid +final case class UniqueAddress(address: Address, longUid: Long) extends Ordered[UniqueAddress] { + + override def hashCode = java.lang.Long.hashCode(longUid) def compare(that: UniqueAddress): Int = { val result = Member.addressOrdering.compare(this.address, that.address) - if (result == 0) if (this.uid < that.uid) -1 else if (this.uid == that.uid) 0 else 1 + if (result == 0) if (this.longUid < that.longUid) -1 else if (this.longUid == that.longUid) 0 else 1 else result } -} + + // for binary compatibility + + @deprecated("Use Long UID constructor instead", since = "2.4.11") + def this(address: Address, uid: Int) = this(address, uid.toLong) + + @deprecated("Use longUid instead", since = "2.4.11") + def uid = longUid.toInt + + /** + * For binary compatibility + * Stops `copy(Address, Long)` copy from being generated, use `apply` instead. + */ + @deprecated("Use Long UID constructor instead", since = "2.4.11") + def copy(address: Address = address, uid: Int = uid) = new UniqueAddress(address, uid) + +} \ No newline at end of file diff --git a/akka-cluster/src/main/scala/akka/cluster/protobuf/ClusterMessageSerializer.scala b/akka-cluster/src/main/scala/akka/cluster/protobuf/ClusterMessageSerializer.scala index 78921cc0f3..970c8e1a6e 100644 --- a/akka-cluster/src/main/scala/akka/cluster/protobuf/ClusterMessageSerializer.scala +++ b/akka-cluster/src/main/scala/akka/cluster/protobuf/ClusterMessageSerializer.scala @@ -126,8 +126,12 @@ class ClusterMessageSerializer(val system: ExtendedActorSystem) extends BaseSeri private def addressToProtoByteArray(address: Address): Array[Byte] = addressToProto(address).build.toByteArray - private def uniqueAddressToProto(uniqueAddress: UniqueAddress): cm.UniqueAddress.Builder = - cm.UniqueAddress.newBuilder().setAddress(addressToProto(uniqueAddress.address)).setUid(uniqueAddress.uid) + private def uniqueAddressToProto(uniqueAddress: UniqueAddress): cm.UniqueAddress.Builder = { + cm.UniqueAddress.newBuilder() + .setAddress(addressToProto(uniqueAddress.address)) + .setUid(uniqueAddress.longUid.toInt) + .setUid2((uniqueAddress.longUid >> 32).toInt) + } private def uniqueAddressToProtoByteArray(uniqueAddress: UniqueAddress): Array[Byte] = uniqueAddressToProto(uniqueAddress).build.toByteArray @@ -161,8 +165,19 @@ class ClusterMessageSerializer(val system: ExtendedActorSystem) extends BaseSeri private def addressFromProto(address: cm.Address): Address = Address(getProtocol(address), getSystem(address), address.getHostname, address.getPort) - private def uniqueAddressFromProto(uniqueAddress: cm.UniqueAddress): UniqueAddress = - UniqueAddress(addressFromProto(uniqueAddress.getAddress), uniqueAddress.getUid) + private def uniqueAddressFromProto(uniqueAddress: cm.UniqueAddress): UniqueAddress = { + + UniqueAddress( + addressFromProto(uniqueAddress.getAddress), + if (uniqueAddress.hasUid2) { + // new remote node join the two parts of the long uid back + (uniqueAddress.getUid2.toLong << 32) | (uniqueAddress.getUid & 0xFFFFFFFFL) + } else { + // old remote node + uniqueAddress.getUid.toLong + } + ) + } private val memberStatusToInt = scala.collection.immutable.HashMap[MemberStatus, Int]( MemberStatus.Joining → cm.MemberStatus.Joining_VALUE, diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterDeathWatchSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterDeathWatchSpec.scala index 0251409999..629cc8829f 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterDeathWatchSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterDeathWatchSpec.scala @@ -59,6 +59,7 @@ abstract class ClusterDeathWatchSpec } "An actor watching a remote actor in the cluster" must { + "receive Terminated when watched node becomes Down/Removed" in within(20 seconds) { awaitClusterUp(first, second, third, fourth) enterBarrier("cluster-up") diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala index 064ec57774..6d4eeb9927 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala @@ -5,20 +5,24 @@ package akka.cluster // TODO remove metrics +import java.util.UUID + import language.implicitConversions -import org.scalatest.{ Suite, Outcome, Canceled } +import org.scalatest.{ Canceled, Outcome, Suite } import org.scalatest.exceptions.TestCanceledException import com.typesafe.config.Config import com.typesafe.config.ConfigFactory import akka.remote.testconductor.RoleName -import akka.remote.testkit.{ STMultiNodeSpec, MultiNodeSpec } +import akka.remote.testkit.{ FlightRecordingSupport, MultiNodeSpec, STMultiNodeSpec } import akka.testkit._ import akka.testkit.TestEvent._ import akka.actor.{ ActorSystem, Address } import akka.event.Logging.ErrorLevel + import scala.concurrent.duration._ import scala.collection.immutable import java.util.concurrent.ConcurrentHashMap + import akka.remote.DefaultFailureDetectorRegistry import akka.actor.ActorRef import akka.actor.Actor @@ -33,8 +37,8 @@ object MultiNodeClusterSpec { def clusterConfig(failureDetectorPuppet: Boolean): Config = if (failureDetectorPuppet) clusterConfigWithFailureDetectorPuppet else clusterConfig - def clusterConfig: Config = ConfigFactory.parseString(""" - akka.actor.provider = akka.cluster.ClusterActorRefProvider + def clusterConfig: Config = ConfigFactory.parseString(s""" + akka.actor.provider = cluster akka.cluster { jmx.enabled = off gossip-interval = 200 ms @@ -47,11 +51,18 @@ object MultiNodeClusterSpec { akka.loglevel = INFO akka.log-dead-letters = off akka.log-dead-letters-during-shutdown = off - akka.remote.log-remote-lifecycle-events = off + akka.remote { + log-remote-lifecycle-events = off + artery.advanced.flight-recorder { + enabled=on + destination=target/flight-recorder-${UUID.randomUUID().toString}.afr + } + } akka.loggers = ["akka.testkit.TestEventListener"] akka.test { single-expect-default = 5 s } + """) // sometimes we need to coordinate test shutdown with messages instead of barriers @@ -77,19 +88,25 @@ object MultiNodeClusterSpec { } } -trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec with WatchedByCoroner { self: MultiNodeSpec ⇒ +trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec with WatchedByCoroner with FlightRecordingSupport { self: MultiNodeSpec ⇒ override def initialParticipants = roles.size private val cachedAddresses = new ConcurrentHashMap[RoleName, Address] - override def atStartup(): Unit = { + override protected def atStartup(): Unit = { startCoroner() muteLog() + self.atStartup() } - override def afterTermination(): Unit = { + override protected def afterTermination(): Unit = { + self.afterTermination() stopCoroner() + if (failed || sys.props.get("akka.remote.artery.always-dump-flight-recorder").isDefined) { + printFlightRecording() + } + deleteFlightRecorderFile() } override def expectedTestDuration = 60.seconds diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeChurnSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeChurnSpec.scala index 564eb27d29..c098e7f5f8 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeChurnSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeChurnSpec.scala @@ -89,6 +89,8 @@ abstract class NodeChurnSpec } "Cluster with short lived members" must { + "TODO work with artery" in (pending) + /* "setup stable nodes" taggedAs LongRunningTest in within(15.seconds) { val logListener = system.actorOf(Props(classOf[LogListener], testActor), "logListener") system.eventStream.subscribe(logListener, classOf[Info]) @@ -125,6 +127,8 @@ abstract class NodeChurnSpec } expectNoMsg(5.seconds) } + */ } + } diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/QuickRestartSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/QuickRestartSpec.scala index 713c02160a..ab973898d3 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/QuickRestartSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/QuickRestartSpec.scala @@ -3,22 +3,17 @@ */ package akka.cluster -import scala.collection.immutable -import scala.language.postfixOps -import scala.concurrent.duration._ -import akka.actor.Address -import akka.cluster.MemberStatus._ -import akka.remote.testkit.MultiNodeConfig -import akka.remote.testkit.MultiNodeSpec +import java.util.concurrent.ThreadLocalRandom + +import akka.actor.{ ActorSystem, Address } +import akka.remote.RARP +import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec } import akka.testkit._ import com.typesafe.config.ConfigFactory -import org.scalatest.BeforeAndAfter -import akka.actor.ActorSystem -import akka.actor.ActorRef -import akka.event.Logging.Info -import akka.actor.Actor -import akka.actor.Props -import java.util.concurrent.ThreadLocalRandom + +import scala.collection.immutable +import scala.concurrent.duration._ +import scala.language.postfixOps // This test was a reproducer for issue #20639 object QuickRestartMultiJvmSpec extends MultiNodeConfig { @@ -72,10 +67,19 @@ abstract class QuickRestartSpec else ActorSystem( system.name, - ConfigFactory.parseString(s""" - akka.cluster.roles = [round-$n] - akka.remote.netty.tcp.port = ${Cluster(restartingSystem).selfAddress.port.get}""") // same port - .withFallback(system.settings.config)) + // use the same port + ConfigFactory.parseString( + if (RARP(system).provider.remoteSettings.Artery.Enabled) + s""" + akka.cluster.roles = [round-$n] + akka.remote.artery.canonical.port = ${Cluster(restartingSystem).selfAddress.port.get} + """ + else + s""" + akka.cluster.roles = [round-$n] + akka.remote.netty.tcp.port = ${Cluster(restartingSystem).selfAddress.port.get} + """ + ).withFallback(system.settings.config)) log.info("Restarting node has address: {}", Cluster(restartingSystem).selfUniqueAddress) Cluster(restartingSystem).joinSeedNodes(seedNodes) within(20.seconds) { diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartFirstSeedNodeSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartFirstSeedNodeSpec.scala index 14ecede33a..313bc3b245 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartFirstSeedNodeSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartFirstSeedNodeSpec.scala @@ -10,6 +10,7 @@ import org.scalatest.BeforeAndAfter import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeSpec import akka.testkit._ + import scala.concurrent.duration._ import akka.actor.Address import akka.actor.ActorSystem @@ -18,6 +19,7 @@ import akka.actor.Actor import akka.actor.RootActorPath import akka.cluster.MemberStatus._ import akka.actor.Deploy +import akka.remote.RARP object RestartFirstSeedNodeMultiJvmSpec extends MultiNodeConfig { val seed1 = role("seed1") @@ -52,8 +54,12 @@ abstract class RestartFirstSeedNodeSpec lazy val restartedSeed1System = ActorSystem( system.name, - ConfigFactory.parseString("akka.remote.netty.tcp.port=" + seedNodes.head.port.get). - withFallback(system.settings.config)) + ConfigFactory.parseString( + if (RARP(system).provider.remoteSettings.Artery.Enabled) + "akka.remote.artery.canonical.port=" + seedNodes.head.port.get + else + "akka.remote.netty.tcp.port=" + seedNodes.head.port.get + ).withFallback(system.settings.config)) override def afterAll(): Unit = { runOn(seed1) { diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNode3Spec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNode3Spec.scala index 58e8a42fbc..01cd5f8e00 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNode3Spec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNode3Spec.scala @@ -5,7 +5,6 @@ package akka.cluster import scala.collection.immutable import scala.concurrent.duration._ - import akka.actor.Actor import akka.actor.ActorSystem import akka.actor.Address @@ -13,6 +12,7 @@ import akka.actor.Deploy import akka.actor.Props import akka.actor.RootActorPath import akka.cluster.MemberStatus._ +import akka.remote.RARP import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeSpec import akka.remote.transport.ThrottlerTransportAdapter.Direction @@ -50,8 +50,12 @@ abstract class RestartNode3Spec lazy val restartedSecondSystem = ActorSystem( system.name, - ConfigFactory.parseString("akka.remote.netty.tcp.port=" + secondUniqueAddress.address.port.get). - withFallback(system.settings.config)) + ConfigFactory.parseString( + if (RARP(system).provider.remoteSettings.Artery.Enabled) + "akka.remote.artery.canonical.port=" + secondUniqueAddress.address.port.get + else + "akka.remote.netty.tcp.port=" + secondUniqueAddress.address.port.get + ).withFallback(system.settings.config)) override def afterAll(): Unit = { runOn(second) { @@ -133,7 +137,7 @@ abstract class RestartNode3Spec awaitAssert { Cluster(system).readView.members.size should ===(3) Cluster(system).readView.members.exists { m ⇒ - m.address == secondUniqueAddress.address && m.uniqueAddress.uid != secondUniqueAddress.uid + m.address == secondUniqueAddress.address && m.uniqueAddress.longUid != secondUniqueAddress.longUid } } } diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNodeSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNodeSpec.scala index 17cad3e2b6..e478402dc6 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNodeSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNodeSpec.scala @@ -141,7 +141,7 @@ abstract class RestartNodeSpec awaitAssert { Cluster(system).readView.members.size should ===(3) Cluster(system).readView.members.exists { m ⇒ - m.address == secondUniqueAddress.address && m.uniqueAddress.uid != secondUniqueAddress.uid + m.address == secondUniqueAddress.address && m.uniqueAddress.longUid != secondUniqueAddress.longUid } } } diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/SharedMediaDriverSupport.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/SharedMediaDriverSupport.scala new file mode 100644 index 0000000000..f45a54b245 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/SharedMediaDriverSupport.scala @@ -0,0 +1,105 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.cluster + +import java.io.File +import java.util.concurrent.atomic.AtomicReference +import java.util.function.Consumer + +import scala.annotation.tailrec +import scala.util.control.NonFatal + +import akka.remote.RemoteSettings +import akka.remote.artery.ArterySettings +import akka.remote.artery.TaskRunner +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import com.typesafe.config.ConfigFactory +import io.aeron.driver.MediaDriver +import io.aeron.driver.ThreadingMode +import org.agrona.IoUtil + +object SharedMediaDriverSupport { + + private val mediaDriver = new AtomicReference[Option[MediaDriver]](None) + + def loadArterySettings(config: MultiNodeConfig): ArterySettings = + (new RemoteSettings(ConfigFactory.load(config.config))).Artery + + def startMediaDriver(config: MultiNodeConfig): Unit = { + val arterySettings = loadArterySettings(config) + if (arterySettings.Enabled) { + val aeronDir = arterySettings.Advanced.AeronDirectoryName + require(aeronDir.nonEmpty, "aeron-dir must be defined") + val driverContext = new MediaDriver.Context + driverContext.aeronDirectoryName(aeronDir) + driverContext.clientLivenessTimeoutNs(arterySettings.Advanced.ClientLivenessTimeout.toNanos) + driverContext.imageLivenessTimeoutNs(arterySettings.Advanced.ImageLivenessTimeout.toNanos) + driverContext.driverTimeoutMs(arterySettings.Advanced.DriverTimeout.toMillis) + + val idleCpuLevel = arterySettings.Advanced.IdleCpuLevel + driverContext + .threadingMode(ThreadingMode.SHARED) + .sharedIdleStrategy(TaskRunner.createIdleStrategy(idleCpuLevel)) + + // Check if the media driver is already started by another multi-node jvm. + // It checks more than one time with a sleep inbetween. The number of checks + // depends on the multi-node index (i). + @tailrec def isDriverInactive(i: Int): Boolean = { + if (i < 0) true + else { + val active = driverContext.isDriverActive(5000, new Consumer[String] { + override def accept(msg: String): Unit = { + println(msg) + } + }) + if (active) false + else { + Thread.sleep(500) + isDriverInactive(i - 1) + } + } + } + + try { + if (isDriverInactive(MultiNodeSpec.selfIndex)) { + val driver = MediaDriver.launchEmbedded(driverContext) + println(s"Started media driver in directory [${driver.aeronDirectoryName}]") + if (!mediaDriver.compareAndSet(None, Some(driver))) { + throw new IllegalStateException("media driver started more than once") + } + } + } catch { + case NonFatal(e) ⇒ + println(s"Failed to start media driver in [${aeronDir}]: ${e.getMessage}") + } + } + } + + def isMediaDriverRunningByThisNode: Boolean = mediaDriver.get.isDefined + + def stopMediaDriver(config: MultiNodeConfig): Unit = { + val maybeDriver = mediaDriver.getAndSet(None) + maybeDriver.foreach { driver ⇒ + val arterySettings = loadArterySettings(config) + + // let other nodes shutdown first + Thread.sleep(5000) + + driver.close() + + try { + if (arterySettings.Advanced.DeleteAeronDirectory) { + IoUtil.delete(new File(driver.aeronDirectoryName), false) + } + } catch { + case NonFatal(e) ⇒ + println( + s"Couldn't delete Aeron embedded media driver files in [${driver.aeronDirectoryName}] " + + s"due to [${e.getMessage}]") + } + } + } + +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala index 1551253ed0..72d3c2e1b3 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala @@ -45,6 +45,7 @@ import akka.actor.ActorIdentity import akka.util.Helpers.ConfigOps import akka.util.Helpers.Requiring import java.lang.management.ManagementFactory +import akka.remote.RARP /** * This test is intended to be used as long running stress test @@ -124,9 +125,9 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig { akka.actor.serialize-messages = off akka.actor.serialize-creators = off - akka.actor.provider = akka.cluster.ClusterActorRefProvider + akka.actor.provider = cluster akka.cluster { - failure-detector.acceptable-heartbeat-pause = 5s + failure-detector.acceptable-heartbeat-pause = 10s auto-down-unreachable-after = 1s publish-stats-interval = 1s } @@ -134,6 +135,12 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig { akka.loglevel = INFO akka.remote.log-remote-lifecycle-events = off + akka.remote.artery.advanced { + idle-cpu-level = 1 + embedded-media-driver = off + aeron-dir = "target/aeron-StressSpec" + } + akka.actor.default-dispatcher.fork-join-executor { parallelism-min = 8 parallelism-max = 8 @@ -699,8 +706,11 @@ class StressMultiJvmNode12 extends StressSpec class StressMultiJvmNode13 extends StressSpec abstract class StressSpec - extends MultiNodeSpec(StressMultiJvmSpec) - with MultiNodeClusterSpec with BeforeAndAfterEach with ImplicitSender { + extends MultiNodeSpec({ + // Aeron media driver must be started before ActorSystem + SharedMediaDriverSupport.startMediaDriver(StressMultiJvmSpec) + StressMultiJvmSpec + }) with MultiNodeClusterSpec with BeforeAndAfterEach with ImplicitSender { import StressMultiJvmSpec._ import ClusterEvent._ @@ -726,6 +736,20 @@ abstract class StressSpec classOf[StatsResult], classOf[PhiResult], RetryTick.getClass)(sys) } + override protected def afterTermination(): Unit = { + SharedMediaDriverSupport.stopMediaDriver(StressMultiJvmSpec) + super.afterTermination() + } + + Runtime.getRuntime.addShutdownHook(new Thread { + override def run(): Unit = { + if (SharedMediaDriverSupport.isMediaDriverRunningByThisNode) + println("Abrupt exit of JVM without closing media driver. This should not happen and may cause test failure.") + } + }) + + def isArteryEnabled: Boolean = RARP(system).provider.remoteSettings.Artery.Enabled + def jvmInfo(): String = { val runtime = ManagementFactory.getRuntimeMXBean val os = ManagementFactory.getOperatingSystemMXBean diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala index db64768148..2934581077 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala @@ -23,13 +23,17 @@ object SunnyWeatherMultiJvmSpec extends MultiNodeConfig { // Note that this test uses default configuration, // not MultiNodeClusterSpec.clusterConfig - commonConfig(ConfigFactory.parseString(""" - akka.actor.provider = akka.cluster.ClusterActorRefProvider - akka.loggers = ["akka.testkit.TestEventListener"] - akka.loglevel = INFO - akka.remote.log-remote-lifecycle-events = off - akka.cluster.failure-detector.monitored-by-nr-of-members = 3 + commonConfig(ConfigFactory.parseString( + """ + akka { + actor.provider = cluster + loggers = ["akka.testkit.TestEventListener"] + loglevel = INFO + remote.log-remote-lifecycle-events = off + cluster.failure-detector.monitored-by-nr-of-members = 3 + } """)) + } class SunnyWeatherMultiJvmNode1 extends SunnyWeatherSpec @@ -38,11 +42,11 @@ class SunnyWeatherMultiJvmNode3 extends SunnyWeatherSpec class SunnyWeatherMultiJvmNode4 extends SunnyWeatherSpec class SunnyWeatherMultiJvmNode5 extends SunnyWeatherSpec -abstract class SunnyWeatherSpec - extends MultiNodeSpec(SunnyWeatherMultiJvmSpec) +abstract class SunnyWeatherSpec extends MultiNodeSpec(SunnyWeatherMultiJvmSpec) with MultiNodeClusterSpec { import SunnyWeatherMultiJvmSpec._ + import ClusterEvent._ "A normal cluster" must { diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/SurviveNetworkInstabilitySpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/SurviveNetworkInstabilitySpec.scala index eeeedb7301..f69be2b65b 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/SurviveNetworkInstabilitySpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/SurviveNetworkInstabilitySpec.scala @@ -7,17 +7,19 @@ import com.typesafe.config.ConfigFactory import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeSpec import akka.remote.transport.ThrottlerTransportAdapter.Direction + import scala.concurrent.duration._ import akka.testkit._ import akka.testkit.TestEvent._ import java.util.concurrent.ThreadLocalRandom + import akka.remote.testconductor.RoleName import akka.actor.Props import akka.actor.Actor + import scala.util.control.NoStackTrace -import akka.remote.QuarantinedEvent +import akka.remote.{ QuarantinedEvent, RARP, RemoteActorRefProvider } import akka.actor.ExtendedActorSystem -import akka.remote.RemoteActorRefProvider import akka.actor.ActorRef import akka.dispatch.sysmsg.Failed import akka.actor.PoisonPill @@ -36,6 +38,7 @@ object SurviveNetworkInstabilityMultiJvmSpec extends MultiNodeConfig { commonConfig(debugConfig(on = false).withFallback( ConfigFactory.parseString(""" akka.remote.system-message-buffer-size=100 + akka.remote.artery.advanced.system-message-buffer-size=100 akka.remote.netty.tcp.connection-timeout = 10s """)). withFallback(MultiNodeClusterSpec.clusterConfig)) @@ -363,13 +366,14 @@ abstract class SurviveNetworkInstabilitySpec } runOn(side2: _*) { + // side2 comes back but stays unreachable val expected = ((side2 ++ side1) map address).toSet clusterView.members.map(_.address) should ===(expected) assertUnreachable(side1: _*) } enterBarrier("after-7") - assertCanTalk((side1AfterJoin): _*) + assertCanTalk(side1AfterJoin: _*) } } diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/UnreachableNodeJoinsAgainSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/UnreachableNodeJoinsAgainSpec.scala index 3284cbb3ce..5b7d26e320 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/UnreachableNodeJoinsAgainSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/UnreachableNodeJoinsAgainSpec.scala @@ -20,6 +20,7 @@ import akka.actor.ActorRef import akka.actor.Props import akka.actor.RootActorPath import akka.cluster.MultiNodeClusterSpec.EndActor +import akka.remote.RARP object UnreachableNodeJoinsAgainMultiNodeConfig extends MultiNodeConfig { val first = role("first") @@ -160,18 +161,30 @@ abstract class UnreachableNodeJoinsAgainSpec runOn(victim) { val victimAddress = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + val freshConfig = + ConfigFactory.parseString( + if (RARP(system).provider.remoteSettings.Artery.Enabled) + s""" + akka.remote.artery.canonical { + hostname = ${victimAddress.host.get} + port = ${victimAddress.port.get} + } + """ + else s""" + akka.remote.netty.tcp { + hostname = ${victimAddress.host.get} + port = ${victimAddress.port.get} + }""" + ).withFallback(system.settings.config) + Await.ready(system.whenTerminated, 10 seconds) + // create new ActorSystem with same host:port - val freshSystem = ActorSystem(system.name, ConfigFactory.parseString(s""" - akka.remote.netty.tcp { - hostname = ${victimAddress.host.get} - port = ${victimAddress.port.get} - } - """).withFallback(system.settings.config)) + val freshSystem = ActorSystem(system.name, freshConfig) try { Cluster(freshSystem).join(masterAddress) - within(15 seconds) { + within(30 seconds) { awaitAssert(Cluster(freshSystem).readView.members.map(_.address) should contain(victimAddress)) awaitAssert(Cluster(freshSystem).readView.members.size should ===(expectedNumberOfMembers)) awaitAssert(Cluster(freshSystem).readView.members.map(_.status) should ===(Set(MemberStatus.Up))) diff --git a/akka-cluster/src/test/scala/akka/cluster/AutoDownSpec.scala b/akka-cluster/src/test/scala/akka/cluster/AutoDownSpec.scala index f9bea50285..f0baa013f1 100644 --- a/akka-cluster/src/test/scala/akka/cluster/AutoDownSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/AutoDownSpec.scala @@ -11,16 +11,14 @@ import akka.actor.ActorRef import akka.actor.Props import akka.cluster.MemberStatus._ import akka.cluster.ClusterEvent._ +import akka.remote.RARP import akka.testkit.AkkaSpec object AutoDownSpec { final case class DownCalled(address: Address) - val memberA = TestMember(Address("akka.tcp", "sys", "a", 2552), Up) - val memberB = TestMember(Address("akka.tcp", "sys", "b", 2552), Up) - val memberC = TestMember(Address("akka.tcp", "sys", "c", 2552), Up) - class AutoDownTestActor( + memberA: Member, autoDownUnreachableAfter: FiniteDuration, probe: ActorRef) extends AutoDownBase(autoDownUnreachableAfter) { @@ -36,13 +34,22 @@ object AutoDownSpec { } } + } -class AutoDownSpec extends AkkaSpec { +class AutoDownSpec extends AkkaSpec("akka.actor.provider=remote") { import AutoDownSpec._ + val protocol = + if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka" + else "akka.tcp" + + val memberA = TestMember(Address(protocol, "sys", "a", 2552), Up) + val memberB = TestMember(Address(protocol, "sys", "b", 2552), Up) + val memberC = TestMember(Address(protocol, "sys", "c", 2552), Up) + def autoDownActor(autoDownUnreachableAfter: FiniteDuration): ActorRef = - system.actorOf(Props(classOf[AutoDownTestActor], autoDownUnreachableAfter, testActor)) + system.actorOf(Props(classOf[AutoDownTestActor], memberA, autoDownUnreachableAfter, testActor)) "AutoDown" must { diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterDeployerSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterDeployerSpec.scala index 9409b89019..d993b58e64 100644 --- a/akka-cluster/src/test/scala/akka/cluster/ClusterDeployerSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/ClusterDeployerSpec.scala @@ -14,7 +14,7 @@ import akka.cluster.routing.ClusterRouterGroupSettings object ClusterDeployerSpec { val deployerConf = ConfigFactory.parseString(""" - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.actor.deployment { /user/service1 { router = round-robin-pool diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala index 766a09479b..be3db51745 100644 --- a/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala @@ -17,11 +17,12 @@ import akka.cluster.ClusterEvent._ import akka.testkit.AkkaSpec import akka.testkit.ImplicitSender import akka.actor.ActorRef +import akka.remote.RARP import akka.testkit.TestProbe object ClusterDomainEventPublisherSpec { val config = """ - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.netty.tcp.port = 0 """ } @@ -29,18 +30,22 @@ object ClusterDomainEventPublisherSpec { class ClusterDomainEventPublisherSpec extends AkkaSpec(ClusterDomainEventPublisherSpec.config) with BeforeAndAfterEach with ImplicitSender { + val protocol = + if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka" + else "akka.tcp" + var publisher: ActorRef = _ - val aUp = TestMember(Address("akka.tcp", "sys", "a", 2552), Up) + val aUp = TestMember(Address(protocol, "sys", "a", 2552), Up) val aLeaving = aUp.copy(status = Leaving) val aExiting = aLeaving.copy(status = Exiting) val aRemoved = aExiting.copy(status = Removed) - val bExiting = TestMember(Address("akka.tcp", "sys", "b", 2552), Exiting) + val bExiting = TestMember(Address(protocol, "sys", "b", 2552), Exiting) val bRemoved = bExiting.copy(status = Removed) - val cJoining = TestMember(Address("akka.tcp", "sys", "c", 2552), Joining, Set("GRP")) + val cJoining = TestMember(Address(protocol, "sys", "c", 2552), Joining, Set("GRP")) val cUp = cJoining.copy(status = Up) val cRemoved = cUp.copy(status = Removed) - val a51Up = TestMember(Address("akka.tcp", "sys", "a", 2551), Up) - val dUp = TestMember(Address("akka.tcp", "sys", "d", 2552), Up, Set("GRP")) + val a51Up = TestMember(Address(protocol, "sys", "a", 2551), Up) + val dUp = TestMember(Address(protocol, "sys", "d", 2552), Up, Set("GRP")) val g0 = Gossip(members = SortedSet(aUp)).seen(aUp.uniqueAddress) val g1 = Gossip(members = SortedSet(aUp, cJoining)).seen(aUp.uniqueAddress).seen(cJoining.uniqueAddress) diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterSpec.scala index 7fec648cc4..2abf34c82f 100644 --- a/akka-cluster/src/test/scala/akka/cluster/ClusterSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/ClusterSpec.scala @@ -25,7 +25,7 @@ object ClusterSpec { publish-stats-interval = 0 s # always, when it happens failure-detector.implementation-class = akka.cluster.FailureDetectorPuppet } - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.log-remote-lifecycle-events = off akka.remote.netty.tcp.port = 0 #akka.loglevel = DEBUG @@ -107,7 +107,7 @@ class ClusterSpec extends AkkaSpec(ClusterSpec.config) with ImplicitSender { "allow join and leave with local address" in { val sys2 = ActorSystem("ClusterSpec2", ConfigFactory.parseString(""" - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.netty.tcp.port = 0 """)) try { diff --git a/akka-cluster/src/test/scala/akka/cluster/DowningProviderSpec.scala b/akka-cluster/src/test/scala/akka/cluster/DowningProviderSpec.scala index 87025c791b..d1c540422b 100644 --- a/akka-cluster/src/test/scala/akka/cluster/DowningProviderSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/DowningProviderSpec.scala @@ -38,7 +38,7 @@ class DowningProviderSpec extends WordSpec with Matchers { """ akka { loglevel = WARNING - actor.provider = "akka.cluster.ClusterActorRefProvider" + actor.provider = "cluster" remote { netty.tcp { hostname = "127.0.0.1" diff --git a/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala b/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala index a07aea4f17..55823c387d 100644 --- a/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala @@ -21,7 +21,7 @@ object MetricsEnabledSpec { akka.cluster.metrics.enabled = on akka.cluster.metrics.collect-interval = 1 s akka.cluster.metrics.gossip-interval = 1 s - akka.actor.provider = "akka.remote.RemoteActorRefProvider" + akka.actor.provider = remote """ } diff --git a/akka-cluster/src/test/scala/akka/cluster/StartupWithOneThreadSpec.scala b/akka-cluster/src/test/scala/akka/cluster/StartupWithOneThreadSpec.scala index 028a881e32..54d3ae9db7 100644 --- a/akka-cluster/src/test/scala/akka/cluster/StartupWithOneThreadSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/StartupWithOneThreadSpec.scala @@ -14,7 +14,7 @@ import akka.actor.ActorLogging object StartupWithOneThreadSpec { val config = """ - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.actor.creation-timeout = 10s akka.remote.netty.tcp.port = 0 diff --git a/akka-cluster/src/test/scala/akka/cluster/protobuf/ClusterMessageSerializerSpec.scala b/akka-cluster/src/test/scala/akka/cluster/protobuf/ClusterMessageSerializerSpec.scala index 8ef6d7e938..e94f1b359c 100644 --- a/akka-cluster/src/test/scala/akka/cluster/protobuf/ClusterMessageSerializerSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/protobuf/ClusterMessageSerializerSpec.scala @@ -11,7 +11,7 @@ import collection.immutable.SortedSet import akka.testkit.AkkaSpec class ClusterMessageSerializerSpec extends AkkaSpec( - "akka.actor.provider = akka.cluster.ClusterActorRefProvider") { + "akka.actor.provider = cluster") { val serializer = new ClusterMessageSerializer(system.asInstanceOf[ExtendedActorSystem]) diff --git a/akka-cluster/src/test/scala/akka/cluster/routing/ClusterRouterSupervisorSpec.scala b/akka-cluster/src/test/scala/akka/cluster/routing/ClusterRouterSupervisorSpec.scala index e034b67ecb..0b895f33b3 100644 --- a/akka-cluster/src/test/scala/akka/cluster/routing/ClusterRouterSupervisorSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/routing/ClusterRouterSupervisorSpec.scala @@ -22,7 +22,7 @@ object ClusterRouterSupervisorSpec { } class ClusterRouterSupervisorSpec extends AkkaSpec(""" - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.netty.tcp.port = 0 """) { diff --git a/akka-cluster/src/test/scala/akka/cluster/routing/WeightedRouteesSpec.scala b/akka-cluster/src/test/scala/akka/cluster/routing/WeightedRouteesSpec.scala index 224394fdae..08f982f1d1 100644 --- a/akka-cluster/src/test/scala/akka/cluster/routing/WeightedRouteesSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/routing/WeightedRouteesSpec.scala @@ -7,19 +7,24 @@ package akka.cluster.routing import com.typesafe.config.ConfigFactory import akka.actor.Address import akka.actor.RootActorPath +import akka.remote.RARP import akka.testkit.AkkaSpec import akka.routing.ActorSelectionRoutee import akka.routing.ActorRefRoutee class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString(""" - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.netty.tcp.port = 0 """)) { - val a1 = Address("akka.tcp", "sys", "a1", 2551) - val b1 = Address("akka.tcp", "sys", "b1", 2551) - val c1 = Address("akka.tcp", "sys", "c1", 2551) - val d1 = Address("akka.tcp", "sys", "d1", 2551) + val protocol = + if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka" + else "akka.tcp" + + val a1 = Address(protocol, "sys", "a1", 2551) + val b1 = Address(protocol, "sys", "b1", 2551) + val c1 = Address(protocol, "sys", "c1", 2551) + val d1 = Address(protocol, "sys", "d1", 2551) val routeeA = ActorSelectionRoutee(system.actorSelection(RootActorPath(a1) / "user" / "a")) val routeeB = ActorSelectionRoutee(system.actorSelection(RootActorPath(b1) / "user" / "b")) diff --git a/akka-distributed-data/src/main/java/akka/cluster/ddata/protobuf/msg/ReplicatorMessages.java b/akka-distributed-data/src/main/java/akka/cluster/ddata/protobuf/msg/ReplicatorMessages.java index 69f9c4a156..ced5725d11 100644 --- a/akka-distributed-data/src/main/java/akka/cluster/ddata/protobuf/msg/ReplicatorMessages.java +++ b/akka-distributed-data/src/main/java/akka/cluster/ddata/protobuf/msg/ReplicatorMessages.java @@ -12421,6 +12421,24 @@ public final class ReplicatorMessages { * required sfixed32 uid = 2; */ int getUid(); + + // optional sfixed32 uid2 = 3; + /** + * optional sfixed32 uid2 = 3; + * + *
+     * 64 bit uids but with backward wire compatibility
+     * 
+ */ + boolean hasUid2(); + /** + * optional sfixed32 uid2 = 3; + * + *
+     * 64 bit uids but with backward wire compatibility
+     * 
+ */ + int getUid2(); } /** * Protobuf type {@code akka.cluster.ddata.UniqueAddress} @@ -12491,6 +12509,11 @@ public final class ReplicatorMessages { uid_ = input.readSFixed32(); break; } + case 29: { + bitField0_ |= 0x00000004; + uid2_ = input.readSFixed32(); + break; + } } } } catch (akka.protobuf.InvalidProtocolBufferException e) { @@ -12569,9 +12592,34 @@ public final class ReplicatorMessages { return uid_; } + // optional sfixed32 uid2 = 3; + public static final int UID2_FIELD_NUMBER = 3; + private int uid2_; + /** + * optional sfixed32 uid2 = 3; + * + *
+     * 64 bit uids but with backward wire compatibility
+     * 
+ */ + public boolean hasUid2() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional sfixed32 uid2 = 3; + * + *
+     * 64 bit uids but with backward wire compatibility
+     * 
+ */ + public int getUid2() { + return uid2_; + } + private void initFields() { address_ = akka.cluster.ddata.protobuf.msg.ReplicatorMessages.Address.getDefaultInstance(); uid_ = 0; + uid2_ = 0; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -12603,6 +12651,9 @@ public final class ReplicatorMessages { if (((bitField0_ & 0x00000002) == 0x00000002)) { output.writeSFixed32(2, uid_); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeSFixed32(3, uid2_); + } getUnknownFields().writeTo(output); } @@ -12620,6 +12671,10 @@ public final class ReplicatorMessages { size += akka.protobuf.CodedOutputStream .computeSFixed32Size(2, uid_); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += akka.protobuf.CodedOutputStream + .computeSFixed32Size(3, uid2_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -12745,6 +12800,8 @@ public final class ReplicatorMessages { bitField0_ = (bitField0_ & ~0x00000001); uid_ = 0; bitField0_ = (bitField0_ & ~0x00000002); + uid2_ = 0; + bitField0_ = (bitField0_ & ~0x00000004); return this; } @@ -12785,6 +12842,10 @@ public final class ReplicatorMessages { to_bitField0_ |= 0x00000002; } result.uid_ = uid_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.uid2_ = uid2_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -12807,6 +12868,9 @@ public final class ReplicatorMessages { if (other.hasUid()) { setUid(other.getUid()); } + if (other.hasUid2()) { + setUid2(other.getUid2()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -12996,6 +13060,55 @@ public final class ReplicatorMessages { return this; } + // optional sfixed32 uid2 = 3; + private int uid2_ ; + /** + * optional sfixed32 uid2 = 3; + * + *
+       * 64 bit uids but with backward wire compatibility
+       * 
+ */ + public boolean hasUid2() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional sfixed32 uid2 = 3; + * + *
+       * 64 bit uids but with backward wire compatibility
+       * 
+ */ + public int getUid2() { + return uid2_; + } + /** + * optional sfixed32 uid2 = 3; + * + *
+       * 64 bit uids but with backward wire compatibility
+       * 
+ */ + public Builder setUid2(int value) { + bitField0_ |= 0x00000004; + uid2_ = value; + onChanged(); + return this; + } + /** + * optional sfixed32 uid2 = 3; + * + *
+       * 64 bit uids but with backward wire compatibility
+       * 
+ */ + public Builder clearUid2() { + bitField0_ = (bitField0_ & ~0x00000004); + uid2_ = 0; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:akka.cluster.ddata.UniqueAddress) } @@ -14806,14 +14919,14 @@ public final class ReplicatorMessages { " \002(\010\0221\n\007entries\030\002 \003(\0132 .akka.cluster.dda" + "ta.Gossip.Entry\032H\n\005Entry\022\013\n\003key\030\001 \002(\t\0222\n" + "\010envelope\030\002 \002(\0132 .akka.cluster.ddata.Dat", - "aEnvelope\"J\n\rUniqueAddress\022,\n\007address\030\001 " + + "aEnvelope\"X\n\rUniqueAddress\022,\n\007address\030\001 " + "\002(\0132\033.akka.cluster.ddata.Address\022\013\n\003uid\030" + - "\002 \002(\017\")\n\007Address\022\020\n\010hostname\030\001 \002(\t\022\014\n\004po" + - "rt\030\002 \002(\r\"V\n\014OtherMessage\022\027\n\017enclosedMess" + - "age\030\001 \002(\014\022\024\n\014serializerId\030\002 \002(\005\022\027\n\017messa" + - "geManifest\030\004 \001(\014\"\036\n\nStringGSet\022\020\n\010elemen" + - "ts\030\001 \003(\tB#\n\037akka.cluster.ddata.protobuf." + - "msgH\001" + "\002 \002(\017\022\014\n\004uid2\030\003 \001(\017\")\n\007Address\022\020\n\010hostna" + + "me\030\001 \002(\t\022\014\n\004port\030\002 \002(\r\"V\n\014OtherMessage\022\027" + + "\n\017enclosedMessage\030\001 \002(\014\022\024\n\014serializerId\030" + + "\002 \002(\005\022\027\n\017messageManifest\030\004 \001(\014\"\036\n\nString" + + "GSet\022\020\n\010elements\030\001 \003(\tB#\n\037akka.cluster.d" + + "data.protobuf.msgH\001" }; akka.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new akka.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -14927,7 +15040,7 @@ public final class ReplicatorMessages { internal_static_akka_cluster_ddata_UniqueAddress_fieldAccessorTable = new akka.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_akka_cluster_ddata_UniqueAddress_descriptor, - new java.lang.String[] { "Address", "Uid", }); + new java.lang.String[] { "Address", "Uid", "Uid2", }); internal_static_akka_cluster_ddata_Address_descriptor = getDescriptor().getMessageTypes().get(15); internal_static_akka_cluster_ddata_Address_fieldAccessorTable = new diff --git a/akka-distributed-data/src/main/protobuf/ReplicatorMessages.proto b/akka-distributed-data/src/main/protobuf/ReplicatorMessages.proto index c666716e3b..9d3a93b68b 100644 --- a/akka-distributed-data/src/main/protobuf/ReplicatorMessages.proto +++ b/akka-distributed-data/src/main/protobuf/ReplicatorMessages.proto @@ -98,6 +98,8 @@ message Gossip { message UniqueAddress { required Address address = 1; required sfixed32 uid = 2; + // 64 bit uids but with backward wire compatibility + optional sfixed32 uid2 = 3; } message Address { diff --git a/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/SerializationSupport.scala b/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/SerializationSupport.scala index 11dab79f11..054ea0ab81 100644 --- a/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/SerializationSupport.scala +++ b/akka-distributed-data/src/main/scala/akka/cluster/ddata/protobuf/SerializationSupport.scala @@ -88,10 +88,21 @@ trait SerializationSupport { Address(addressProtocol, system.name, address.getHostname, address.getPort) def uniqueAddressToProto(uniqueAddress: UniqueAddress): dm.UniqueAddress.Builder = - dm.UniqueAddress.newBuilder().setAddress(addressToProto(uniqueAddress.address)).setUid(uniqueAddress.uid) + dm.UniqueAddress.newBuilder().setAddress(addressToProto(uniqueAddress.address)) + .setUid(uniqueAddress.longUid.toInt) + .setUid2((uniqueAddress.longUid >> 32).toInt) def uniqueAddressFromProto(uniqueAddress: dm.UniqueAddress): UniqueAddress = - UniqueAddress(addressFromProto(uniqueAddress.getAddress), uniqueAddress.getUid) + UniqueAddress( + addressFromProto(uniqueAddress.getAddress), + if (uniqueAddress.hasUid2) { + // new remote node join the two parts of the long uid back + (uniqueAddress.getUid2.toLong << 32) | (uniqueAddress.getUid & 0xFFFFFFFFL) + } else { + // old remote node + uniqueAddress.getUid.toLong + } + ) def resolveActorRef(path: String): ActorRef = system.provider.resolveActorRef(path) diff --git a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/JepsenInspiredInsertSpec.scala b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/JepsenInspiredInsertSpec.scala index b7a70f86b4..cc1c5cb0c5 100644 --- a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/JepsenInspiredInsertSpec.scala +++ b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/JepsenInspiredInsertSpec.scala @@ -24,7 +24,7 @@ object JepsenInspiredInsertSpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.log-dead-letters = off akka.log-dead-letters-during-shutdown = off akka.remote.log-remote-lifecycle-events = ERROR diff --git a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/PerformanceSpec.scala b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/PerformanceSpec.scala index 589b55d948..d2b1142621 100644 --- a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/PerformanceSpec.scala +++ b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/PerformanceSpec.scala @@ -26,7 +26,7 @@ object PerformanceSpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = ERROR akka.stdout-loglevel = ERROR - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.log-dead-letters = off akka.log-dead-letters-during-shutdown = off akka.remote.log-remote-lifecycle-events = ERROR diff --git a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorChaosSpec.scala b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorChaosSpec.scala index 7fe8e31c73..44cea5dbcb 100644 --- a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorChaosSpec.scala +++ b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorChaosSpec.scala @@ -22,7 +22,7 @@ object ReplicatorChaosSpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.cluster.roles = ["backend"] akka.log-dead-letters-during-shutdown = off """)) diff --git a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorPruningSpec.scala b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorPruningSpec.scala index 36cc61e623..dcc25e9a22 100644 --- a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorPruningSpec.scala +++ b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorPruningSpec.scala @@ -21,7 +21,7 @@ object ReplicatorPruningSpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.log-dead-letters-during-shutdown = off """)) diff --git a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorSpec.scala b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorSpec.scala index 0755e35cc9..d4407634d6 100644 --- a/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorSpec.scala +++ b/akka-distributed-data/src/multi-jvm/scala/akka/cluster/ddata/ReplicatorSpec.scala @@ -20,7 +20,7 @@ object ReplicatorSpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.log-dead-letters-during-shutdown = off """)) diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/LocalConcurrencySpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/LocalConcurrencySpec.scala index ccc2a2f7eb..854f0d359d 100644 --- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/LocalConcurrencySpec.scala +++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/LocalConcurrencySpec.scala @@ -44,7 +44,7 @@ class LocalConcurrencySpec(_system: ActorSystem) extends TestKit(_system) this(ActorSystem( "LocalConcurrencySpec", ConfigFactory.parseString(""" - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.netty.tcp.port=0 """))) } diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/LotsOfDataBot.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/LotsOfDataBot.scala index 89c6b12071..ef9732b079 100644 --- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/LotsOfDataBot.scala +++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/LotsOfDataBot.scala @@ -40,7 +40,7 @@ object LotsOfDataBot { ConfigFactory.parseString(""" passive = off max-entries = 100000 - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote { netty.tcp { hostname = "127.0.0.1" diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/WriteAggregatorSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/WriteAggregatorSpec.scala index 5ce047ddcf..23c2aecabe 100644 --- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/WriteAggregatorSpec.scala +++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/WriteAggregatorSpec.scala @@ -13,6 +13,7 @@ import akka.actor.ActorRef import akka.cluster.ddata.Replicator.Internal._ import akka.cluster.ddata.Replicator._ import akka.actor.ActorSelection +import akka.remote.RARP object WriteAggregatorSpec { @@ -50,12 +51,16 @@ object WriteAggregatorSpec { } class WriteAggregatorSpec extends AkkaSpec(""" - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.netty.tcp.port=0 """) with ImplicitSender { - val nodeA = Address("akka.tcp", "Sys", "a", 2552) + val protocol = + if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka" + else "akka.tcp" + + val nodeA = Address(protocol, "Sys", "a", 2552) val nodeB = nodeA.copy(host = Some("b")) val nodeC = nodeA.copy(host = Some("c")) val nodeD = nodeA.copy(host = Some("d")) diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/protobuf/ReplicatedDataSerializerSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/protobuf/ReplicatedDataSerializerSpec.scala index 3512224da2..0869b20b1f 100644 --- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/protobuf/ReplicatedDataSerializerSpec.scala +++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/protobuf/ReplicatedDataSerializerSpec.scala @@ -23,20 +23,23 @@ import akka.cluster.ddata.Replicator.Internal._ import akka.cluster.ddata.VersionVector import akka.testkit.TestKit import akka.cluster.UniqueAddress +import akka.remote.RARP import com.typesafe.config.ConfigFactory class ReplicatedDataSerializerSpec extends TestKit(ActorSystem( "ReplicatedDataSerializerSpec", ConfigFactory.parseString(""" - akka.actor.provider=akka.cluster.ClusterActorRefProvider + akka.actor.provider=cluster akka.remote.netty.tcp.port=0 """))) with WordSpecLike with Matchers with BeforeAndAfterAll { val serializer = new ReplicatedDataSerializer(system.asInstanceOf[ExtendedActorSystem]) - val address1 = UniqueAddress(Address("akka.tcp", system.name, "some.host.org", 4711), 1) - val address2 = UniqueAddress(Address("akka.tcp", system.name, "other.host.org", 4711), 2) - val address3 = UniqueAddress(Address("akka.tcp", system.name, "some.host.org", 4712), 3) + val Protocol = if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka" else "akka.tcp" + + val address1 = UniqueAddress(Address(Protocol, system.name, "some.host.org", 4711), 1) + val address2 = UniqueAddress(Address(Protocol, system.name, "other.host.org", 4711), 2) + val address3 = UniqueAddress(Address(Protocol, system.name, "some.host.org", 4712), 3) override def afterAll { shutdown() diff --git a/akka-distributed-data/src/test/scala/akka/cluster/ddata/protobuf/ReplicatorMessageSerializerSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/protobuf/ReplicatorMessageSerializerSpec.scala index 7206a7d11f..70c314315f 100644 --- a/akka-distributed-data/src/test/scala/akka/cluster/ddata/protobuf/ReplicatorMessageSerializerSpec.scala +++ b/akka-distributed-data/src/test/scala/akka/cluster/ddata/protobuf/ReplicatorMessageSerializerSpec.scala @@ -21,20 +21,23 @@ import akka.cluster.ddata.Replicator.Internal._ import akka.testkit.TestKit import akka.util.ByteString import akka.cluster.UniqueAddress +import akka.remote.RARP import com.typesafe.config.ConfigFactory class ReplicatorMessageSerializerSpec extends TestKit(ActorSystem( "ReplicatorMessageSerializerSpec", ConfigFactory.parseString(""" - akka.actor.provider=akka.cluster.ClusterActorRefProvider + akka.actor.provider=cluster akka.remote.netty.tcp.port=0 """))) with WordSpecLike with Matchers with BeforeAndAfterAll { val serializer = new ReplicatorMessageSerializer(system.asInstanceOf[ExtendedActorSystem]) - val address1 = UniqueAddress(Address("akka.tcp", system.name, "some.host.org", 4711), 1) - val address2 = UniqueAddress(Address("akka.tcp", system.name, "other.host.org", 4711), 2) - val address3 = UniqueAddress(Address("akka.tcp", system.name, "some.host.org", 4712), 3) + val Protocol = if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka" else "akka.tcp" + + val address1 = UniqueAddress(Address(Protocol, system.name, "some.host.org", 4711), 1) + val address2 = UniqueAddress(Address(Protocol, system.name, "other.host.org", 4711), 2) + val address3 = UniqueAddress(Address(Protocol, system.name, "some.host.org", 4712), 3) val keyA = GSetKey[String]("A") diff --git a/akka-docs/rst/general/configuration.rst b/akka-docs/rst/general/configuration.rst index 379d080a3b..ffaf9549fc 100644 --- a/akka-docs/rst/general/configuration.rst +++ b/akka-docs/rst/general/configuration.rst @@ -153,7 +153,7 @@ A custom ``application.conf`` might look like this:: logging-filter = "akka.event.slf4j.Slf4jLoggingFilter" actor { - provider = "akka.cluster.ClusterActorRefProvider" + provider = "cluster" default-dispatcher { # Throughput for default Dispatcher, set to 1 for as fair as possible diff --git a/akka-docs/rst/java/cluster-client.rst b/akka-docs/rst/java/cluster-client.rst index 29aa4413a6..8f95e46d0c 100644 --- a/akka-docs/rst/java/cluster-client.rst +++ b/akka-docs/rst/java/cluster-client.rst @@ -19,8 +19,8 @@ i.e. not necessarily the initial contact points. provided in a more efficient way by :ref:`distributed-pub-sub-java` for actors that belong to the same cluster. -Also, note it's necessary to change ``akka.actor.provider`` from ``akka.actor.LocalActorRefProvider`` -to ``akka.remote.RemoteActorRefProvider`` or ``akka.cluster.ClusterActorRefProvider`` when using +Also, note it's necessary to change ``akka.actor.provider`` from ``local`` +to ``remote`` or ``cluster`` when using the cluster client. The receptionist is supposed to be started on all nodes, or all nodes with specified role, diff --git a/akka-docs/rst/java/cluster-usage.rst b/akka-docs/rst/java/cluster-usage.rst index f6b8362c8b..b012ec35d5 100644 --- a/akka-docs/rst/java/cluster-usage.rst +++ b/akka-docs/rst/java/cluster-usage.rst @@ -31,7 +31,7 @@ The ``application.conf`` configuration looks like this: .. includecode:: ../../../akka-samples/akka-sample-cluster-java/src/main/resources/application.conf#snippet To enable cluster capabilities in your Akka project you should, at a minimum, add the :ref:`remoting-java` -settings, but with ``akka.cluster.ClusterActorRefProvider``. +settings, but with ``cluster``. The ``akka.cluster.seed-nodes`` should normally also be added to your ``application.conf`` file. .. note:: diff --git a/akka-docs/rst/java/remoting.rst b/akka-docs/rst/java/remoting.rst index 8423d5007c..b1e8390ece 100644 --- a/akka-docs/rst/java/remoting.rst +++ b/akka-docs/rst/java/remoting.rst @@ -31,7 +31,7 @@ to your ``application.conf`` file:: akka { actor { - provider = "akka.remote.RemoteActorRefProvider" + provider = remote } remote { enabled-transports = ["akka.remote.netty.tcp"] @@ -44,7 +44,7 @@ to your ``application.conf`` file:: As you can see in the example above there are four things you need to add to get started: -* Change provider from ``akka.actor.LocalActorRefProvider`` to ``akka.remote.RemoteActorRefProvider`` +* Change provider from ``local`` to ``remote`` * Add host name - the machine you want to run the actor system on; this host name is exactly what is passed to remote systems in order to identify this system and consequently used for connecting back to this system if need be, diff --git a/akka-docs/rst/scala/cluster-client.rst b/akka-docs/rst/scala/cluster-client.rst index 1289f325b4..b6a65bec1f 100644 --- a/akka-docs/rst/scala/cluster-client.rst +++ b/akka-docs/rst/scala/cluster-client.rst @@ -19,8 +19,8 @@ i.e. not necessarily the initial contact points. provided in a more efficient way by :ref:`distributed-pub-sub-scala` for actors that belong to the same cluster. -Also, note it's necessary to change ``akka.actor.provider`` from ``akka.actor.LocalActorRefProvider`` -to ``akka.remote.RemoteActorRefProvider`` or ``akka.cluster.ClusterActorRefProvider`` when using +Also, note it's necessary to change ``akka.actor.provider`` from ``local`` +to ``remote`` or ``cluster`` when using the cluster client. The receptionist is supposed to be started on all nodes, or all nodes with specified role, diff --git a/akka-docs/rst/scala/cluster-usage.rst b/akka-docs/rst/scala/cluster-usage.rst index da89399565..4c3b3ebec7 100644 --- a/akka-docs/rst/scala/cluster-usage.rst +++ b/akka-docs/rst/scala/cluster-usage.rst @@ -25,7 +25,7 @@ The ``application.conf`` configuration looks like this: .. includecode:: ../../../akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf#snippet To enable cluster capabilities in your Akka project you should, at a minimum, add the :ref:`remoting-scala` -settings, but with ``akka.cluster.ClusterActorRefProvider``. +settings, but with ``cluster``. The ``akka.cluster.seed-nodes`` should normally also be added to your ``application.conf`` file. .. note:: diff --git a/akka-docs/rst/scala/code/docs/cluster/ClusterDocSpec.scala b/akka-docs/rst/scala/code/docs/cluster/ClusterDocSpec.scala index 22cedd56c1..8680cd9406 100644 --- a/akka-docs/rst/scala/code/docs/cluster/ClusterDocSpec.scala +++ b/akka-docs/rst/scala/code/docs/cluster/ClusterDocSpec.scala @@ -10,7 +10,7 @@ object ClusterDocSpec { val config = """ - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.netty.tcp.port = 0 """ } diff --git a/akka-docs/rst/scala/code/docs/ddata/DistributedDataDocSpec.scala b/akka-docs/rst/scala/code/docs/ddata/DistributedDataDocSpec.scala index 9da5fd3901..75c2f5211a 100644 --- a/akka-docs/rst/scala/code/docs/ddata/DistributedDataDocSpec.scala +++ b/akka-docs/rst/scala/code/docs/ddata/DistributedDataDocSpec.scala @@ -19,7 +19,7 @@ object DistributedDataDocSpec { val config = """ - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.netty.tcp.port = 0 #//#serializer-config diff --git a/akka-docs/rst/scala/code/docs/remoting/RemoteDeploymentDocSpec.scala b/akka-docs/rst/scala/code/docs/remoting/RemoteDeploymentDocSpec.scala index f331ff38c5..3f25fb38b7 100644 --- a/akka-docs/rst/scala/code/docs/remoting/RemoteDeploymentDocSpec.scala +++ b/akka-docs/rst/scala/code/docs/remoting/RemoteDeploymentDocSpec.scala @@ -19,7 +19,7 @@ object RemoteDeploymentDocSpec { } class RemoteDeploymentDocSpec extends AkkaSpec(""" - akka.actor.provider = "akka.remote.RemoteActorRefProvider" + akka.actor.provider = remote akka.remote.netty.tcp { port = 0 } diff --git a/akka-docs/rst/scala/remoting.rst b/akka-docs/rst/scala/remoting.rst index 7849ebf879..4dd3e801c8 100644 --- a/akka-docs/rst/scala/remoting.rst +++ b/akka-docs/rst/scala/remoting.rst @@ -27,7 +27,7 @@ to your ``application.conf`` file:: akka { actor { - provider = "akka.remote.RemoteActorRefProvider" + provider = remote } remote { enabled-transports = ["akka.remote.netty.tcp"] @@ -40,7 +40,7 @@ to your ``application.conf`` file:: As you can see in the example above there are four things you need to add to get started: -* Change provider from ``akka.actor.LocalActorRefProvider`` to ``akka.remote.RemoteActorRefProvider`` +* Change provider from ``local`` to ``remote`` * Add host name - the machine you want to run the actor system on; this host name is exactly what is passed to remote systems in order to identify this system and consequently used for connecting back to this system if need be, diff --git a/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolGateway.scala b/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolGateway.scala index a3573c9c1d..21e01a84b3 100644 --- a/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolGateway.scala +++ b/akka-http-core/src/main/scala/akka/http/impl/engine/client/PoolGateway.scala @@ -89,4 +89,4 @@ private[http] object PoolGateway { private[this] val uniqueGatewayId = new AtomicLong(0) def newUniqueGatewayIdentifier = UniqueGateway(uniqueGatewayId.incrementAndGet()) -} \ No newline at end of file +} diff --git a/akka-http-core/src/test/scala/akka/http/impl/util/One2OneBidiFlowSpec.scala b/akka-http-core/src/test/scala/akka/http/impl/util/One2OneBidiFlowSpec.scala index 31f2bd24de..75d5739c41 100644 --- a/akka-http-core/src/test/scala/akka/http/impl/util/One2OneBidiFlowSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/impl/util/One2OneBidiFlowSpec.scala @@ -114,7 +114,7 @@ class One2OneBidiFlowSpec extends AkkaSpec { out.sendComplete() // To please assertAllStagesStopped } - "not pull when input is closed before surpressed pull can be acted on" in assertAllStagesStopped { + "not pull when input is closed before surpressed pull can be acted on" in assertAllStagesStopped { val in = TestPublisher.probe[Int]() val out = TestSubscriber.probe[Int]() val wrappedIn = TestSubscriber.probe[Int]() diff --git a/akka-multi-node-testkit/src/main/resources/reference.conf b/akka-multi-node-testkit/src/main/resources/reference.conf index 6d6010b7f4..f5193dd30f 100644 --- a/akka-multi-node-testkit/src/main/resources/reference.conf +++ b/akka-multi-node-testkit/src/main/resources/reference.conf @@ -13,7 +13,7 @@ akka { barrier-timeout = 30s # Timeout for interrogation of TestConductor’s Controller actor - query-timeout = 5s + query-timeout = 10s # Threshold for packet size in time unit above which the failure injector will # split the packet and deliver in smaller portions; do not give value smaller diff --git a/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Conductor.scala b/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Conductor.scala index 5ac5c688a4..7f12b6d4e5 100644 --- a/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Conductor.scala +++ b/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Conductor.scala @@ -122,10 +122,17 @@ trait Conductor { this: TestConductorExt ⇒ def blackhole(node: RoleName, target: RoleName, direction: Direction): Future[Done] = throttle(node, target, direction, 0f) - private def requireTestConductorTranport(): Unit = - if (!transport.defaultAddress.protocol.contains(".trttl.gremlin.")) - throw new ConfigurationException("To use this feature you must activate the failure injector adapters " + - "(trttl, gremlin) by specifying `testTransport(on = true)` in your MultiNodeConfig.") + private def requireTestConductorTranport(): Unit = { + if (transport.provider.remoteSettings.Artery.Enabled) { + if (!transport.provider.remoteSettings.Artery.Advanced.TestMode) + throw new ConfigurationException("To use this feature you must activate the test mode " + + "by specifying `testTransport(on = true)` in your MultiNodeConfig.") + } else { + if (!transport.defaultAddress.protocol.contains(".trttl.gremlin.")) + throw new ConfigurationException("To use this feature you must activate the failure injector adapters " + + "(trttl, gremlin) by specifying `testTransport(on = true)` in your MultiNodeConfig.") + } + } /** * Switch the Netty pipeline of the remote support into pass through mode for diff --git a/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/FlightRecordingSupport.scala b/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/FlightRecordingSupport.scala new file mode 100644 index 0000000000..a19b067f46 --- /dev/null +++ b/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/FlightRecordingSupport.scala @@ -0,0 +1,59 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.testkit + +import java.nio.file.{ FileSystems, Files, Path } + +import akka.remote.RARP +import akka.remote.artery.FlightRecorderReader + +/** + * Provides test framework agnostic methods to dump the artery flight recorder data after a test has completed - you + * must integrate the logic with the testing tool you use yourself. + * + * The flight recorder must be enabled and the flight recorder destination must be an absolute file name so + * that the akka config can be used to find it. For example you could ensure a unique file per test using + * something like this in your config: + * {{{ + * akka.remote.artery.advanced.flight-recorder { + * enabled=on + * destination=target/flight-recorder-${UUID.randomUUID().toString}.afr + * } + * }}} + * + * You need to hook in dump and deletion of files where it makes sense in your tests. (For example, dump after all tests has + * run and there was a failure and then delete) + */ +trait FlightRecordingSupport { self: MultiNodeSpec ⇒ + private lazy val arteryEnabled = + RARP(system).provider.remoteSettings.Artery.Enabled + private lazy val flightRecorderFile: Path = + FileSystems.getDefault.getPath(RARP(system).provider.remoteSettings.Artery.Advanced.FlightRecorderDestination) + + /** + * Delete flight the recorder file if it exists + */ + final protected def deleteFlightRecorderFile(): Unit = { + if (arteryEnabled && destinationIsValidForDump() && Files.exists(flightRecorderFile)) { + Files.delete(flightRecorderFile) + } + } + + /** + * Dump the contents of the flight recorder file to standard output + */ + final protected def printFlightRecording(): Unit = { + if (arteryEnabled && destinationIsValidForDump() && Files.exists(flightRecorderFile)) { + // use stdout/println as we do not know if the system log is alive + println(s"Flight recorder dump from '$flightRecorderFile':") + FlightRecorderReader.dumpToStdout(flightRecorderFile) + } + } + + private def destinationIsValidForDump() = { + val path = flightRecorderFile.toString + path != "" && path.endsWith(".afr") + } + +} diff --git a/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala b/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala index 7bf179e12a..aaa899cac7 100644 --- a/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala +++ b/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala @@ -5,15 +5,18 @@ package akka.remote.testkit import language.implicitConversions import java.net.{ InetAddress, InetSocketAddress } -import com.typesafe.config.{ ConfigObject, ConfigFactory, Config } -import scala.concurrent.{ Await, Awaitable } + +import com.typesafe.config.{ Config, ConfigFactory, ConfigObject } + +import scala.concurrent.{ Await, Awaitable, Future } import scala.util.control.NonFatal import scala.collection.immutable import akka.actor._ import akka.util.Timeout -import akka.remote.testconductor.{ TestConductorExt, TestConductor, RoleName } +import akka.remote.testconductor.{ RoleName, TestConductor, TestConductorExt } import akka.testkit._ import akka.testkit.TestEvent._ + import scala.concurrent.duration._ import akka.remote.testconductor.RoleName import akka.actor.RootActorPath @@ -56,6 +59,10 @@ abstract class MultiNodeConfig { log-received-messages = on log-sent-messages = on } + akka.remote.artery { + log-received-messages = on + log-sent-messages = on + } akka.actor.debug { receive = on fsm = on @@ -94,11 +101,12 @@ abstract class MultiNodeConfig { _roles(MultiNodeSpec.selfIndex) } - private[testkit] def config: Config = { + private[akka] def config: Config = { val transportConfig = if (_testTransport) ConfigFactory.parseString( """ akka.remote.netty.tcp.applied-adapters = [trttl, gremlin] + akka.remote.artery.advanced.test-mode = on """) else ConfigFactory.empty @@ -195,9 +203,11 @@ object MultiNodeSpec { require(selfIndex >= 0 && selfIndex < maxNodes, "multinode.index is out of bounds: " + selfIndex) private[testkit] val nodeConfig = mapToConfig(Map( - "akka.actor.provider" → "akka.remote.RemoteActorRefProvider", + "akka.actor.provider" → "remote", + "akka.remote.artery.canonical.hostname" → selfName, "akka.remote.netty.tcp.hostname" → selfName, - "akka.remote.netty.tcp.port" → selfPort)) + "akka.remote.netty.tcp.port" → selfPort, + "akka.remote.artery.canonical.port" → selfPort)) private[testkit] val baseConfig: Config = ConfigFactory.parseString(""" akka { diff --git a/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/PerfFlamesSupport.scala b/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/PerfFlamesSupport.scala new file mode 100644 index 0000000000..3ca461388c --- /dev/null +++ b/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/PerfFlamesSupport.scala @@ -0,0 +1,57 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.testkit + +import java.io.File + +import akka.remote.testconductor.RoleName + +import scala.concurrent.Future +import scala.concurrent.duration._ + +/** + * INTERNAL API: Support trait allowing trivially recording perf metrics from [[MultiNodeSpec]]s + */ +private[akka] trait PerfFlamesSupport { _: MultiNodeSpec ⇒ + + /** + * Runs `perf-java-flames` script on given node (JVM process). + * Refer to https://github.com/jrudolph/perf-map-agent for options and manual. + * + * Options are currently to be passed in via `export PERF_MAP_OPTIONS` etc. + */ + def runPerfFlames(nodes: RoleName*)(delay: FiniteDuration, time: FiniteDuration = 15.seconds): Unit = { + if (isPerfJavaFlamesAvailable && isNode(nodes: _*)) { + import scala.concurrent.ExecutionContext.Implicits.global + + val afterDelay = akka.pattern.after(delay, system.scheduler)(Future.successful("GO!")) + afterDelay onComplete { it ⇒ + import java.lang.management._ + val name = ManagementFactory.getRuntimeMXBean.getName + val pid = name.substring(0, name.indexOf('@')).toInt + + val perfCommand = s"$perfJavaFlamesPath $pid" + println(s"[perf @ $myself($pid)][OUT]: " + perfCommand) + + import scala.sys.process._ + perfCommand.run(new ProcessLogger { + override def buffer[T](f: ⇒ T): T = f + override def out(s: ⇒ String): Unit = println(s"[perf @ $myself($pid)][OUT] " + s) + override def err(s: ⇒ String): Unit = println(s"[perf @ $myself($pid)][ERR] " + s) + }) + } + } + } + + def perfJavaFlamesPath: String = + "/home/ubuntu/perf-java-flames" + + def isPerfJavaFlamesAvailable: Boolean = { + val isIt = new File(perfJavaFlamesPath).exists() + if (!isIt) println(s"WARN: perf-java-flames not available under [$perfJavaFlamesPath]! Skipping perf profiling.") + isIt + } + +} diff --git a/akka-persistence-shared/src/test/scala/akka/persistence/journal/leveldb/PersistencePluginProxySpec.scala b/akka-persistence-shared/src/test/scala/akka/persistence/journal/leveldb/PersistencePluginProxySpec.scala index 6e53e17bdb..7e8e57d462 100644 --- a/akka-persistence-shared/src/test/scala/akka/persistence/journal/leveldb/PersistencePluginProxySpec.scala +++ b/akka-persistence-shared/src/test/scala/akka/persistence/journal/leveldb/PersistencePluginProxySpec.scala @@ -15,7 +15,7 @@ object PersistencePluginProxySpec { """ akka { actor { - provider = "akka.remote.RemoteActorRefProvider" + provider = remote } persistence { journal { diff --git a/akka-persistence-shared/src/test/scala/akka/persistence/journal/leveldb/SharedLeveldbJournalSpec.scala b/akka-persistence-shared/src/test/scala/akka/persistence/journal/leveldb/SharedLeveldbJournalSpec.scala index 308e4a067b..6d3609372f 100644 --- a/akka-persistence-shared/src/test/scala/akka/persistence/journal/leveldb/SharedLeveldbJournalSpec.scala +++ b/akka-persistence-shared/src/test/scala/akka/persistence/journal/leveldb/SharedLeveldbJournalSpec.scala @@ -13,7 +13,7 @@ object SharedLeveldbJournalSpec { """ akka { actor { - provider = "akka.remote.RemoteActorRefProvider" + provider = remote } persistence { journal { diff --git a/akka-persistence-shared/src/test/scala/akka/persistence/serialization/SerializerSpec.scala b/akka-persistence-shared/src/test/scala/akka/persistence/serialization/SerializerSpec.scala index 7e9ef28bde..8a2bfffa2f 100644 --- a/akka-persistence-shared/src/test/scala/akka/persistence/serialization/SerializerSpec.scala +++ b/akka-persistence-shared/src/test/scala/akka/persistence/serialization/SerializerSpec.scala @@ -44,7 +44,7 @@ object SerializerSpecConfigs { """ akka { actor { - provider = "akka.remote.RemoteActorRefProvider" + provider = remote } remote { enabled-transports = ["akka.remote.netty.tcp"] diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/AttemptSysMsgRedeliverySpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/AttemptSysMsgRedeliverySpec.scala index f39fd2e4bc..ba801dd1b7 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/AttemptSysMsgRedeliverySpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/AttemptSysMsgRedeliverySpec.scala @@ -13,17 +13,38 @@ import akka.remote.transport.ThrottlerTransportAdapter.Direction import akka.testkit._ import testkit.{ STMultiNodeSpec, MultiNodeConfig, MultiNodeSpec } import akka.actor.PoisonPill +import com.typesafe.config.ConfigFactory -object AttemptSysMsgRedeliveryMultiJvmSpec extends MultiNodeConfig { +class AttemptSysMsgRedeliveryMultiJvmSpec(artery: Boolean) extends MultiNodeConfig { val first = role("first") val second = role("second") val third = role("third") - commonConfig(debugConfig(on = false)) + commonConfig(debugConfig(on = false).withFallback( + ConfigFactory.parseString(s""" + akka.remote.artery.enabled = $artery + """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf)) testTransport(on = true) +} + +class AttemptSysMsgRedeliveryMultiJvmNode1 extends AttemptSysMsgRedeliverySpec( + new AttemptSysMsgRedeliveryMultiJvmSpec(artery = false)) +class AttemptSysMsgRedeliveryMultiJvmNode2 extends AttemptSysMsgRedeliverySpec( + new AttemptSysMsgRedeliveryMultiJvmSpec(artery = false)) +class AttemptSysMsgRedeliveryMultiJvmNode3 extends AttemptSysMsgRedeliverySpec( + new AttemptSysMsgRedeliveryMultiJvmSpec(artery = false)) + +class ArteryAttemptSysMsgRedeliveryMultiJvmNode1 extends AttemptSysMsgRedeliverySpec( + new AttemptSysMsgRedeliveryMultiJvmSpec(artery = true)) +class ArteryAttemptSysMsgRedeliveryMultiJvmNode2 extends AttemptSysMsgRedeliverySpec( + new AttemptSysMsgRedeliveryMultiJvmSpec(artery = true)) +class ArteryAttemptSysMsgRedeliveryMultiJvmNode3 extends AttemptSysMsgRedeliverySpec( + new AttemptSysMsgRedeliveryMultiJvmSpec(artery = true)) + +object AttemptSysMsgRedeliverySpec { class Echo extends Actor { def receive = { case m ⇒ sender ! m @@ -31,13 +52,10 @@ object AttemptSysMsgRedeliveryMultiJvmSpec extends MultiNodeConfig { } } -class AttemptSysMsgRedeliveryMultiJvmNode1 extends AttemptSysMsgRedeliverySpec -class AttemptSysMsgRedeliveryMultiJvmNode2 extends AttemptSysMsgRedeliverySpec -class AttemptSysMsgRedeliveryMultiJvmNode3 extends AttemptSysMsgRedeliverySpec - -class AttemptSysMsgRedeliverySpec extends MultiNodeSpec(AttemptSysMsgRedeliveryMultiJvmSpec) - with STMultiNodeSpec with ImplicitSender with DefaultTimeout { - import AttemptSysMsgRedeliveryMultiJvmSpec._ +abstract class AttemptSysMsgRedeliverySpec(multiNodeConfig: AttemptSysMsgRedeliveryMultiJvmSpec) + extends RemotingMultiNodeSpec(multiNodeConfig) { + import multiNodeConfig._ + import AttemptSysMsgRedeliverySpec._ def initialParticipants = roles.size diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/LookupRemoteActorSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/LookupRemoteActorSpec.scala index fd31448d07..b5b5d23726 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/LookupRemoteActorSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/LookupRemoteActorSpec.scala @@ -11,28 +11,38 @@ import testkit.{ STMultiNodeSpec, MultiNodeConfig, MultiNodeSpec } import akka.testkit._ import akka.actor.Identify import akka.actor.ActorIdentity +import com.typesafe.config.ConfigFactory -object LookupRemoteActorMultiJvmSpec extends MultiNodeConfig { +class LookupRemoteActorMultiJvmSpec(artery: Boolean) extends MultiNodeConfig { - class SomeActor extends Actor { - def receive = { - case "identify" ⇒ sender() ! self - } - } - - commonConfig(debugConfig(on = false)) + commonConfig(debugConfig(on = false).withFallback( + ConfigFactory.parseString(s""" + akka.remote.artery.enabled = $artery + """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf)) val master = role("master") val slave = role("slave") } -class LookupRemoteActorMultiJvmNode1 extends LookupRemoteActorSpec -class LookupRemoteActorMultiJvmNode2 extends LookupRemoteActorSpec +class LookupRemoteActorMultiJvmNode1 extends LookupRemoteActorSpec(new LookupRemoteActorMultiJvmSpec(artery = false)) +class LookupRemoteActorMultiJvmNode2 extends LookupRemoteActorSpec(new LookupRemoteActorMultiJvmSpec(artery = false)) -class LookupRemoteActorSpec extends MultiNodeSpec(LookupRemoteActorMultiJvmSpec) - with STMultiNodeSpec with ImplicitSender with DefaultTimeout { - import LookupRemoteActorMultiJvmSpec._ +class ArteryLookupRemoteActorMultiJvmNode1 extends LookupRemoteActorSpec(new LookupRemoteActorMultiJvmSpec(artery = true)) +class ArteryLookupRemoteActorMultiJvmNode2 extends LookupRemoteActorSpec(new LookupRemoteActorMultiJvmSpec(artery = true)) + +object LookupRemoteActorSpec { + class SomeActor extends Actor { + def receive = { + case "identify" ⇒ sender() ! self + } + } +} + +abstract class LookupRemoteActorSpec(multiNodeConfig: LookupRemoteActorMultiJvmSpec) + extends RemotingMultiNodeSpec(multiNodeConfig) { + import multiNodeConfig._ + import LookupRemoteActorSpec._ def initialParticipants = 2 diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/NewRemoteActorSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/NewRemoteActorSpec.scala index 3f59d73ce3..a1e76b8ae7 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/NewRemoteActorSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/NewRemoteActorSpec.scala @@ -14,22 +14,13 @@ import akka.testkit._ import com.typesafe.config.ConfigFactory import scala.concurrent.duration._ -object NewRemoteActorMultiJvmSpec extends MultiNodeConfig { - - class SomeActor extends Actor { - def receive = { - case "identify" ⇒ sender() ! self - } - } - - class SomeActorWithParam(ignored: String) extends Actor { - def receive = { - case "identify" ⇒ sender() ! self - } - } +class NewRemoteActorMultiJvmSpec(artery: Boolean) extends MultiNodeConfig { commonConfig(debugConfig(on = false).withFallback( - ConfigFactory.parseString("akka.remote.log-remote-lifecycle-events = off"))) + ConfigFactory.parseString(s""" + akka.remote.log-remote-lifecycle-events = off + akka.remote.artery.enabled = $artery + """).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf))) val master = role("master") val slave = role("slave") @@ -43,12 +34,30 @@ object NewRemoteActorMultiJvmSpec extends MultiNodeConfig { deployOnAll("""/service-hello2.remote = "@slave@" """) } -class NewRemoteActorMultiJvmNode1 extends NewRemoteActorSpec -class NewRemoteActorMultiJvmNode2 extends NewRemoteActorSpec +class NewRemoteActorMultiJvmNode1 extends NewRemoteActorSpec(new NewRemoteActorMultiJvmSpec(artery = false)) +class NewRemoteActorMultiJvmNode2 extends NewRemoteActorSpec(new NewRemoteActorMultiJvmSpec(artery = false)) -class NewRemoteActorSpec extends MultiNodeSpec(NewRemoteActorMultiJvmSpec) - with STMultiNodeSpec with ImplicitSender with DefaultTimeout { - import NewRemoteActorMultiJvmSpec._ +class ArteryNewRemoteActorMultiJvmNode1 extends NewRemoteActorSpec(new NewRemoteActorMultiJvmSpec(artery = true)) +class ArteryNewRemoteActorMultiJvmNode2 extends NewRemoteActorSpec(new NewRemoteActorMultiJvmSpec(artery = true)) + +object NewRemoteActorSpec { + class SomeActor extends Actor { + def receive = { + case "identify" ⇒ sender() ! self + } + } + + class SomeActorWithParam(ignored: String) extends Actor { + def receive = { + case "identify" ⇒ sender() ! self + } + } +} + +abstract class NewRemoteActorSpec(multiNodeConfig: NewRemoteActorMultiJvmSpec) + extends RemotingMultiNodeSpec(multiNodeConfig) { + import multiNodeConfig._ + import NewRemoteActorSpec._ def initialParticipants = roles.size diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/PiercingShouldKeepQuarantineSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/PiercingShouldKeepQuarantineSpec.scala index c9ee08dacc..2ce5a016ae 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/PiercingShouldKeepQuarantineSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/PiercingShouldKeepQuarantineSpec.scala @@ -7,32 +7,39 @@ import akka.testkit._ import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec } import akka.remote.testconductor.RoleName -object PiercingShouldKeepQuarantineSpec extends MultiNodeConfig { +class PiercingShouldKeepQuarantineConfig(artery: Boolean) extends MultiNodeConfig { val first = role("first") val second = role("second") commonConfig(debugConfig(on = false).withFallback( - ConfigFactory.parseString(""" - #akka.loglevel = INFO - #akka.remote.log-remote-lifecycle-events = INFO + ConfigFactory.parseString(s""" akka.remote.retry-gate-closed-for = 0.5s - """))) - - class Subject extends Actor { - def receive = { - case "getuid" ⇒ sender() ! AddressUidExtension(context.system).addressUid - } - } + akka.remote.artery.enabled = $artery + """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf)) } -class PiercingShouldKeepQuarantineSpecMultiJvmNode1 extends PiercingShouldKeepQuarantineSpec -class PiercingShouldKeepQuarantineSpecMultiJvmNode2 extends PiercingShouldKeepQuarantineSpec +class PiercingShouldKeepQuarantineSpecMultiJvmNode1 extends PiercingShouldKeepQuarantineSpec( + new PiercingShouldKeepQuarantineConfig(artery = false)) +class PiercingShouldKeepQuarantineSpecMultiJvmNode2 extends PiercingShouldKeepQuarantineSpec( + new PiercingShouldKeepQuarantineConfig(artery = false)) -abstract class PiercingShouldKeepQuarantineSpec extends MultiNodeSpec(PiercingShouldKeepQuarantineSpec) - with STMultiNodeSpec - with ImplicitSender { +class ArteryPiercingShouldKeepQuarantineSpecMultiJvmNode1 extends PiercingShouldKeepQuarantineSpec( + new PiercingShouldKeepQuarantineConfig(artery = true)) +class ArteryPiercingShouldKeepQuarantineSpecMultiJvmNode2 extends PiercingShouldKeepQuarantineSpec( + new PiercingShouldKeepQuarantineConfig(artery = true)) +object PiercingShouldKeepQuarantineSpec { + class Subject extends Actor { + def receive = { + case "getuid" ⇒ sender() ! AddressUidExtension(context.system).longAddressUid + } + } +} + +abstract class PiercingShouldKeepQuarantineSpec(multiNodeConfig: PiercingShouldKeepQuarantineConfig) + extends RemotingMultiNodeSpec(multiNodeConfig) { + import multiNodeConfig._ import PiercingShouldKeepQuarantineSpec._ override def initialParticipants = roles.size @@ -45,11 +52,11 @@ abstract class PiercingShouldKeepQuarantineSpec extends MultiNodeSpec(PiercingSh // Communicate with second system system.actorSelection(node(second) / "user" / "subject") ! "getuid" - val uid = expectMsgType[Int](10.seconds) + val uid = expectMsgType[Long](10.seconds) enterBarrier("actor-identified") // Manually Quarantine the other system - RARP(system).provider.transport.quarantine(node(second).address, Some(uid)) + RARP(system).provider.transport.quarantine(node(second).address, Some(uid), "test") // Quarantining is not immediate Thread.sleep(1000) diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteDeliverySpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteDeliverySpec.scala index c60ac55204..36bc57e4fb 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteDeliverySpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteDeliverySpec.scala @@ -18,13 +18,26 @@ import akka.testkit._ import akka.actor.ActorIdentity import akka.actor.Identify -object RemoteDeliveryMultiJvmSpec extends MultiNodeConfig { +class RemoteDeliveryConfig(artery: Boolean) extends MultiNodeConfig { val first = role("first") val second = role("second") val third = role("third") - commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString("akka.loglevel=INFO"))) + commonConfig(debugConfig(on = false).withFallback( + ConfigFactory.parseString(s""" + akka.remote.artery.enabled = $artery + """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf)) +} +class RemoteDeliveryMultiJvmNode1 extends RemoteDeliverySpec(new RemoteDeliveryConfig(artery = false)) +class RemoteDeliveryMultiJvmNode2 extends RemoteDeliverySpec(new RemoteDeliveryConfig(artery = false)) +class RemoteDeliveryMultiJvmNode3 extends RemoteDeliverySpec(new RemoteDeliveryConfig(artery = false)) + +class ArteryRemoteDeliveryMultiJvmNode1 extends RemoteDeliverySpec(new RemoteDeliveryConfig(artery = true)) +class ArteryRemoteDeliveryMultiJvmNode2 extends RemoteDeliverySpec(new RemoteDeliveryConfig(artery = true)) +class ArteryRemoteDeliveryMultiJvmNode3 extends RemoteDeliverySpec(new RemoteDeliveryConfig(artery = true)) + +object RemoteDeliverySpec { final case class Letter(n: Int, route: List[ActorRef]) class Postman extends Actor { @@ -32,18 +45,12 @@ object RemoteDeliveryMultiJvmSpec extends MultiNodeConfig { case Letter(n, route) ⇒ route.head ! Letter(n, route.tail) } } - } -class RemoteDeliveryMultiJvmNode1 extends RemoteDeliverySpec -class RemoteDeliveryMultiJvmNode2 extends RemoteDeliverySpec -class RemoteDeliveryMultiJvmNode3 extends RemoteDeliverySpec - -abstract class RemoteDeliverySpec - extends MultiNodeSpec(RemoteDeliveryMultiJvmSpec) - with STMultiNodeSpec with ImplicitSender { - - import RemoteDeliveryMultiJvmSpec._ +abstract class RemoteDeliverySpec(multiNodeConfig: RemoteDeliveryConfig) + extends RemotingMultiNodeSpec(multiNodeConfig) { + import multiNodeConfig._ + import RemoteDeliverySpec._ override def initialParticipants = roles.size @@ -52,7 +59,7 @@ abstract class RemoteDeliverySpec expectMsgType[ActorIdentity].ref.get } - "Remoting with TCP" must { + "Remote message delivery" must { "not drop messages under normal circumstances" in { system.actorOf(Props[Postman], "postman-" + myself.name) diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteDeploymentDeathWatchSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteDeploymentDeathWatchSpec.scala index bffed713b8..bc4e23f86d 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteDeploymentDeathWatchSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteDeploymentDeathWatchSpec.scala @@ -17,46 +17,61 @@ import akka.remote.testkit.STMultiNodeSpec import akka.testkit._ import akka.testkit.TestEvent._ -object RemoteDeploymentDeathWatchMultiJvmSpec extends MultiNodeConfig { +class RemoteDeploymentDeathWatchMultiJvmSpec(artery: Boolean) extends MultiNodeConfig { val first = role("first") val second = role("second") val third = role("third") commonConfig(debugConfig(on = false).withFallback( - ConfigFactory.parseString(""" + ConfigFactory.parseString(s""" akka.loglevel = INFO akka.remote.log-remote-lifecycle-events = off - """))) + akka.remote.artery.enabled = $artery + """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf)) deployOn(second, """/hello.remote = "@third@" """) +} + +// Several different variations of the test + +class RemoteDeploymentDeathWatchFastMultiJvmNode1 extends RemoteDeploymentNodeDeathWatchFastSpec(artery = false) +class RemoteDeploymentDeathWatchFastMultiJvmNode2 extends RemoteDeploymentNodeDeathWatchFastSpec(artery = false) +class RemoteDeploymentDeathWatchFastMultiJvmNode3 extends RemoteDeploymentNodeDeathWatchFastSpec(artery = false) + +class ArteryRemoteDeploymentDeathWatchFastMultiJvmNode1 extends RemoteDeploymentNodeDeathWatchFastSpec(artery = true) +class ArteryRemoteDeploymentDeathWatchFastMultiJvmNode2 extends RemoteDeploymentNodeDeathWatchFastSpec(artery = true) +class ArteryRemoteDeploymentDeathWatchFastMultiJvmNode3 extends RemoteDeploymentNodeDeathWatchFastSpec(artery = true) + +abstract class RemoteDeploymentNodeDeathWatchFastSpec(artery: Boolean) extends RemoteDeploymentDeathWatchSpec( + new RemoteDeploymentDeathWatchMultiJvmSpec(artery)) { + override def scenario = "fast" +} + +class RemoteDeploymentDeathWatchSlowMultiJvmNode1 extends RemoteDeploymentNodeDeathWatchSlowSpec(artery = false) +class RemoteDeploymentDeathWatchSlowMultiJvmNode2 extends RemoteDeploymentNodeDeathWatchSlowSpec(artery = false) +class RemoteDeploymentDeathWatchSlowMultiJvmNode3 extends RemoteDeploymentNodeDeathWatchSlowSpec(artery = false) + +class ArteryRemoteDeploymentDeathWatchSlowMultiJvmNode1 extends RemoteDeploymentNodeDeathWatchSlowSpec(artery = true) +class ArteryRemoteDeploymentDeathWatchSlowMultiJvmNode2 extends RemoteDeploymentNodeDeathWatchSlowSpec(artery = true) +class ArteryRemoteDeploymentDeathWatchSlowMultiJvmNode3 extends RemoteDeploymentNodeDeathWatchSlowSpec(artery = true) + +abstract class RemoteDeploymentNodeDeathWatchSlowSpec(artery: Boolean) extends RemoteDeploymentDeathWatchSpec( + new RemoteDeploymentDeathWatchMultiJvmSpec(artery)) { + override def scenario = "slow" + override def sleep(): Unit = Thread.sleep(3000) +} + +object RemoteDeploymentDeathWatchSpec { class Hello extends Actor { def receive = Actor.emptyBehavior } } -// Several different variations of the test - -class RemoteDeploymentDeathWatchFastMultiJvmNode1 extends RemoteDeploymentNodeDeathWatchFastSpec -class RemoteDeploymentDeathWatchFastMultiJvmNode2 extends RemoteDeploymentNodeDeathWatchFastSpec -class RemoteDeploymentDeathWatchFastMultiJvmNode3 extends RemoteDeploymentNodeDeathWatchFastSpec -abstract class RemoteDeploymentNodeDeathWatchFastSpec extends RemoteDeploymentDeathWatchSpec { - override def scenario = "fast" -} - -class RemoteDeploymentDeathWatchSlowMultiJvmNode1 extends RemoteDeploymentNodeDeathWatchSlowSpec -class RemoteDeploymentDeathWatchSlowMultiJvmNode2 extends RemoteDeploymentNodeDeathWatchSlowSpec -class RemoteDeploymentDeathWatchSlowMultiJvmNode3 extends RemoteDeploymentNodeDeathWatchSlowSpec -abstract class RemoteDeploymentNodeDeathWatchSlowSpec extends RemoteDeploymentDeathWatchSpec { - override def scenario = "slow" - override def sleep(): Unit = Thread.sleep(3000) -} - -abstract class RemoteDeploymentDeathWatchSpec - extends MultiNodeSpec(RemoteDeploymentDeathWatchMultiJvmSpec) - with STMultiNodeSpec with ImplicitSender { - - import RemoteDeploymentDeathWatchMultiJvmSpec._ +abstract class RemoteDeploymentDeathWatchSpec(multiNodeConfig: RemoteDeploymentDeathWatchMultiJvmSpec) + extends RemotingMultiNodeSpec(multiNodeConfig) { + import multiNodeConfig._ + import RemoteDeploymentDeathWatchSpec._ def scenario: String // Possible to override to let them heartbeat for a while. diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteGatePiercingSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteGatePiercingSpec.scala index 9492cfc0cd..deb454dce7 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteGatePiercingSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteGatePiercingSpec.scala @@ -29,7 +29,7 @@ object RemoteGatePiercingSpec extends MultiNodeConfig { akka.loglevel = INFO akka.remote.log-remote-lifecycle-events = INFO akka.remote.transport-failure-detector.acceptable-heartbeat-pause = 5 s - """))) + """))) nodeConfig(first)( ConfigFactory.parseString("akka.remote.retry-gate-closed-for = 1 d # Keep it long")) @@ -51,8 +51,7 @@ class RemoteGatePiercingSpecMultiJvmNode1 extends RemoteGatePiercingSpec class RemoteGatePiercingSpecMultiJvmNode2 extends RemoteGatePiercingSpec abstract class RemoteGatePiercingSpec - extends MultiNodeSpec(RemoteGatePiercingSpec) - with STMultiNodeSpec with ImplicitSender { + extends RemotingMultiNodeSpec(RemoteGatePiercingSpec) { import RemoteGatePiercingSpec._ diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeDeathWatchSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeDeathWatchSpec.scala index be2c79dd61..e71eefaf8d 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeDeathWatchSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeDeathWatchSpec.scala @@ -5,7 +5,7 @@ package akka.remote import language.postfixOps import scala.concurrent.duration._ -import com.typesafe.config.ConfigFactory +import com.typesafe.config.{ Config, ConfigFactory } import akka.actor.Actor import akka.actor.ActorIdentity import akka.actor.ActorRef @@ -19,19 +19,52 @@ import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.STMultiNodeSpec import akka.testkit._ -object RemoteNodeDeathWatchMultiJvmSpec extends MultiNodeConfig { +class RemoteNodeDeathWatchConfig(artery: Boolean) extends MultiNodeConfig { val first = role("first") val second = role("second") val third = role("third") commonConfig(debugConfig(on = false).withFallback( - ConfigFactory.parseString(""" + ConfigFactory.parseString(s""" akka.loglevel = INFO akka.remote.log-remote-lifecycle-events = off ## Use a tighter setting than the default, otherwise it takes 20s for DeathWatch to trigger akka.remote.watch-failure-detector.acceptable-heartbeat-pause = 3 s - """))) + akka.remote.artery.enabled = $artery + """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf)) +} + +// Several different variations of the test + +class RemoteNodeDeathWatchFastMultiJvmNode1 extends RemoteNodeDeathWatchFastSpec(artery = false) +class RemoteNodeDeathWatchFastMultiJvmNode2 extends RemoteNodeDeathWatchFastSpec(artery = false) +class RemoteNodeDeathWatchFastMultiJvmNode3 extends RemoteNodeDeathWatchFastSpec(artery = false) + +class ArteryRemoteNodeDeathWatchFastMultiJvmNode1 extends RemoteNodeDeathWatchFastSpec(artery = true) +class ArteryRemoteNodeDeathWatchFastMultiJvmNode2 extends RemoteNodeDeathWatchFastSpec(artery = true) +class ArteryRemoteNodeDeathWatchFastMultiJvmNode3 extends RemoteNodeDeathWatchFastSpec(artery = true) + +abstract class RemoteNodeDeathWatchFastSpec(artery: Boolean) extends RemoteNodeDeathWatchSpec( + new RemoteNodeDeathWatchConfig(artery)) { + override def scenario = "fast" +} + +class RemoteNodeDeathWatchSlowMultiJvmNode1 extends RemoteNodeDeathWatchSlowSpec(artery = false) +class RemoteNodeDeathWatchSlowMultiJvmNode2 extends RemoteNodeDeathWatchSlowSpec(artery = false) +class RemoteNodeDeathWatchSlowMultiJvmNode3 extends RemoteNodeDeathWatchSlowSpec(artery = false) + +class ArteryRemoteNodeDeathWatchSlowMultiJvmNode1 extends RemoteNodeDeathWatchSlowSpec(artery = true) +class ArteryRemoteNodeDeathWatchSlowMultiJvmNode2 extends RemoteNodeDeathWatchSlowSpec(artery = true) +class ArteryRemoteNodeDeathWatchSlowMultiJvmNode3 extends RemoteNodeDeathWatchSlowSpec(artery = true) + +abstract class RemoteNodeDeathWatchSlowSpec(artery: Boolean) extends RemoteNodeDeathWatchSpec( + new RemoteNodeDeathWatchConfig(artery)) { + override def scenario = "slow" + override def sleep(): Unit = Thread.sleep(3000) +} + +object RemoteNodeDeathWatchSpec { final case class WatchIt(watchee: ActorRef) final case class UnwatchIt(watchee: ActorRef) case object Ack @@ -58,28 +91,10 @@ object RemoteNodeDeathWatchMultiJvmSpec extends MultiNodeConfig { } -// Several different variations of the test - -class RemoteNodeDeathWatchFastMultiJvmNode1 extends RemoteNodeDeathWatchFastSpec -class RemoteNodeDeathWatchFastMultiJvmNode2 extends RemoteNodeDeathWatchFastSpec -class RemoteNodeDeathWatchFastMultiJvmNode3 extends RemoteNodeDeathWatchFastSpec -abstract class RemoteNodeDeathWatchFastSpec extends RemoteNodeDeathWatchSpec { - override def scenario = "fast" -} - -class RemoteNodeDeathWatchSlowMultiJvmNode1 extends RemoteNodeDeathWatchSlowSpec -class RemoteNodeDeathWatchSlowMultiJvmNode2 extends RemoteNodeDeathWatchSlowSpec -class RemoteNodeDeathWatchSlowMultiJvmNode3 extends RemoteNodeDeathWatchSlowSpec -abstract class RemoteNodeDeathWatchSlowSpec extends RemoteNodeDeathWatchSpec { - override def scenario = "slow" - override def sleep(): Unit = Thread.sleep(3000) -} - -abstract class RemoteNodeDeathWatchSpec - extends MultiNodeSpec(RemoteNodeDeathWatchMultiJvmSpec) - with STMultiNodeSpec with ImplicitSender { - - import RemoteNodeDeathWatchMultiJvmSpec._ +abstract class RemoteNodeDeathWatchSpec(multiNodeConfig: RemoteNodeDeathWatchConfig) + extends RemotingMultiNodeSpec(multiNodeConfig) { + import multiNodeConfig._ + import RemoteNodeDeathWatchSpec._ import RemoteWatcher._ def scenario: String diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeRestartDeathWatchSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeRestartDeathWatchSpec.scala index b84753e156..43bcbd2af5 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeRestartDeathWatchSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeRestartDeathWatchSpec.scala @@ -23,20 +23,35 @@ import akka.actor.ExtendedActorSystem import akka.actor.ActorSystem import akka.actor.RootActorPath -object RemoteNodeRestartDeathWatchMultiJvmSpec extends MultiNodeConfig { +class RemoteNodeRestartDeathWatchConfig(artery: Boolean) extends MultiNodeConfig { val first = role("first") val second = role("second") commonConfig(debugConfig(on = false).withFallback( - ConfigFactory.parseString(""" + ConfigFactory.parseString(s""" akka.loglevel = INFO akka.remote.log-remote-lifecycle-events = off akka.remote.transport-failure-detector.heartbeat-interval = 1 s akka.remote.transport-failure-detector.acceptable-heartbeat-pause = 3 s + akka.remote.artery.enabled = $artery """))) testTransport(on = true) +} + +class RemoteNodeRestartDeathWatchMultiJvmNode1 extends RemoteNodeRestartDeathWatchSpec( + new RemoteNodeRestartDeathWatchConfig(artery = false)) +class RemoteNodeRestartDeathWatchMultiJvmNode2 extends RemoteNodeRestartDeathWatchSpec( + new RemoteNodeRestartDeathWatchConfig(artery = false)) + +// FIXME this is failing with Artery +//class ArteryRemoteNodeRestartDeathWatchMultiJvmNode1 extends RemoteNodeRestartDeathWatchSpec( +// new RemoteNodeRestartDeathWatchConfig(artery = true)) +//class ArteryRemoteNodeRestartDeathWatchMultiJvmNode2 extends RemoteNodeRestartDeathWatchSpec( +// new RemoteNodeRestartDeathWatchConfig(artery = true)) + +object RemoteNodeRestartDeathWatchSpec { class Subject extends Actor { def receive = { case "shutdown" ⇒ @@ -45,19 +60,12 @@ object RemoteNodeRestartDeathWatchMultiJvmSpec extends MultiNodeConfig { case msg ⇒ sender() ! msg } } - } -// Several different variations of the test - -class RemoteNodeRestartDeathWatchMultiJvmNode1 extends RemoteNodeRestartDeathWatchSpec -class RemoteNodeRestartDeathWatchMultiJvmNode2 extends RemoteNodeRestartDeathWatchSpec - -abstract class RemoteNodeRestartDeathWatchSpec - extends MultiNodeSpec(RemoteNodeRestartDeathWatchMultiJvmSpec) - with STMultiNodeSpec with ImplicitSender { - - import RemoteNodeRestartDeathWatchMultiJvmSpec._ +abstract class RemoteNodeRestartDeathWatchSpec(multiNodeConfig: RemoteNodeRestartDeathWatchConfig) + extends RemotingMultiNodeSpec(multiNodeConfig) { + import multiNodeConfig._ + import RemoteNodeRestartDeathWatchSpec._ override def initialParticipants = roles.size @@ -104,11 +112,9 @@ abstract class RemoteNodeRestartDeathWatchSpec Await.ready(system.whenTerminated, 30.seconds) val freshSystem = ActorSystem(system.name, ConfigFactory.parseString(s""" - akka.remote.netty.tcp { - hostname = ${addr.host.get} - port = ${addr.port.get} - } - """).withFallback(system.settings.config)) + akka.remote.netty.tcp.port = ${addr.port.get} + akka.remote.artery.canonical.port = ${addr.port.get} + """).withFallback(system.settings.config)) freshSystem.actorOf(Props[Subject], "subject") Await.ready(freshSystem.whenTerminated, 30.seconds) diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeRestartGateSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeRestartGateSpec.scala index 08da06b2c7..00232f85ac 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeRestartGateSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeRestartGateSpec.scala @@ -46,8 +46,7 @@ class RemoteNodeRestartGateSpecMultiJvmNode1 extends RemoteNodeRestartGateSpec class RemoteNodeRestartGateSpecMultiJvmNode2 extends RemoteNodeRestartGateSpec abstract class RemoteNodeRestartGateSpec - extends MultiNodeSpec(RemoteNodeRestartGateSpec) - with STMultiNodeSpec with ImplicitSender { + extends RemotingMultiNodeSpec(RemoteNodeRestartGateSpec) { import RemoteNodeRestartGateSpec._ diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeShutdownAndComesBackSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeShutdownAndComesBackSpec.scala index b321462305..3906efb013 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeShutdownAndComesBackSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeShutdownAndComesBackSpec.scala @@ -30,7 +30,7 @@ object RemoteNodeShutdownAndComesBackSpec extends MultiNodeConfig { akka.remote.transport-failure-detector.heartbeat-interval = 1 s akka.remote.transport-failure-detector.acceptable-heartbeat-pause = 3 s akka.remote.watch-failure-detector.acceptable-heartbeat-pause = 60 s - """))) + """))) testTransport(on = true) @@ -47,8 +47,7 @@ class RemoteNodeShutdownAndComesBackMultiJvmNode1 extends RemoteNodeShutdownAndC class RemoteNodeShutdownAndComesBackMultiJvmNode2 extends RemoteNodeShutdownAndComesBackSpec abstract class RemoteNodeShutdownAndComesBackSpec - extends MultiNodeSpec(RemoteNodeShutdownAndComesBackSpec) - with STMultiNodeSpec with ImplicitSender { + extends RemotingMultiNodeSpec(RemoteNodeShutdownAndComesBackSpec) { import RemoteNodeShutdownAndComesBackSpec._ @@ -135,11 +134,9 @@ abstract class RemoteNodeShutdownAndComesBackSpec Await.ready(system.whenTerminated, 30.seconds) val freshSystem = ActorSystem(system.name, ConfigFactory.parseString(s""" - akka.remote.netty.tcp { - hostname = ${addr.host.get} - port = ${addr.port.get} - } - """).withFallback(system.settings.config)) + akka.remote.netty.tcp.port = ${addr.port.get} + akka.remote.artery.canonical.port = ${addr.port.get} + """).withFallback(system.settings.config)) freshSystem.actorOf(Props[Subject], "subject") Await.ready(freshSystem.whenTerminated, 30.seconds) diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteQuarantinePiercingSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteQuarantinePiercingSpec.scala index 1683bba4ba..e4d88b7fe1 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteQuarantinePiercingSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteQuarantinePiercingSpec.scala @@ -18,39 +18,50 @@ import akka.remote.testconductor.RoleName import akka.actor.Identify import scala.concurrent.Await -object RemoteQuarantinePiercingSpec extends MultiNodeConfig { +class RemoteQuarantinePiercingConfig(artery: Boolean) extends MultiNodeConfig { val first = role("first") val second = role("second") commonConfig(debugConfig(on = false).withFallback( - ConfigFactory.parseString(""" + ConfigFactory.parseString(s""" akka.loglevel = INFO akka.remote.log-remote-lifecycle-events = INFO - """))) - - class Subject extends Actor { - def receive = { - case "shutdown" ⇒ context.system.terminate() - case "identify" ⇒ sender() ! (AddressUidExtension(context.system).addressUid → self) - } - } + akka.remote.artery.enabled = $artery + """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf)) } -class RemoteQuarantinePiercingMultiJvmNode1 extends RemoteQuarantinePiercingSpec -class RemoteQuarantinePiercingMultiJvmNode2 extends RemoteQuarantinePiercingSpec +class RemoteQuarantinePiercingMultiJvmNode1 extends RemoteQuarantinePiercingSpec( + new RemoteQuarantinePiercingConfig(artery = false)) +class RemoteQuarantinePiercingMultiJvmNode2 extends RemoteQuarantinePiercingSpec( + new RemoteQuarantinePiercingConfig(artery = false)) -abstract class RemoteQuarantinePiercingSpec extends MultiNodeSpec(RemoteQuarantinePiercingSpec) - with STMultiNodeSpec - with ImplicitSender { +class ArteryRemoteQuarantinePiercingMultiJvmNode1 extends RemoteQuarantinePiercingSpec( + new RemoteQuarantinePiercingConfig(artery = true)) +class ArteryRemoteQuarantinePiercingMultiJvmNode2 extends RemoteQuarantinePiercingSpec( + new RemoteQuarantinePiercingConfig(artery = true)) +object RemoteQuarantinePiercingSpec { + class Subject extends Actor { + def receive = { + case "shutdown" ⇒ context.system.terminate() + case "identify" ⇒ sender() ! (AddressUidExtension(context.system).longAddressUid → self) + } + } +} + +abstract class RemoteQuarantinePiercingSpec(multiNodeConfig: RemoteQuarantinePiercingConfig) + extends RemotingMultiNodeSpec(multiNodeConfig) { + import multiNodeConfig._ import RemoteQuarantinePiercingSpec._ override def initialParticipants = roles.size - def identify(role: RoleName, actorName: String): (Int, ActorRef) = { - system.actorSelection(node(role) / "user" / actorName) ! "identify" - expectMsgType[(Int, ActorRef)] + def identifyWithUid(role: RoleName, actorName: String, timeout: FiniteDuration = remainingOrDefault): (Long, ActorRef) = { + within(timeout) { + system.actorSelection(node(role) / "user" / actorName) ! "identify" + expectMsgType[(Long, ActorRef)] + } } "RemoteNodeShutdownAndComesBack" must { @@ -61,11 +72,11 @@ abstract class RemoteQuarantinePiercingSpec extends MultiNodeSpec(RemoteQuaranti enterBarrier("actors-started") // Acquire ActorRef from first system - val (uidFirst, subjectFirst) = identify(second, "subject") + val (uidFirst, subjectFirst) = identifyWithUid(second, "subject", 5.seconds) enterBarrier("actor-identified") // Manually Quarantine the other system - RARP(system).provider.transport.quarantine(node(second).address, Some(uidFirst)) + RARP(system).provider.transport.quarantine(node(second).address, Some(uidFirst), "test") // Quarantine is up -- Cannot communicate with remote system any more system.actorSelection(RootActorPath(secondAddress) / "user" / "subject") ! "identify" @@ -79,7 +90,7 @@ abstract class RemoteQuarantinePiercingSpec extends MultiNodeSpec(RemoteQuaranti // retry because the Subject actor might not be started yet awaitAssert { system.actorSelection(RootActorPath(secondAddress) / "user" / "subject") ! "identify" - val (uidSecond, subjectSecond) = expectMsgType[(Int, ActorRef)](1.second) + val (uidSecond, subjectSecond) = expectMsgType[(Long, ActorRef)](1.second) uidSecond should not be (uidFirst) subjectSecond should not be (subjectFirst) } @@ -101,11 +112,9 @@ abstract class RemoteQuarantinePiercingSpec extends MultiNodeSpec(RemoteQuaranti Await.ready(system.whenTerminated, 30.seconds) val freshSystem = ActorSystem(system.name, ConfigFactory.parseString(s""" - akka.remote.netty.tcp { - hostname = ${addr.host.get} - port = ${addr.port.get} - } - """).withFallback(system.settings.config)) + akka.remote.netty.tcp.port = ${addr.port.get} + akka.remote.artery.canonical.port = ${addr.port.get} + """).withFallback(system.settings.config)) freshSystem.actorOf(Props[Subject], "subject") Await.ready(freshSystem.whenTerminated, 30.seconds) diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteReDeploymentSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteReDeploymentSpec.scala index 06208299e0..29602a6d15 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteReDeploymentSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteReDeploymentSpec.scala @@ -19,12 +19,12 @@ import akka.actor.ActorLogging import akka.remote.testconductor.TestConductor import akka.testkit.TestProbe -object RemoteReDeploymentMultiJvmSpec extends MultiNodeConfig { +class RemoteReDeploymentConfig(artery: Boolean) extends MultiNodeConfig { val first = role("first") val second = role("second") commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString( - """akka.remote.transport-failure-detector { + s"""akka.remote.transport-failure-detector { threshold=0.1 heartbeat-interval=0.1s acceptable-heartbeat-pause=1s @@ -33,11 +33,52 @@ object RemoteReDeploymentMultiJvmSpec extends MultiNodeConfig { threshold=0.1 heartbeat-interval=0.1s acceptable-heartbeat-pause=2.5s - }"""))) + } + akka.remote.artery.enabled = $artery + """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf)) + testTransport(on = true) deployOn(second, "/parent/hello.remote = \"@first@\"") +} +class RemoteReDeploymentFastMultiJvmNode1 extends RemoteReDeploymentFastMultiJvmSpec(artery = false) +class RemoteReDeploymentFastMultiJvmNode2 extends RemoteReDeploymentFastMultiJvmSpec(artery = false) + +class ArteryRemoteReDeploymentFastMultiJvmNode1 extends RemoteReDeploymentFastMultiJvmSpec(artery = true) +class ArteryRemoteReDeploymentFastMultiJvmNode2 extends RemoteReDeploymentFastMultiJvmSpec(artery = true) + +abstract class RemoteReDeploymentFastMultiJvmSpec(artery: Boolean) extends RemoteReDeploymentMultiJvmSpec( + new RemoteReDeploymentConfig(artery)) { + override def sleepAfterKill = 0.seconds // new association will come in while old is still “healthy” + override def expectQuarantine = false +} + +class RemoteReDeploymentMediumMultiJvmNode1 extends RemoteReDeploymentMediumMultiJvmSpec(artery = false) +class RemoteReDeploymentMediumMultiJvmNode2 extends RemoteReDeploymentMediumMultiJvmSpec(artery = false) + +class ArteryRemoteReDeploymentMediumMultiJvmNode1 extends RemoteReDeploymentMediumMultiJvmSpec(artery = true) +class ArteryRemoteReDeploymentMediumMultiJvmNode2 extends RemoteReDeploymentMediumMultiJvmSpec(artery = true) + +abstract class RemoteReDeploymentMediumMultiJvmSpec(artery: Boolean) extends RemoteReDeploymentMultiJvmSpec( + new RemoteReDeploymentConfig(artery)) { + override def sleepAfterKill = 1.seconds // new association will come in while old is gated in ReliableDeliverySupervisor + override def expectQuarantine = false +} + +class RemoteReDeploymentSlowMultiJvmNode1 extends RemoteReDeploymentSlowMultiJvmSpec(artery = false) +class RemoteReDeploymentSlowMultiJvmNode2 extends RemoteReDeploymentSlowMultiJvmSpec(artery = false) + +class ArteryRemoteReDeploymentSlowMultiJvmNode1 extends RemoteReDeploymentSlowMultiJvmSpec(artery = true) +class ArteryRemoteReDeploymentSlowMultiJvmNode2 extends RemoteReDeploymentSlowMultiJvmSpec(artery = true) + +abstract class RemoteReDeploymentSlowMultiJvmSpec(artery: Boolean) extends RemoteReDeploymentMultiJvmSpec( + new RemoteReDeploymentConfig(artery)) { + override def sleepAfterKill = 10.seconds // new association will come in after old has been quarantined + override def expectQuarantine = true +} + +object RemoteReDeploymentMultiJvmSpec { class Parent extends Actor { val monitor = context.actorSelection("/user/echo") def receive = { @@ -64,35 +105,15 @@ object RemoteReDeploymentMultiJvmSpec extends MultiNodeConfig { def echoProps(target: ActorRef) = Props(new Echo(target)) } -class RemoteReDeploymentFastMultiJvmNode1 extends RemoteReDeploymentFastMultiJvmSpec -class RemoteReDeploymentFastMultiJvmNode2 extends RemoteReDeploymentFastMultiJvmSpec -abstract class RemoteReDeploymentFastMultiJvmSpec extends RemoteReDeploymentMultiJvmSpec { - override def sleepAfterKill = 0.seconds // new association will come in while old is still “healthy” - override def expectQuarantine = false -} - -class RemoteReDeploymentMediumMultiJvmNode1 extends RemoteReDeploymentMediumMultiJvmSpec -class RemoteReDeploymentMediumMultiJvmNode2 extends RemoteReDeploymentMediumMultiJvmSpec -abstract class RemoteReDeploymentMediumMultiJvmSpec extends RemoteReDeploymentMultiJvmSpec { - override def sleepAfterKill = 1.seconds // new association will come in while old is gated in ReliableDeliverySupervisor - override def expectQuarantine = false -} - -class RemoteReDeploymentSlowMultiJvmNode1 extends RemoteReDeploymentSlowMultiJvmSpec -class RemoteReDeploymentSlowMultiJvmNode2 extends RemoteReDeploymentSlowMultiJvmSpec -abstract class RemoteReDeploymentSlowMultiJvmSpec extends RemoteReDeploymentMultiJvmSpec { - override def sleepAfterKill = 10.seconds // new association will come in after old has been quarantined - override def expectQuarantine = true -} - -abstract class RemoteReDeploymentMultiJvmSpec extends MultiNodeSpec(RemoteReDeploymentMultiJvmSpec) - with STMultiNodeSpec with ImplicitSender { +abstract class RemoteReDeploymentMultiJvmSpec(multiNodeConfig: RemoteReDeploymentConfig) + extends RemotingMultiNodeSpec(multiNodeConfig) { def sleepAfterKill: FiniteDuration def expectQuarantine: Boolean def initialParticipants = roles.size + import multiNodeConfig._ import RemoteReDeploymentMultiJvmSpec._ "A remote deployment target system" must { @@ -155,6 +176,9 @@ abstract class RemoteReDeploymentMultiJvmSpec extends MultiNodeSpec(RemoteReDepl expectNoMsg(1.second) + runOn(second) { + Await.result(sys.terminate(), 10.seconds) + } } } diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteRestartedQuarantinedSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteRestartedQuarantinedSpec.scala index 10837756f1..11c5ae726c 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteRestartedQuarantinedSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteRestartedQuarantinedSpec.scala @@ -56,8 +56,7 @@ class RemoteRestartedQuarantinedSpecMultiJvmNode1 extends RemoteRestartedQuarant class RemoteRestartedQuarantinedSpecMultiJvmNode2 extends RemoteRestartedQuarantinedSpec abstract class RemoteRestartedQuarantinedSpec - extends MultiNodeSpec(RemoteRestartedQuarantinedSpec) - with STMultiNodeSpec with ImplicitSender { + extends RemotingMultiNodeSpec(RemoteRestartedQuarantinedSpec) { import RemoteRestartedQuarantinedSpec._ @@ -80,7 +79,7 @@ abstract class RemoteRestartedQuarantinedSpec val (uid, ref) = identifyWithUid(second, "subject") - RARP(system).provider.transport.quarantine(node(second).address, Some(uid)) + RARP(system).provider.transport.quarantine(node(second).address, Some(uid), "test") enterBarrier("quarantined") enterBarrier("still-quarantined") diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemotingMultiNodeSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemotingMultiNodeSpec.scala new file mode 100644 index 0000000000..fddc369110 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemotingMultiNodeSpec.scala @@ -0,0 +1,48 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote + +import java.util.UUID + +import akka.remote.testkit.{ FlightRecordingSupport, MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec } +import akka.testkit.{ DefaultTimeout, ImplicitSender } +import com.typesafe.config.ConfigFactory +import org.scalatest.{ Outcome, Suite } + +object RemotingMultiNodeSpec { + + def arteryFlightRecordingConf = + ConfigFactory.parseString( + s""" + akka.remote.artery.advanced.flight-recorder { + enabled=on + destination=target/flight-recorder-${UUID.randomUUID().toString}.afr + } + """) + +} + +abstract class RemotingMultiNodeSpec(config: MultiNodeConfig) extends MultiNodeSpec(config) + with Suite + with STMultiNodeSpec + with FlightRecordingSupport + with ImplicitSender + with DefaultTimeout { self: MultiNodeSpec ⇒ + + // Keep track of failure so we can print artery flight recording on failure + private var failed = false + final override protected def withFixture(test: NoArgTest): Outcome = { + val out = super.withFixture(test) + if (!out.isSucceeded) + failed = true + out + } + + override def afterTermination(): Unit = { + if (failed || sys.props.get("akka.remote.artery.always-dump-flight-recorder").isDefined) { + printFlightRecording() + } + deleteFlightRecorderFile() + } +} diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/Ticket15109Spec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/Ticket15109Spec.scala index dd87973f18..ae00b68ac2 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/Ticket15109Spec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/Ticket15109Spec.scala @@ -47,9 +47,7 @@ object Ticket15109Spec extends MultiNodeConfig { class Ticket15109SpecMultiJvmNode1 extends Ticket15109Spec class Ticket15109SpecMultiJvmNode2 extends Ticket15109Spec -abstract class Ticket15109Spec extends MultiNodeSpec(Ticket15109Spec) - with STMultiNodeSpec - with ImplicitSender { +abstract class Ticket15109Spec extends RemotingMultiNodeSpec(Ticket15109Spec) { import Ticket15109Spec._ diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/AeronStreamConcistencySpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/AeronStreamConcistencySpec.scala new file mode 100644 index 0000000000..240607651d --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/AeronStreamConcistencySpec.scala @@ -0,0 +1,163 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.concurrent.atomic.AtomicInteger + +import scala.concurrent.Await +import scala.concurrent.duration._ +import akka.Done +import akka.remote.testconductor.RoleName +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.remote.testkit.STMultiNodeSpec +import akka.stream.ActorMaterializer +import akka.stream.KillSwitches +import akka.stream.ThrottleMode +import akka.stream.scaladsl.Source +import akka.testkit._ +import com.typesafe.config.ConfigFactory +import io.aeron.Aeron +import io.aeron.driver.MediaDriver +import akka.actor.ExtendedActorSystem +import org.agrona.IoUtil +import java.io.File + +import akka.util.ByteString + +object AeronStreamConsistencySpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + + val barrierTimeout = 5.minutes + + commonConfig(debugConfig(on = false).withFallback( + ConfigFactory.parseString(s""" + akka { + loglevel = INFO + actor { + provider = remote + } + remote.artery.enabled = off + } + """))) + +} + +class AeronStreamConsistencySpecMultiJvmNode1 extends AeronStreamConsistencySpec +class AeronStreamConsistencySpecMultiJvmNode2 extends AeronStreamConsistencySpec + +abstract class AeronStreamConsistencySpec + extends MultiNodeSpec(AeronStreamConsistencySpec) + with STMultiNodeSpec with ImplicitSender { + + import AeronStreamConsistencySpec._ + + val driver = MediaDriver.launchEmbedded() + + val aeron = { + val ctx = new Aeron.Context + ctx.aeronDirectoryName(driver.aeronDirectoryName) + Aeron.connect(ctx) + } + + val idleCpuLevel = system.settings.config.getInt("akka.remote.artery.advanced.idle-cpu-level") + val taskRunner = { + val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem], idleCpuLevel) + r.start() + r + } + + val pool = new EnvelopeBufferPool(1024 * 1024, 128) + + lazy implicit val mat = ActorMaterializer()(system) + import system.dispatcher + + override def initialParticipants = roles.size + + def channel(roleName: RoleName) = { + val a = node(roleName).address + s"aeron:udp?endpoint=${a.host.get}:${a.port.get}" + } + + val streamId = 1 + val giveUpMessageAfter = 30.seconds + + override def afterAll(): Unit = { + taskRunner.stop() + aeron.close() + driver.close() + IoUtil.delete(new File(driver.aeronDirectoryName), true) + super.afterAll() + } + + "Message consistency of Aeron Streams" must { + + "start echo" in { + runOn(second) { + // just echo back + Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner, pool, IgnoreEventSink)) + .runWith(new AeronSink(channel(first), streamId, aeron, taskRunner, pool, giveUpMessageAfter, IgnoreEventSink)) + } + enterBarrier("echo-started") + } + + "deliver messages in order without loss" in { + runOn(first) { + val totalMessages = 50000 + val count = new AtomicInteger + val done = TestLatch(1) + val killSwitch = KillSwitches.shared("test") + val started = TestProbe() + val startMsg = "0".getBytes("utf-8") + Source.fromGraph(new AeronSource(channel(first), streamId, aeron, taskRunner, pool, IgnoreEventSink)) + .via(killSwitch.flow) + .runForeach { envelope ⇒ + val bytes = ByteString.fromByteBuffer(envelope.byteBuffer) + if (bytes.length == 1 && bytes(0) == startMsg(0)) + started.ref ! Done + else { + val c = count.incrementAndGet() + val x = new String(bytes.toArray, "utf-8").toInt + if (x != c) { + throw new IllegalArgumentException(s"# wrong message $x expected $c") + } + if (c == totalMessages) + done.countDown() + } + pool.release(envelope) + }.onFailure { + case e ⇒ e.printStackTrace + } + + within(10.seconds) { + Source(1 to 100).map { _ ⇒ + val envelope = pool.acquire() + envelope.byteBuffer.put(startMsg) + envelope.byteBuffer.flip() + envelope + } + .throttle(1, 200.milliseconds, 1, ThrottleMode.Shaping) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpMessageAfter, IgnoreEventSink)) + started.expectMsg(Done) + } + + Source(1 to totalMessages) + .throttle(10000, 1.second, 1000, ThrottleMode.Shaping) + .map { n ⇒ + val envelope = pool.acquire() + envelope.byteBuffer.put(n.toString.getBytes("utf-8")) + envelope.byteBuffer.flip() + envelope + } + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpMessageAfter, IgnoreEventSink)) + + Await.ready(done, 20.seconds) + killSwitch.shutdown() + } + enterBarrier("after-1") + } + + } +} diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/AeronStreamLatencySpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/AeronStreamLatencySpec.scala new file mode 100644 index 0000000000..e4cda8e744 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/AeronStreamLatencySpec.scala @@ -0,0 +1,329 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.io.File +import java.util.concurrent.CyclicBarrier +import java.util.concurrent.Executors +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.atomic.AtomicLong +import java.util.concurrent.atomic.AtomicLongArray +import java.util.concurrent.locks.LockSupport + +import scala.concurrent.duration._ + +import akka.Done +import akka.actor._ +import akka.remote.testconductor.RoleName +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.remote.testkit.STMultiNodeSpec +import akka.stream.ActorMaterializer +import akka.stream.KillSwitches +import akka.stream.ThrottleMode +import akka.stream.scaladsl.Flow +import akka.stream.scaladsl.Keep +import akka.stream.scaladsl.Source +import akka.testkit._ +import akka.util.ByteString +import com.typesafe.config.ConfigFactory +import io.aeron.Aeron +import io.aeron.CncFileDescriptor +import io.aeron.driver.MediaDriver +import org.HdrHistogram.Histogram +import org.agrona.IoUtil +import org.agrona.concurrent.ManyToOneConcurrentArrayQueue + +object AeronStreamLatencySpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + + val barrierTimeout = 5.minutes + + commonConfig(debugConfig(on = false).withFallback( + ConfigFactory.parseString(s""" + # for serious measurements you should increase the totalMessagesFactor (10) and repeatCount (3) + akka.test.AeronStreamLatencySpec.totalMessagesFactor = 1.0 + akka.test.AeronStreamLatencySpec.repeatCount = 1 + akka { + loglevel = ERROR + testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s + actor { + provider = remote + serialize-creators = false + serialize-messages = false + } + remote.artery { + enabled = off + advanced.idle-cpu-level=8 + } + } + """))) + + final case class TestSettings( + testName: String, + messageRate: Int, // msg/s + payloadSize: Int, + repeat: Int) + +} + +class AeronStreamLatencySpecMultiJvmNode1 extends AeronStreamLatencySpec +class AeronStreamLatencySpecMultiJvmNode2 extends AeronStreamLatencySpec + +abstract class AeronStreamLatencySpec + extends MultiNodeSpec(AeronStreamLatencySpec) + with STMultiNodeSpec with ImplicitSender { + + import AeronStreamLatencySpec._ + + val totalMessagesFactor = system.settings.config.getDouble("akka.test.AeronStreamLatencySpec.totalMessagesFactor") + val repeatCount = system.settings.config.getInt("akka.test.AeronStreamLatencySpec.repeatCount") + + var plots = LatencyPlots() + + val driver = MediaDriver.launchEmbedded() + + val pool = new EnvelopeBufferPool(1024 * 1024, 128) + + val cncByteBuffer = IoUtil.mapExistingFile(new File(driver.aeronDirectoryName, CncFileDescriptor.CNC_FILE), "cnc"); + val stats = + new AeronStat(AeronStat.mapCounters(cncByteBuffer)) + + val aeron = { + val ctx = new Aeron.Context + ctx.aeronDirectoryName(driver.aeronDirectoryName) + Aeron.connect(ctx) + } + + val idleCpuLevel = system.settings.config.getInt("akka.remote.artery.advanced.idle-cpu-level") + val taskRunner = { + val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem], idleCpuLevel) + r.start() + r + } + + lazy implicit val mat = ActorMaterializer()(system) + import system.dispatcher + + override def initialParticipants = roles.size + + def channel(roleName: RoleName) = { + val a = node(roleName).address + s"aeron:udp?endpoint=${a.host.get}:${a.port.get}" + } + + val streamId = 1 + val giveUpMessageAfter = 30.seconds + + lazy val reporterExecutor = Executors.newFixedThreadPool(1) + def reporter(name: String): TestRateReporter = { + val r = new TestRateReporter(name) + reporterExecutor.execute(r) + r + } + + override def afterAll(): Unit = { + reporterExecutor.shutdown() + taskRunner.stop() + aeron.close() + driver.close() + IoUtil.unmap(cncByteBuffer) + IoUtil.delete(new File(driver.aeronDirectoryName), true) + runOn(first) { + println(plots.plot50.csv(system.name + "50")) + println(plots.plot90.csv(system.name + "90")) + println(plots.plot99.csv(system.name + "99")) + } + super.afterAll() + } + + def printTotal(testName: String, payloadSize: Long, histogram: Histogram, totalDurationNanos: Long, lastRepeat: Boolean): Unit = { + import scala.collection.JavaConverters._ + val percentiles = histogram.percentiles(5) + def percentile(p: Double): Double = + percentiles.iterator().asScala.collectFirst { + case value if (p - 0.5) < value.getPercentileLevelIteratedTo && + value.getPercentileLevelIteratedTo < (p + 0.5) ⇒ value.getValueIteratedTo / 1000.0 + }.getOrElse(Double.NaN) + + val throughput = 1000.0 * histogram.getTotalCount / totalDurationNanos.nanos.toMillis + + println(s"=== AeronStreamLatency $testName: RTT " + + f"50%%ile: ${percentile(50.0)}%.0f µs, " + + f"90%%ile: ${percentile(90.0)}%.0f µs, " + + f"99%%ile: ${percentile(99.0)}%.0f µs, " + + f"rate: ${throughput}%,.0f msg/s") + println("Histogram of RTT latencies in microseconds.") + histogram.outputPercentileDistribution(System.out, 1000.0) + + // only use the last repeat for the plots + if (lastRepeat) { + plots = plots.copy( + plot50 = plots.plot50.add(testName, percentile(50.0)), + plot90 = plots.plot90.add(testName, percentile(90.0)), + plot99 = plots.plot99.add(testName, percentile(99.0))) + } + } + + def printStats(side: String): Unit = { + println(side + " stats:") + stats.print(System.out) + } + + val scenarios = List( + TestSettings( + testName = "rate-100-size-100", + messageRate = 100, + payloadSize = 100, + repeat = repeatCount), + TestSettings( + testName = "rate-1000-size-100", + messageRate = 1000, + payloadSize = 100, + repeat = repeatCount), + TestSettings( + testName = "rate-10000-size-100", + messageRate = 10000, + payloadSize = 100, + repeat = repeatCount), + TestSettings( + testName = "rate-20000-size-100", + messageRate = 20000, + payloadSize = 100, + repeat = repeatCount), + TestSettings( + testName = "rate-1000-size-1k", + messageRate = 1000, + payloadSize = 1000, + repeat = repeatCount)) + + def test(testSettings: TestSettings): Unit = { + import testSettings._ + + runOn(first) { + val payload = ("1" * payloadSize).getBytes("utf-8") + // by default run for 2 seconds, but can be adjusted with the totalMessagesFactor + val totalMessages = (2 * messageRate * totalMessagesFactor).toInt + val sendTimes = new AtomicLongArray(totalMessages) + val histogram = new Histogram(SECONDS.toNanos(10), 3) + + val rep = reporter(testName) + val barrier = new CyclicBarrier(2) + val count = new AtomicInteger + val startTime = new AtomicLong + val lastRepeat = new AtomicBoolean(false) + val killSwitch = KillSwitches.shared(testName) + val started = TestProbe() + val startMsg = "0".getBytes("utf-8") + Source.fromGraph(new AeronSource(channel(first), streamId, aeron, taskRunner, pool, IgnoreEventSink)) + .via(killSwitch.flow) + .runForeach { envelope ⇒ + val bytes = ByteString.fromByteBuffer(envelope.byteBuffer) + if (bytes.length == 1 && bytes(0) == startMsg(0)) + started.ref ! Done + else { + if (bytes.length != payloadSize) throw new IllegalArgumentException("Invalid message") + rep.onMessage(1, payloadSize) + val c = count.incrementAndGet() + val d = System.nanoTime() - sendTimes.get(c - 1) + histogram.recordValue(d) + if (c == totalMessages) { + val totalDurationNanos = System.nanoTime() - startTime.get + printTotal(testName, bytes.length, histogram, totalDurationNanos, lastRepeat.get) + barrier.await() // this is always the last party + } + } + pool.release(envelope) + } + + within(10.seconds) { + Source(1 to 50).map { _ ⇒ + val envelope = pool.acquire() + envelope.byteBuffer.put(startMsg) + envelope.byteBuffer.flip() + envelope + } + .throttle(1, 200.milliseconds, 1, ThrottleMode.Shaping) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpMessageAfter, IgnoreEventSink)) + started.expectMsg(Done) + } + + for (rep ← 1 to repeat) { + histogram.reset() + count.set(0) + lastRepeat.set(rep == repeat) + + val sendFlow = Flow[Unit] + .map { _ ⇒ + val envelope = pool.acquire() + envelope.byteBuffer.put(payload) + envelope.byteBuffer.flip() + envelope + } + + val queueValue = Source.fromGraph(new SendQueue[Unit]) + .via(sendFlow) + .to(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpMessageAfter, IgnoreEventSink)) + .run() + + val queue = new ManyToOneConcurrentArrayQueue[Unit](1024) + queueValue.inject(queue) + Thread.sleep(3000) // let materialization complete + + startTime.set(System.nanoTime()) + + var i = 0 + var adjust = 0L + // increase the rate somewhat to compensate for overhead, based on heuristics + val adjustRateFactor = + if (messageRate <= 100) 1.05 + else if (messageRate <= 1000) 1.1 + else if (messageRate <= 10000) 1.2 + else if (messageRate <= 20000) 1.3 + else 1.4 + val targetDelay = (SECONDS.toNanos(1) / (messageRate * adjustRateFactor)).toLong + while (i < totalMessages) { + LockSupport.parkNanos(targetDelay - adjust) + val now = System.nanoTime() + sendTimes.set(i, now) + if (i >= 1) { + val diff = now - sendTimes.get(i - 1) + adjust = math.max(0L, (diff - targetDelay) / 2) + } + + if (!queueValue.offer(())) + fail("sendQueue full") + i += 1 + } + + barrier.await((totalMessages / messageRate) + 10, SECONDS) + } + + killSwitch.shutdown() + rep.halt() + } + + printStats(myself.name) + enterBarrier("after-" + testName) + } + + "Latency of Aeron Streams" must { + + "start echo" in { + runOn(second) { + // just echo back + Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner, pool, IgnoreEventSink)) + .runWith(new AeronSink(channel(first), streamId, aeron, taskRunner, pool, giveUpMessageAfter, IgnoreEventSink)) + } + enterBarrier("echo-started") + } + + for (s ← scenarios) { + s"be low for ${s.testName}, at ${s.messageRate} msg/s, payloadSize = ${s.payloadSize}" in test(s) + } + + } +} diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/AeronStreamMaxThroughputSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/AeronStreamMaxThroughputSpec.scala new file mode 100644 index 0000000000..6215080ce5 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/AeronStreamMaxThroughputSpec.scala @@ -0,0 +1,235 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.net.InetAddress +import java.util.concurrent.Executors + +import scala.collection.AbstractIterator +import scala.concurrent.Await +import scala.concurrent.duration._ +import akka.actor._ +import akka.remote.testconductor.RoleName +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.remote.testkit.STMultiNodeSpec +import akka.stream.ActorMaterializer +import akka.stream.scaladsl.Source +import akka.testkit._ +import com.typesafe.config.ConfigFactory +import io.aeron.Aeron +import io.aeron.driver.MediaDriver +import akka.stream.KillSwitches +import java.io.File + +import akka.util.ByteString +import io.aeron.CncFileDescriptor +import org.agrona.IoUtil + +object AeronStreamMaxThroughputSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + + val barrierTimeout = 5.minutes + + commonConfig(debugConfig(on = false).withFallback( + ConfigFactory.parseString(s""" + # for serious measurements you should increase the totalMessagesFactor (20) + akka.test.AeronStreamMaxThroughputSpec.totalMessagesFactor = 1.0 + akka { + loglevel = ERROR + testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s + actor { + provider = remote + serialize-creators = false + serialize-messages = false + } + remote.artery.enabled = off + } + """))) + + final case class TestSettings( + testName: String, + totalMessages: Long, + payloadSize: Int) + + def iterate(start: Long, end: Long): Iterator[Long] = new AbstractIterator[Long] { + private[this] var first = true + private[this] var acc = start + def hasNext: Boolean = acc < end + def next(): Long = { + if (!hasNext) throw new NoSuchElementException("next on empty iterator") + if (first) first = false + else acc += 1 + + acc + } + } + +} + +class AeronStreamMaxThroughputSpecMultiJvmNode1 extends AeronStreamMaxThroughputSpec +class AeronStreamMaxThroughputSpecMultiJvmNode2 extends AeronStreamMaxThroughputSpec + +abstract class AeronStreamMaxThroughputSpec + extends MultiNodeSpec(AeronStreamMaxThroughputSpec) + with STMultiNodeSpec with ImplicitSender { + + import AeronStreamMaxThroughputSpec._ + + val totalMessagesFactor = system.settings.config.getDouble("akka.test.AeronStreamMaxThroughputSpec.totalMessagesFactor") + + var plot = PlotResult() + + val driver = MediaDriver.launchEmbedded() + + val pool = new EnvelopeBufferPool(1024 * 1024, 128) + + val cncByteBuffer = IoUtil.mapExistingFile(new File(driver.aeronDirectoryName, CncFileDescriptor.CNC_FILE), "cnc"); + val stats = + new AeronStat(AeronStat.mapCounters(cncByteBuffer)) + + val aeron = { + val ctx = new Aeron.Context + ctx.aeronDirectoryName(driver.aeronDirectoryName) + Aeron.connect(ctx) + } + + val idleCpuLevel = system.settings.config.getInt("akka.remote.artery.advanced.idle-cpu-level") + val taskRunner = { + val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem], idleCpuLevel) + r.start() + r + } + + lazy implicit val mat = ActorMaterializer()(system) + import system.dispatcher + + def adjustedTotalMessages(n: Long): Long = (n * totalMessagesFactor).toLong + + override def initialParticipants = roles.size + + def channel(roleName: RoleName) = { + val a = node(roleName).address + s"aeron:udp?endpoint=${a.host.get}:${a.port.get}" + } + + val streamId = 1 + val giveUpMessageAfter = 30.seconds + + lazy val reporterExecutor = Executors.newFixedThreadPool(1) + def reporter(name: String): TestRateReporter = { + val r = new TestRateReporter(name) + reporterExecutor.execute(r) + r + } + + override def afterAll(): Unit = { + reporterExecutor.shutdown() + taskRunner.stop() + aeron.close() + driver.close() + IoUtil.unmap(cncByteBuffer) + IoUtil.delete(new File(driver.aeronDirectoryName), true) + runOn(second) { + println(plot.csv(system.name)) + } + super.afterAll() + } + + def printTotal(testName: String, total: Long, startTime: Long, payloadSize: Long): Unit = { + val d = (System.nanoTime - startTime).nanos.toMillis + val throughput = 1000.0 * total / d + println(f"=== AeronStreamMaxThroughput $testName: " + + f"${throughput}%,.0f msg/s, ${throughput * payloadSize}%,.0f bytes/s, " + + s"payload size $payloadSize, " + + s"$d ms to deliver $total messages") + plot = plot.add(testName, throughput * payloadSize / 1024 / 1024) + } + + def printStats(side: String): Unit = { + println(side + " stats:") + stats.print(System.out) + } + + val scenarios = List( + TestSettings( + testName = "size-100", + totalMessages = adjustedTotalMessages(1000000), + payloadSize = 100), + TestSettings( + testName = "size-1k", + totalMessages = adjustedTotalMessages(100000), + payloadSize = 1000), + TestSettings( + testName = "size-10k", + totalMessages = adjustedTotalMessages(10000), + payloadSize = 10000)) + + def test(testSettings: TestSettings): Unit = { + import testSettings._ + val receiverName = testName + "-rcv" + + runOn(second) { + val rep = reporter(testName) + var t0 = System.nanoTime() + var count = 0L + val done = TestLatch(1) + val killSwitch = KillSwitches.shared(testName) + Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner, pool, IgnoreEventSink)) + .via(killSwitch.flow) + .runForeach { envelope ⇒ + val bytes = ByteString.fromByteBuffer(envelope.byteBuffer) + rep.onMessage(1, bytes.length) + count += 1 + if (count == 1) { + t0 = System.nanoTime() + } else if (count == totalMessages) { + printTotal(testName, totalMessages, t0, payloadSize) + done.countDown() + killSwitch.shutdown() + } + pool.release(envelope) + }.onFailure { + case e ⇒ + e.printStackTrace + } + + enterBarrier(receiverName + "-started") + Await.ready(done, barrierTimeout) + rep.halt() + printStats("receiver") + enterBarrier(testName + "-done") + } + + runOn(first) { + enterBarrier(receiverName + "-started") + + val payload = ("0" * payloadSize).getBytes("utf-8") + val t0 = System.nanoTime() + Source.fromIterator(() ⇒ iterate(1, totalMessages)) + .map { n ⇒ + val envelope = pool.acquire() + envelope.byteBuffer.put(payload) + envelope.byteBuffer.flip() + envelope + } + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpMessageAfter, IgnoreEventSink)) + + printStats("sender") + enterBarrier(testName + "-done") + + } + + enterBarrier("after-" + testName) + } + + "Max throughput of Aeron Streams" must { + + for (s ← scenarios) { + s"be great for ${s.testName}, payloadSize = ${s.payloadSize}" in test(s) + } + + } +} diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/HandshakeRestartReceiverSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/HandshakeRestartReceiverSpec.scala new file mode 100644 index 0000000000..a48cf210c6 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/HandshakeRestartReceiverSpec.scala @@ -0,0 +1,117 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.Await +import scala.concurrent.duration._ + +import akka.actor._ +import akka.remote.AddressUidExtension +import akka.remote.RARP +import akka.remote.testconductor.RoleName +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.remote.testkit.STMultiNodeSpec +import akka.testkit._ +import com.typesafe.config.ConfigFactory + +object HandshakeRestartReceiverSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + + commonConfig(debugConfig(on = false).withFallback( + ConfigFactory.parseString(s""" + akka { + loglevel = INFO + actor.provider = remote + remote.artery { + enabled = on + } + } + """))) + + class Subject extends Actor { + def receive = { + case "shutdown" ⇒ context.system.terminate() + case "identify" ⇒ sender() ! (AddressUidExtension(context.system).longAddressUid → self) + } + } + +} + +class HandshakeRestartReceiverSpecMultiJvmNode1 extends HandshakeRestartReceiverSpec +class HandshakeRestartReceiverSpecMultiJvmNode2 extends HandshakeRestartReceiverSpec + +abstract class HandshakeRestartReceiverSpec + extends MultiNodeSpec(HandshakeRestartReceiverSpec) + with STMultiNodeSpec with ImplicitSender { + + import HandshakeRestartReceiverSpec._ + + override def initialParticipants = roles.size + + override def afterAll(): Unit = { + super.afterAll() + } + + def identifyWithUid(rootPath: ActorPath, actorName: String, timeout: FiniteDuration = remainingOrDefault): (Long, ActorRef) = { + within(timeout) { + system.actorSelection(rootPath / "user" / actorName) ! "identify" + expectMsgType[(Long, ActorRef)] + } + } + + "Artery Handshake" must { + + "detect restarted receiver and initiate new handshake" in { + runOn(second) { + system.actorOf(Props[Subject], "subject") + } + enterBarrier("subject-started") + + runOn(first) { + val secondRootPath = node(second) + val (secondUid, _) = identifyWithUid(secondRootPath, "subject", 5.seconds) + + val secondAddress = node(second).address + val secondAssociation = RARP(system).provider.transport.asInstanceOf[ArteryTransport].association(secondAddress) + val secondUniqueRemoteAddress = Await.result(secondAssociation.associationState.uniqueRemoteAddress, 3.seconds) + secondUniqueRemoteAddress.address should ===(secondAddress) + secondUniqueRemoteAddress.uid should ===(secondUid) + + enterBarrier("before-shutdown") + testConductor.shutdown(second).await + + within(30.seconds) { + awaitAssert { + identifyWithUid(secondRootPath, "subject2", 1.second) + } + } + val (secondUid2, subject2) = identifyWithUid(secondRootPath, "subject2") + secondUid2 should !==(secondUid) + val secondUniqueRemoteAddress2 = Await.result(secondAssociation.associationState.uniqueRemoteAddress, 3.seconds) + secondUniqueRemoteAddress2.uid should ===(secondUid2) + secondUniqueRemoteAddress2.address should ===(secondAddress) + secondUniqueRemoteAddress2 should !==(secondUniqueRemoteAddress) + + subject2 ! "shutdown" + } + + runOn(second) { + val addr = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + enterBarrier("before-shutdown") + + Await.result(system.whenTerminated, 10.seconds) + + val freshSystem = ActorSystem(system.name, ConfigFactory.parseString(s""" + akka.remote.artery.canonical.port = ${addr.port.get} + """).withFallback(system.settings.config)) + freshSystem.actorOf(Props[Subject], "subject2") + + Await.result(freshSystem.whenTerminated, 45.seconds) + } + } + + } +} diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/LatencySpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/LatencySpec.scala new file mode 100644 index 0000000000..884a1fbbed --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/LatencySpec.scala @@ -0,0 +1,346 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.concurrent.Executors +import java.util.concurrent.atomic.{ AtomicBoolean, AtomicLongArray } +import java.util.concurrent.locks.LockSupport + +import scala.concurrent.duration._ +import akka.actor._ +import akka.remote.RemotingMultiNodeSpec +import akka.remote.testconductor.RoleName +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.remote.testkit.STMultiNodeSpec +import akka.stream.ActorMaterializer +import akka.testkit._ +import com.typesafe.config.ConfigFactory +import org.HdrHistogram.Histogram +import akka.stream.scaladsl.Source +import akka.stream.ThrottleMode + +object LatencySpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + + val barrierTimeout = 5.minutes + + commonConfig(debugConfig(on = false).withFallback( + ConfigFactory.parseString(s""" + # for serious measurements you should increase the totalMessagesFactor (30) and repeatCount (3) + akka.test.LatencySpec.totalMessagesFactor = 1.0 + akka.test.LatencySpec.repeatCount = 1 + akka.test.LatencySpec.real-message = off + akka { + loglevel = ERROR + # avoid TestEventListener + loggers = ["akka.event.Logging$$DefaultLogger"] + testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s + actor { + provider = remote + serialize-creators = false + serialize-messages = false + } + remote.artery { + enabled = on + advanced.idle-cpu-level=7 + + advanced.compression { + actor-refs.advertisement-interval = 2 second + manifests.advertisement-interval = 2 second + } + } + } + """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf)) + + final case object Reset + + def echoProps(): Props = + Props(new Echo) + + class Echo extends Actor { + // FIXME to avoid using new RemoteActorRef each time + var cachedSender: ActorRef = null + + def receive = { + case Reset ⇒ + cachedSender = null + sender() ! Reset + case msg ⇒ + if (cachedSender == null) cachedSender = sender() + cachedSender ! msg + } + } + + def receiverProps(reporter: RateReporter, settings: TestSettings, totalMessages: Int, + sendTimes: AtomicLongArray, histogram: Histogram, plotsRef: ActorRef): Props = + Props(new Receiver(reporter, settings, totalMessages, sendTimes, histogram, plotsRef)) + + class Receiver(reporter: RateReporter, settings: TestSettings, totalMessages: Int, + sendTimes: AtomicLongArray, histogram: Histogram, plotsRef: ActorRef) extends Actor { + import settings._ + + var count = 0 + var startTime = System.nanoTime() + val taskRunnerMetrics = new TaskRunnerMetrics(context.system) + var reportedArrayOOB = false + + def receive = { + case bytes: Array[Byte] ⇒ + if (bytes.length != 0) { + if (bytes.length != payloadSize) throw new IllegalArgumentException("Invalid message") + receiveMessage(bytes.length) + } + case _: TestMessage ⇒ + receiveMessage(payloadSize) + } + + def receiveMessage(size: Int): Unit = { + if (count == 0) + startTime = System.nanoTime() + reporter.onMessage(1, payloadSize) + count += 1 + val d = System.nanoTime() - sendTimes.get(count - 1) + try { + histogram.recordValue(d) + } catch { + case e: ArrayIndexOutOfBoundsException ⇒ + // Report it only once instead of flooding the console + if (!reportedArrayOOB) { + e.printStackTrace() + reportedArrayOOB = true + } + } + if (count == totalMessages) { + printTotal(testName, size, histogram, System.nanoTime() - startTime) + context.stop(self) + } + } + + def printTotal(testName: String, payloadSize: Long, histogram: Histogram, totalDurationNanos: Long): Unit = { + import scala.collection.JavaConverters._ + val percentiles = histogram.percentiles(5) + def percentile(p: Double): Double = + percentiles.iterator().asScala.collectFirst { + case value if (p - 0.5) < value.getPercentileLevelIteratedTo && + value.getPercentileLevelIteratedTo < (p + 0.5) ⇒ value.getValueIteratedTo / 1000.0 + }.getOrElse(Double.NaN) + + val throughput = 1000.0 * histogram.getTotalCount / math.max(1, totalDurationNanos.nanos.toMillis) + + println(s"=== Latency $testName: RTT " + + f"50%%ile: ${percentile(50.0)}%.0f µs, " + + f"90%%ile: ${percentile(90.0)}%.0f µs, " + + f"99%%ile: ${percentile(99.0)}%.0f µs, " + + f"rate: ${throughput}%,.0f msg/s") + println("Histogram of RTT latencies in microseconds.") + histogram.outputPercentileDistribution(System.out, 1000.0) + + taskRunnerMetrics.printHistograms() + + val plots = LatencyPlots( + PlotResult().add(testName, percentile(50.0)), + PlotResult().add(testName, percentile(90.0)), + PlotResult().add(testName, percentile(99.0))) + plotsRef ! plots + } + } + + final case class TestSettings( + testName: String, + messageRate: Int, // msg/s + payloadSize: Int, + repeat: Int, + realMessage: Boolean) + +} + +class LatencySpecMultiJvmNode1 extends LatencySpec +class LatencySpecMultiJvmNode2 extends LatencySpec + +abstract class LatencySpec + extends RemotingMultiNodeSpec(LatencySpec) { + + import LatencySpec._ + + val totalMessagesFactor = system.settings.config.getDouble("akka.test.LatencySpec.totalMessagesFactor") + val repeatCount = system.settings.config.getInt("akka.test.LatencySpec.repeatCount") + val realMessage = system.settings.config.getBoolean("akka.test.LatencySpec.real-message") + + var plots = LatencyPlots() + + lazy implicit val mat = ActorMaterializer()(system) + import system.dispatcher + + override def initialParticipants = roles.size + + lazy val reporterExecutor = Executors.newFixedThreadPool(1) + def reporter(name: String): TestRateReporter = { + val r = new TestRateReporter(name) + reporterExecutor.execute(r) + r + } + + override def afterAll(): Unit = { + reporterExecutor.shutdown() + runOn(first) { + println(plots.plot50.csv(system.name + "50")) + println(plots.plot90.csv(system.name + "90")) + println(plots.plot99.csv(system.name + "99")) + } + super.afterAll() + } + + def identifyEcho(name: String = "echo", r: RoleName = second): ActorRef = { + system.actorSelection(node(r) / "user" / name) ! Identify(None) + expectMsgType[ActorIdentity].ref.get + } + + val scenarios = List( + TestSettings( + testName = "warmup", + messageRate = 10000, + payloadSize = 100, + repeat = repeatCount, + realMessage), + TestSettings( + testName = "rate-100-size-100", + messageRate = 100, + payloadSize = 100, + repeat = repeatCount, + realMessage), + TestSettings( + testName = "rate-1000-size-100", + messageRate = 1000, + payloadSize = 100, + repeat = repeatCount, + realMessage), + TestSettings( + testName = "rate-10000-size-100", + messageRate = 10000, + payloadSize = 100, + repeat = repeatCount, + realMessage), + TestSettings( + testName = "rate-20000-size-100", + messageRate = 20000, + payloadSize = 100, + repeat = repeatCount, + realMessage), + TestSettings( + testName = "rate-1000-size-1k", + messageRate = 1000, + payloadSize = 1000, + repeat = repeatCount, + realMessage)) + + def test(testSettings: TestSettings): Unit = { + import testSettings._ + + runOn(first) { + val payload = ("0" * payloadSize).getBytes("utf-8") + // by default run for 2 seconds, but can be adjusted with the totalMessagesFactor + val totalMessages = (2 * messageRate * totalMessagesFactor).toInt + val sendTimes = new AtomicLongArray(totalMessages) + val histogram = new Histogram(SECONDS.toNanos(10), 3) + val rep = reporter(testName) + + val echo = identifyEcho() + val plotProbe = TestProbe() + + // increase the rate somewhat to compensate for overhead, based on heuristics + // will also be adjusted based on measurement when using > 1 repeat + @volatile var adjustRateFactor = + if (messageRate <= 100) 1.05 + else if (messageRate <= 1000) 1.1 + else if (messageRate <= 10000) 1.2 + else if (messageRate <= 20000) 1.3 + else 1.4 + + for (n ← 1 to repeat) { + echo ! Reset + expectMsg(Reset) + histogram.reset() + val receiver = system.actorOf(receiverProps(rep, testSettings, totalMessages, sendTimes, histogram, plotProbe.ref)) + + // warmup for 3 seconds to init compression + val warmup = Source(1 to 30) + .throttle(10, 1.second, 10, ThrottleMode.Shaping) + .runForeach { n ⇒ + echo.tell(Array.emptyByteArray, receiver) + } + + warmup.foreach { _ ⇒ + var i = 0 + var adjust = 0L + val targetDelay = (SECONDS.toNanos(1) / (messageRate * adjustRateFactor)).toLong + while (i < totalMessages) { + LockSupport.parkNanos(targetDelay - adjust) + val now = System.nanoTime() + sendTimes.set(i, now) + if (i >= 1) { + val diff = now - sendTimes.get(i - 1) + adjust = math.max(0L, (diff - targetDelay) / 2) + } + + val msg = + if (testSettings.realMessage) + TestMessage( + id = i, + name = "abc", + status = i % 2 == 0, + description = "ABC", + payload = payload, + items = Vector(TestMessage.Item(1, "A"), TestMessage.Item(2, "B"))) + else payload + + echo.tell(payload, receiver) + i += 1 + } + + // measure rate and adjust for next repeat round + val d = (sendTimes.get(totalMessages - 1) - sendTimes.get(0)) + val measuredRate = totalMessages * SECONDS.toNanos(1) / math.max(1, d) + val previousTargetRate = messageRate * adjustRateFactor + adjustRateFactor = (previousTargetRate / math.max(1, measuredRate)) + println(s"Measured send rate $measuredRate msg/s (new adjustment facor: $adjustRateFactor)") + } + + watch(receiver) + expectTerminated(receiver, ((totalMessages / messageRate) + 20).seconds) + val p = plotProbe.expectMsgType[LatencyPlots] + // only use the last repeat for the plots + if (n == repeat) { + plots = plots.copy( + plot50 = plots.plot50.addAll(p.plot50), + plot90 = plots.plot90.addAll(p.plot90), + plot99 = plots.plot99.addAll(p.plot99)) + } + } + + rep.halt() + } + + enterBarrier("after-" + testName) + } + + "Latency of Artery" must { + + "start echo" in { + runOn(second) { + // just echo back + system.actorOf(echoProps, "echo") + } + enterBarrier("echo-started") + } + + for (s ← scenarios) { + s"be low for ${s.testName}, at ${s.messageRate} msg/s, payloadSize = ${s.payloadSize}" in test(s) + } + + // TODO add more tests + + } +} diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/MaxThroughputSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/MaxThroughputSpec.scala new file mode 100644 index 0000000000..ca19ed0339 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/MaxThroughputSpec.scala @@ -0,0 +1,406 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.nio.ByteBuffer +import java.util.concurrent.Executors +import java.util.concurrent.TimeUnit.NANOSECONDS + +import scala.concurrent.duration._ +import akka.actor._ +import akka.remote.{ RemotingMultiNodeSpec, RARP, RemoteActorRefProvider } +import akka.remote.testconductor.RoleName +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.remote.testkit.PerfFlamesSupport +import akka.remote.testkit.STMultiNodeSpec +import akka.serialization.ByteBufferSerializer +import akka.serialization.SerializerWithStringManifest +import akka.testkit._ +import com.typesafe.config.ConfigFactory +import akka.remote.artery.compress.CompressionProtocol.Events.ReceivedActorRefCompressionTable + +object MaxThroughputSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + + val barrierTimeout = 5.minutes + + commonConfig(debugConfig(on = false).withFallback( + ConfigFactory.parseString(s""" + # for serious measurements you should increase the totalMessagesFactor (20) + akka.test.MaxThroughputSpec.totalMessagesFactor = 1.0 + akka.test.MaxThroughputSpec.real-message = off + akka { + loglevel = INFO + log-dead-letters = 1000000 + # avoid TestEventListener + loggers = ["akka.event.Logging$$DefaultLogger"] + testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s + actor { + provider = remote + serialize-creators = false + serialize-messages = false + + serializers { + test = "akka.remote.artery.MaxThroughputSpec$$TestSerializer" + test-message = "akka.remote.artery.TestMessageSerializer" + } + serialization-bindings { + "akka.remote.artery.MaxThroughputSpec$$FlowControl" = test + "akka.remote.artery.TestMessage" = test-message + } + } + remote.artery { + enabled = on + + # for serious measurements when running this test on only one machine + # it is recommended to use external media driver + # See akka-remote-tests/src/test/resources/aeron.properties + #advanced.embedded-media-driver = off + #advanced.aeron-dir = "target/aeron" + + advanced.compression { + actor-refs.advertisement-interval = 2 second + manifests.advertisement-interval = 2 second + } + } + } + """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf)) + + case object Run + sealed trait Echo extends DeadLetterSuppression with JavaSerializable + final case object Start extends Echo + final case object End extends Echo + final case class EndResult(totalReceived: Long) extends JavaSerializable + final case class FlowControl(burstStartTime: Long) extends Echo + + def receiverProps(reporter: RateReporter, payloadSize: Int, printTaskRunnerMetrics: Boolean): Props = + Props(new Receiver(reporter, payloadSize, printTaskRunnerMetrics)).withDispatcher("akka.remote.default-remote-dispatcher") + + class Receiver(reporter: RateReporter, payloadSize: Int, printTaskRunnerMetrics: Boolean) extends Actor { + private var c = 0L + private val taskRunnerMetrics = new TaskRunnerMetrics(context.system) + + def receive = { + case msg: Array[Byte] ⇒ + if (msg.length != payloadSize) throw new IllegalArgumentException("Invalid message") + reporter.onMessage(1, payloadSize) + c += 1 + case msg: TestMessage ⇒ + reporter.onMessage(1, payloadSize) + c += 1 + case Start ⇒ + c = 0 + sender() ! Start + case End ⇒ + if (printTaskRunnerMetrics) + taskRunnerMetrics.printHistograms() + sender() ! EndResult(c) + context.stop(self) + case m: Echo ⇒ + sender() ! m + + } + } + + def senderProps(target: ActorRef, testSettings: TestSettings, plotRef: ActorRef, + printTaskRunnerMetrics: Boolean): Props = + Props(new Sender(target, testSettings, plotRef, printTaskRunnerMetrics)) + + class Sender(target: ActorRef, testSettings: TestSettings, plotRef: ActorRef, printTaskRunnerMetrics: Boolean) + extends Actor { + import testSettings._ + val payload = ("0" * testSettings.payloadSize).getBytes("utf-8") + var startTime = 0L + var remaining = totalMessages + var maxRoundTripMillis = 0L + val taskRunnerMetrics = new TaskRunnerMetrics(context.system) + + context.system.eventStream.subscribe(self, classOf[ReceivedActorRefCompressionTable]) + + val compressionEnabled = + RARP(context.system).provider.transport.isInstanceOf[ArteryTransport] && + RARP(context.system).provider.remoteSettings.Artery.Enabled + + def receive = { + case Run ⇒ + if (compressionEnabled) { + target ! payload + context.setReceiveTimeout(1.second) + context.become(waitingForCompression) + } else { + sendBatch() // first some warmup + target ! Start // then Start, which will echo back here + context.become(active) + } + } + + def waitingForCompression: Receive = { + case ReceivedActorRefCompressionTable(_, table) ⇒ + if (table.dictionary.contains(target)) { + sendBatch() // first some warmup + target ! Start // then Start, which will echo back here + context.setReceiveTimeout(Duration.Undefined) + context.become(active) + } else + target ! payload + case ReceiveTimeout ⇒ + target ! payload + } + + def active: Receive = { + case Start ⇒ + println(s"${self.path.name}: Starting benchmark of $totalMessages messages with burst size " + + s"$burstSize and payload size $payloadSize") + startTime = System.nanoTime + remaining = totalMessages + // have a few batches in flight to make sure there are always messages to send + (1 to 3).foreach { _ ⇒ + val t0 = System.nanoTime() + sendBatch() + sendFlowControl(t0) + } + + case c @ FlowControl(t0) ⇒ + val now = System.nanoTime() + val duration = NANOSECONDS.toMillis(now - t0) + maxRoundTripMillis = math.max(maxRoundTripMillis, duration) + + sendBatch() + sendFlowControl(now) + + case EndResult(totalReceived) ⇒ + val took = NANOSECONDS.toMillis(System.nanoTime - startTime) + val throughput = (totalReceived * 1000.0 / took) + println( + s"=== MaxThroughput ${self.path.name}: " + + f"throughput ${throughput * testSettings.senderReceiverPairs}%,.0f msg/s, " + + f"${throughput * payloadSize * testSettings.senderReceiverPairs}%,.0f bytes/s (payload), " + + f"${throughput * totalSize(context.system) * testSettings.senderReceiverPairs}%,.0f bytes/s (total" + + (if (RARP(context.system).provider.remoteSettings.Artery.Advanced.Compression.Enabled) ",compression" else "") + "), " + + s"dropped ${totalMessages - totalReceived}, " + + s"max round-trip $maxRoundTripMillis ms, " + + s"burst size $burstSize, " + + s"payload size $payloadSize, " + + s"total size ${totalSize(context.system)}, " + + s"$took ms to deliver $totalReceived messages") + + if (printTaskRunnerMetrics) + taskRunnerMetrics.printHistograms() + + plotRef ! PlotResult().add(testName, throughput * payloadSize * testSettings.senderReceiverPairs / 1024 / 1024) + context.stop(self) + + case c: ReceivedActorRefCompressionTable ⇒ + } + + def sendBatch(): Unit = { + val batchSize = math.min(remaining, burstSize) + var i = 0 + while (i < batchSize) { + val msg = + if (realMessage) + TestMessage( + id = totalMessages - remaining + i, + name = "abc", + status = i % 2 == 0, + description = "ABC", + payload = payload, + items = Vector(TestMessage.Item(1, "A"), TestMessage.Item(2, "B"))) + else payload + + // target ! msg + target.tell(msg, ActorRef.noSender) + i += 1 + } + remaining -= batchSize + } + + def sendFlowControl(t0: Long): Unit = { + if (remaining <= 0) + target ! End + else + target ! FlowControl(t0) + } + } + + final case class TestSettings( + testName: String, + totalMessages: Long, + burstSize: Int, + payloadSize: Int, + senderReceiverPairs: Int, + realMessage: Boolean) { + // data based on measurement + def totalSize(system: ActorSystem) = payloadSize + (if (RARP(system).provider.remoteSettings.Artery.Advanced.Compression.Enabled) 38 else 110) + } + + class TestSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest with ByteBufferSerializer { + + val FlowControlManifest = "A" + + override val identifier: Int = 100 + + override def manifest(o: AnyRef): String = + o match { + case _: FlowControl ⇒ FlowControlManifest + } + + override def toBinary(o: AnyRef, buf: ByteBuffer): Unit = + o match { + case FlowControl(burstStartTime) ⇒ buf.putLong(burstStartTime) + } + + override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = + manifest match { + case FlowControlManifest ⇒ FlowControl(buf.getLong) + } + + override def toBinary(o: AnyRef): Array[Byte] = o match { + case FlowControl(burstStartTime) ⇒ + val buf = ByteBuffer.allocate(8) + toBinary(o, buf) + buf.flip() + val bytes = Array.ofDim[Byte](buf.remaining) + buf.get(bytes) + bytes + } + + override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef = + fromBinary(ByteBuffer.wrap(bytes), manifest) + } + +} + +class MaxThroughputSpecMultiJvmNode1 extends MaxThroughputSpec +class MaxThroughputSpecMultiJvmNode2 extends MaxThroughputSpec + +abstract class MaxThroughputSpec extends RemotingMultiNodeSpec(MaxThroughputSpec) with PerfFlamesSupport { + + import MaxThroughputSpec._ + + val totalMessagesFactor = system.settings.config.getDouble("akka.test.MaxThroughputSpec.totalMessagesFactor") + val realMessage = system.settings.config.getBoolean("akka.test.MaxThroughputSpec.real-message") + + var plot = PlotResult() + + def adjustedTotalMessages(n: Long): Long = (n * totalMessagesFactor).toLong + + override def initialParticipants = roles.size + + def remoteSettings = system.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].remoteSettings + + lazy val reporterExecutor = Executors.newFixedThreadPool(1) + def reporter(name: String): TestRateReporter = { + val r = new TestRateReporter(name) + reporterExecutor.execute(r) + r + } + + override def afterAll(): Unit = { + reporterExecutor.shutdown() + runOn(first) { + println(plot.csv(system.name)) + } + super.afterAll() + } + + def identifyReceiver(name: String, r: RoleName = second): ActorRef = { + system.actorSelection(node(r) / "user" / name) ! Identify(None) + expectMsgType[ActorIdentity].ref.get + } + + val scenarios = List( + TestSettings( + testName = "warmup", + totalMessages = adjustedTotalMessages(20000), + burstSize = 1000, + payloadSize = 100, + senderReceiverPairs = 1, + realMessage), + TestSettings( + testName = "1-to-1", + totalMessages = adjustedTotalMessages(50000), + burstSize = 1000, + payloadSize = 100, + senderReceiverPairs = 1, + realMessage), + TestSettings( + testName = "1-to-1-size-1k", + totalMessages = adjustedTotalMessages(20000), + burstSize = 1000, + payloadSize = 1000, + senderReceiverPairs = 1, + realMessage), + TestSettings( + testName = "1-to-1-size-10k", + totalMessages = adjustedTotalMessages(10000), + burstSize = 1000, + payloadSize = 10000, + senderReceiverPairs = 1, + realMessage), + TestSettings( + testName = "5-to-5", + totalMessages = adjustedTotalMessages(20000), + burstSize = 200, // don't exceed the send queue capacity 200*5*3=3000 + payloadSize = 100, + senderReceiverPairs = 5, + realMessage)) + + def test(testSettings: TestSettings): Unit = { + import testSettings._ + val receiverName = testName + "-rcv" + + runPerfFlames(first, second)(delay = 5.seconds, time = 15.seconds) + + runOn(second) { + val rep = reporter(testName) + for (n ← 1 to senderReceiverPairs) { + val receiver = system.actorOf( + receiverProps(rep, payloadSize, printTaskRunnerMetrics = n == 1), + receiverName + n) + } + enterBarrier(receiverName + "-started") + enterBarrier(testName + "-done") + rep.halt() + } + + runOn(first) { + enterBarrier(receiverName + "-started") + val ignore = TestProbe() + val senders = for (n ← 1 to senderReceiverPairs) yield { + val receiver = identifyReceiver(receiverName + n) + val plotProbe = TestProbe() + val snd = system.actorOf( + senderProps(receiver, testSettings, plotProbe.ref, printTaskRunnerMetrics = n == 1), + testName + "-snd" + n) + val terminationProbe = TestProbe() + terminationProbe.watch(snd) + snd ! Run + (snd, terminationProbe, plotProbe) + } + senders.foreach { + case (snd, terminationProbe, plotProbe) ⇒ + if (snd == senders.head._1) { + terminationProbe.expectTerminated(snd, barrierTimeout) + val plotResult = plotProbe.expectMsgType[PlotResult] + plot = plot.addAll(plotResult) + } else + terminationProbe.expectTerminated(snd, 10.seconds) + } + enterBarrier(testName + "-done") + } + + enterBarrier("after-" + testName) + } + + "Max throughput of Artery" must { + + for (s ← scenarios) { + s"be great for ${s.testName}, burstSize = ${s.burstSize}, payloadSize = ${s.payloadSize}" in test(s) + } + + } +} diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/PlotResult.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/PlotResult.scala new file mode 100644 index 0000000000..01033b679c --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/PlotResult.scala @@ -0,0 +1,23 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +final case class PlotResult(values: Vector[(String, Number)] = Vector.empty) { + + def add(key: String, value: Number): PlotResult = + copy(values = values :+ (key → value)) + + def addAll(p: PlotResult): PlotResult = + copy(values ++ p.values) + + def csvLabels: String = values.map(_._1).mkString("\"", "\",\"", "\"") + + def csvValues: String = values.map(_._2).mkString("\"", "\",\"", "\"") + + // this can be split to two lines with bash: cut -d':' -f2,3 | tr ':' $'\n' + def csv(name: String): String = s"PLOT_${name}:${csvLabels}:${csvValues}" + +} + +final case class LatencyPlots(plot50: PlotResult = PlotResult(), plot90: PlotResult = PlotResult(), plot99: PlotResult = PlotResult()) diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteRestartedQuarantinedSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteRestartedQuarantinedSpec.scala new file mode 100644 index 0000000000..34d78ba270 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteRestartedQuarantinedSpec.scala @@ -0,0 +1,134 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.remote.transport.AssociationHandle + +import language.postfixOps +import scala.concurrent.duration._ +import com.typesafe.config.ConfigFactory +import akka.actor._ +import akka.remote.testconductor.RoleName +import akka.remote.transport.ThrottlerTransportAdapter.{ Direction, ForceDisassociate, ForceDisassociateExplicitly } +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.remote.testkit.STMultiNodeSpec +import akka.testkit._ +import akka.actor.ActorIdentity +import akka.remote.testconductor.RoleName +import akka.actor.Identify + +import scala.concurrent.Await +import akka.remote.{ AddressUidExtension, RemotingMultiNodeSpec, RARP, ThisActorSystemQuarantinedEvent } + +object RemoteRestartedQuarantinedSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + + commonConfig(debugConfig(on = false).withFallback( + ConfigFactory.parseString(""" + akka.loglevel = WARNING + akka.remote.log-remote-lifecycle-events = WARNING + akka.remote.artery.enabled = on + """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf)) + + class Subject extends Actor { + def receive = { + case "shutdown" ⇒ context.system.terminate() + case "identify" ⇒ sender() ! (AddressUidExtension(context.system).longAddressUid → self) + } + } + +} + +class RemoteRestartedQuarantinedSpecMultiJvmNode1 extends RemoteRestartedQuarantinedSpec +class RemoteRestartedQuarantinedSpecMultiJvmNode2 extends RemoteRestartedQuarantinedSpec + +abstract class RemoteRestartedQuarantinedSpec extends RemotingMultiNodeSpec(RemoteRestartedQuarantinedSpec) { + + import RemoteRestartedQuarantinedSpec._ + + override def initialParticipants = 2 + + def identifyWithUid(role: RoleName, actorName: String, timeout: FiniteDuration = remainingOrDefault): (Long, ActorRef) = { + within(timeout) { + system.actorSelection(node(role) / "user" / actorName) ! "identify" + expectMsgType[(Long, ActorRef)] + } + } + + "A restarted quarantined system" must { + + "should not crash the other system (#17213)" taggedAs LongRunningTest in { + + system.actorOf(Props[Subject], "subject") + enterBarrier("subject-started") + + runOn(first) { + val secondAddress = node(second).address + + val (uid, ref) = identifyWithUid(second, "subject", 5.seconds) + + enterBarrier("before-quarantined") + RARP(system).provider.transport.quarantine(node(second).address, Some(uid), "test") + + enterBarrier("quarantined") + enterBarrier("still-quarantined") + + testConductor.shutdown(second).await + + within(30.seconds) { + awaitAssert { + system.actorSelection(RootActorPath(secondAddress) / "user" / "subject") ! Identify("subject") + expectMsgType[ActorIdentity](1.second).ref.get + } + } + + system.actorSelection(RootActorPath(secondAddress) / "user" / "subject") ! "shutdown" + } + + runOn(second) { + val addr = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + val firstAddress = node(first).address + system.eventStream.subscribe(testActor, classOf[ThisActorSystemQuarantinedEvent]) + + val (firstUid, ref) = identifyWithUid(first, "subject", 5.seconds) + + enterBarrier("before-quarantined") + enterBarrier("quarantined") + + expectMsgPF(10 seconds) { + case ThisActorSystemQuarantinedEvent(local, remote) ⇒ + } + + // check that we quarantine back + val firstAssociation = RARP(system).provider.transport.asInstanceOf[ArteryTransport].association(firstAddress) + awaitAssert { + firstAssociation.associationState.isQuarantined(firstUid) + firstAssociation.associationState.isQuarantined() + } + + enterBarrier("still-quarantined") + + Await.result(system.whenTerminated, 10.seconds) + + val freshSystem = ActorSystem(system.name, ConfigFactory.parseString(s""" + akka.remote.artery.canonical.port = ${addr.port.get} + """).withFallback(system.settings.config)) + + val probe = TestProbe()(freshSystem) + + freshSystem.actorSelection(RootActorPath(firstAddress) / "user" / "subject").tell(Identify("subject"), probe.ref) + probe.expectMsgType[ActorIdentity](5.seconds).ref should not be (None) + + // Now the other system will be able to pass, too + freshSystem.actorOf(Props[Subject], "subject") + + Await.ready(freshSystem.whenTerminated, 10.seconds) + } + + } + + } +} diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/SurviveNetworkPartitionSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/SurviveNetworkPartitionSpec.scala new file mode 100644 index 0000000000..72d02c5952 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/SurviveNetworkPartitionSpec.scala @@ -0,0 +1,110 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.duration._ +import akka.actor._ +import akka.actor.ActorIdentity +import akka.actor.Identify +import akka.remote.{ RemotingMultiNodeSpec, QuarantinedEvent, RARP } +import akka.remote.testconductor.RoleName +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.remote.testkit.STMultiNodeSpec +import akka.remote.transport.ThrottlerTransportAdapter.Direction +import akka.testkit._ +import com.typesafe.config.ConfigFactory + +object SurviveNetworkPartitionSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + + commonConfig(debugConfig(on = false).withFallback( + ConfigFactory.parseString(""" + akka.loglevel = INFO + akka.remote.artery.enabled = on + akka.remote.artery.advanced.give-up-system-message-after = 4s + """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf)) + + testTransport(on = true) +} + +class SurviveNetworkPartitionSpecMultiJvmNode1 extends SurviveNetworkPartitionSpec +class SurviveNetworkPartitionSpecMultiJvmNode2 extends SurviveNetworkPartitionSpec + +abstract class SurviveNetworkPartitionSpec extends RemotingMultiNodeSpec(SurviveNetworkPartitionSpec) { + + import SurviveNetworkPartitionSpec._ + + override def initialParticipants = roles.size + + "Network partition" must { + + "not quarantine system when it heals within 'give-up-system-message-after'" taggedAs LongRunningTest in { + + runOn(second) { + system.actorOf(TestActors.echoActorProps, "echo1") + } + enterBarrier("echo-started") + + runOn(first) { + system.actorSelection(node(second) / "user" / "echo1") ! Identify(None) + val ref = expectMsgType[ActorIdentity].ref.get + ref ! "ping1" + expectMsg("ping1") + + // network partition + testConductor.blackhole(first, second, Direction.Both).await + + // send system message during network partition + watch(ref) + // keep the network partition for a while, but shorter than give-up-system-message-after + expectNoMsg(RARP(system).provider.remoteSettings.Artery.Advanced.GiveUpSystemMessageAfter - 2.second) + + // heal the network partition + testConductor.passThrough(first, second, Direction.Both).await + + // not quarantined + ref ! "ping2" + expectMsg("ping2") + + ref ! PoisonPill + expectTerminated(ref) + } + + enterBarrier("done") + } + + "quarantine system when it doesn't heal within 'give-up-system-message-after'" taggedAs LongRunningTest in { + + runOn(second) { + system.actorOf(TestActors.echoActorProps, "echo2") + } + enterBarrier("echo-started") + + runOn(first) { + val qProbe = TestProbe() + system.eventStream.subscribe(qProbe.ref, classOf[QuarantinedEvent]) + system.actorSelection(node(second) / "user" / "echo2") ! Identify(None) + val ref = expectMsgType[ActorIdentity].ref.get + ref ! "ping1" + expectMsg("ping1") + + // network partition + testConductor.blackhole(first, second, Direction.Both).await + + // send system message during network partition + watch(ref) + // keep the network partition for a while, longer than give-up-system-message-after + expectNoMsg(RARP(system).provider.remoteSettings.Artery.Advanced.GiveUpSystemMessageAfter - 1.second) + qProbe.expectMsgType[QuarantinedEvent](5.seconds).address should ===(node(second).address) + + expectTerminated(ref) + } + + enterBarrier("done") + } + + } +} diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TaskRunnerMetrics.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TaskRunnerMetrics.scala new file mode 100644 index 0000000000..c26e5b88b8 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TaskRunnerMetrics.scala @@ -0,0 +1,57 @@ +package akka.remote.artery + +import akka.actor.ActorSystem +import akka.actor.ExtendedActorSystem +import akka.remote.RemoteActorRefProvider +import org.HdrHistogram.Histogram +import java.util.concurrent.TimeUnit.SECONDS + +class TaskRunnerMetrics(system: ActorSystem) { + + private var entryOffset = 0 + + def printHistograms(): Unit = { + val aeronSourceHistogram = new Histogram(SECONDS.toNanos(10), 3) + val aeronSinkHistogram = new Histogram(SECONDS.toNanos(10), 3) + system.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].transport match { + case a: ArteryTransport ⇒ + a.afrFileChannel.foreach { afrFileChannel ⇒ + var c = 0 + var aeronSourceMaxBeforeDelegate = 0L + var aeronSinkMaxBeforeDelegate = 0L + val reader = new FlightRecorderReader(afrFileChannel) + reader.structure.hiFreqLog.logs.foreach(_.compactEntries.foreach { entry ⇒ + c += 1 + if (c > entryOffset) { + entry.code match { + case FlightRecorderEvents.AeronSource_ReturnFromTaskRunner ⇒ + aeronSourceHistogram.recordValue(entry.param) + case FlightRecorderEvents.AeronSink_ReturnFromTaskRunner ⇒ + aeronSinkHistogram.recordValue(entry.param) + case FlightRecorderEvents.AeronSource_DelegateToTaskRunner ⇒ + aeronSourceMaxBeforeDelegate = math.max(aeronSourceMaxBeforeDelegate, entry.param) + case FlightRecorderEvents.AeronSink_DelegateToTaskRunner ⇒ + aeronSinkMaxBeforeDelegate = math.max(aeronSinkMaxBeforeDelegate, entry.param) + case _ ⇒ + } + } + }) + + reader.close() + entryOffset = c + + if (aeronSourceHistogram.getTotalCount > 0) { + println(s"Histogram of AeronSource tasks in microseconds. Max count before delegate: $aeronSourceMaxBeforeDelegate") + aeronSourceHistogram.outputPercentileDistribution(System.out, 1000.0) + } + + if (aeronSinkHistogram.getTotalCount > 0) { + println(s"Histogram of AeronSink tasks in microseconds. Max count before delegate: $aeronSinkMaxBeforeDelegate") + aeronSinkHistogram.outputPercentileDistribution(System.out, 1000.0) + } + } + case _ ⇒ + } + } + +} diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TestMessage.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TestMessage.scala new file mode 100644 index 0000000000..0379ea3799 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TestMessage.scala @@ -0,0 +1,65 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.actor.ExtendedActorSystem +import akka.serialization.SerializerWithStringManifest +import akka.serialization.ByteBufferSerializer +import akka.remote.artery.protobuf.{ TestMessages ⇒ proto } +import akka.protobuf.ByteString +import java.util.concurrent.locks.LockSupport + +object TestMessage { + final case class Item(id: Long, name: String) +} + +final case class TestMessage( + id: Long, + name: String, + status: Boolean, + description: String, + payload: Array[Byte], + items: Vector[TestMessage.Item]) + +class TestMessageSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest { + + val TestMessageManifest = "A" + + override val identifier: Int = 101 + + override def manifest(o: AnyRef): String = + o match { + case _: TestMessage ⇒ TestMessageManifest + } + + override def toBinary(o: AnyRef): Array[Byte] = o match { + case msg: TestMessage ⇒ + val builder = proto.TestMessage.newBuilder() + .setId(msg.id) + .setName(msg.name) + .setDescription(msg.description) + .setStatus(msg.status) + .setPayload(ByteString.copyFrom(msg.payload)) + msg.items.foreach { item ⇒ + builder.addItems(proto.Item.newBuilder().setId(item.id).setName(item.name)) + } + builder.build().toByteArray() + } + + override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef = { + val protoMsg = proto.TestMessage.parseFrom(bytes) + import scala.collection.JavaConverters._ + val items = protoMsg.getItemsList.asScala.map { item ⇒ + TestMessage.Item(item.getId, item.getName) + }.toVector + + TestMessage( + id = protoMsg.getId, + name = protoMsg.getName, + description = protoMsg.getDescription, + status = protoMsg.getStatus, + payload = protoMsg.getPayload.toByteArray(), + items = items) + } +} diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TestRateReporter.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TestRateReporter.scala new file mode 100644 index 0000000000..6446aa7404 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TestRateReporter.scala @@ -0,0 +1,19 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.concurrent.TimeUnit.SECONDS +import java.util.concurrent.Executors + +class TestRateReporter(name: String) extends RateReporter( + SECONDS.toNanos(1), + new RateReporter.Reporter { + override def onReport(messagesPerSec: Double, bytesPerSec: Double, totalMessages: Long, totalBytes: Long): Unit = { + println(name + + f": ${messagesPerSec}%,.0f msgs/sec, ${bytesPerSec}%,.0f bytes/sec, " + + f"totals ${totalMessages}%,d messages ${totalBytes / (1024 * 1024)}%,d MB") + } + }) { + +} diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteRandomSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteRandomSpec.scala index 9270c85e25..b92d7198ee 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteRandomSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteRandomSpec.scala @@ -9,26 +9,25 @@ import akka.actor.ActorRef import akka.actor.Address import akka.actor.PoisonPill import akka.actor.Props -import akka.remote.testkit.{ STMultiNodeSpec, MultiNodeConfig, MultiNodeSpec } +import akka.remote.RemotingMultiNodeSpec +import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec } import akka.routing.Broadcast import akka.routing.RandomPool import akka.routing.RoutedActorRef import akka.testkit._ +import com.typesafe.config.ConfigFactory -object RemoteRandomMultiJvmSpec extends MultiNodeConfig { - - class SomeActor extends Actor { - def receive = { - case "hit" ⇒ sender() ! self - } - } +class RemoteRandomConfig(artery: Boolean) extends MultiNodeConfig { val first = role("first") val second = role("second") val third = role("third") val fourth = role("fourth") - commonConfig(debugConfig(on = false)) + commonConfig(debugConfig(on = false).withFallback( + ConfigFactory.parseString(s""" + akka.remote.artery.enabled = $artery + """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf)) deployOnAll(""" /service-hello { @@ -39,14 +38,28 @@ object RemoteRandomMultiJvmSpec extends MultiNodeConfig { """) } -class RemoteRandomMultiJvmNode1 extends RemoteRandomSpec -class RemoteRandomMultiJvmNode2 extends RemoteRandomSpec -class RemoteRandomMultiJvmNode3 extends RemoteRandomSpec -class RemoteRandomMultiJvmNode4 extends RemoteRandomSpec +class RemoteRandomMultiJvmNode1 extends RemoteRandomSpec(new RemoteRandomConfig(artery = false)) +class RemoteRandomMultiJvmNode2 extends RemoteRandomSpec(new RemoteRandomConfig(artery = false)) +class RemoteRandomMultiJvmNode3 extends RemoteRandomSpec(new RemoteRandomConfig(artery = false)) +class RemoteRandomMultiJvmNode4 extends RemoteRandomSpec(new RemoteRandomConfig(artery = false)) -class RemoteRandomSpec extends MultiNodeSpec(RemoteRandomMultiJvmSpec) - with STMultiNodeSpec with ImplicitSender with DefaultTimeout { - import RemoteRandomMultiJvmSpec._ +class ArteryRemoteRandomMultiJvmNode1 extends RemoteRandomSpec(new RemoteRandomConfig(artery = true)) +class ArteryRemoteRandomMultiJvmNode2 extends RemoteRandomSpec(new RemoteRandomConfig(artery = true)) +class ArteryRemoteRandomMultiJvmNode3 extends RemoteRandomSpec(new RemoteRandomConfig(artery = true)) +class ArteryRemoteRandomMultiJvmNode4 extends RemoteRandomSpec(new RemoteRandomConfig(artery = true)) + +object RemoteRandomSpec { + class SomeActor extends Actor { + def receive = { + case "hit" ⇒ sender() ! self + } + } +} + +class RemoteRandomSpec(multiNodeConfig: RemoteRandomConfig) extends RemotingMultiNodeSpec(multiNodeConfig) + with DefaultTimeout { + import multiNodeConfig._ + import RemoteRandomSpec._ def initialParticipants = roles.size @@ -90,4 +103,4 @@ class RemoteRandomSpec extends MultiNodeSpec(RemoteRandomMultiJvmSpec) } } } -} \ No newline at end of file +} diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteRoundRobinSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteRoundRobinSpec.scala index c80789bd0c..ba56c4b623 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteRoundRobinSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteRoundRobinSpec.scala @@ -3,47 +3,30 @@ */ package akka.remote.routing -import language.postfixOps -import scala.collection.immutable -import akka.actor.Actor -import akka.actor.ActorRef -import akka.actor.Props -import akka.actor.PoisonPill -import akka.actor.Address -import scala.concurrent.Await +import akka.actor.{ Actor, ActorRef, Address, PoisonPill, Props } import akka.pattern.ask -import akka.remote.testkit.{ STMultiNodeSpec, MultiNodeConfig, MultiNodeSpec } -import akka.routing.Broadcast -import akka.routing.GetRoutees -import akka.routing.Routees -import akka.routing.RoundRobinPool -import akka.routing.RoundRobinGroup -import akka.routing.RoutedActorRef -import akka.routing.Resizer -import akka.routing.Routee -import akka.routing.FromConfig +import akka.remote.RemotingMultiNodeSpec +import akka.remote.testkit.MultiNodeConfig +import akka.routing._ import akka.testkit._ +import com.typesafe.config.ConfigFactory + +import scala.collection.immutable +import scala.concurrent.Await import scala.concurrent.duration._ +import scala.language.postfixOps -object RemoteRoundRobinMultiJvmSpec extends MultiNodeConfig { - - class SomeActor extends Actor { - def receive = { - case "hit" ⇒ sender() ! self - } - } - - class TestResizer extends Resizer { - override def isTimeForResize(messageCounter: Long): Boolean = messageCounter <= 10 - override def resize(currentRoutees: immutable.IndexedSeq[Routee]): Int = 1 - } +class RemoteRoundRobinConfig(artery: Boolean) extends MultiNodeConfig { val first = role("first") val second = role("second") val third = role("third") val fourth = role("fourth") - commonConfig(debugConfig(on = false)) + commonConfig(debugConfig(on = false).withFallback( + ConfigFactory.parseString(s""" + akka.remote.artery.enabled = $artery + """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf)) deployOnAll(""" /service-hello { @@ -56,7 +39,7 @@ object RemoteRoundRobinMultiJvmSpec extends MultiNodeConfig { router = round-robin-pool target.nodes = ["@first@", "@second@", "@third@"] } - + /service-hello3 { router = round-robin-group routees.paths = [ @@ -67,14 +50,33 @@ object RemoteRoundRobinMultiJvmSpec extends MultiNodeConfig { """) } -class RemoteRoundRobinMultiJvmNode1 extends RemoteRoundRobinSpec -class RemoteRoundRobinMultiJvmNode2 extends RemoteRoundRobinSpec -class RemoteRoundRobinMultiJvmNode3 extends RemoteRoundRobinSpec -class RemoteRoundRobinMultiJvmNode4 extends RemoteRoundRobinSpec +class RemoteRoundRobinMultiJvmNode1 extends RemoteRoundRobinSpec(new RemoteRoundRobinConfig(artery = false)) +class RemoteRoundRobinMultiJvmNode2 extends RemoteRoundRobinSpec(new RemoteRoundRobinConfig(artery = false)) +class RemoteRoundRobinMultiJvmNode3 extends RemoteRoundRobinSpec(new RemoteRoundRobinConfig(artery = false)) +class RemoteRoundRobinMultiJvmNode4 extends RemoteRoundRobinSpec(new RemoteRoundRobinConfig(artery = false)) -class RemoteRoundRobinSpec extends MultiNodeSpec(RemoteRoundRobinMultiJvmSpec) - with STMultiNodeSpec with ImplicitSender with DefaultTimeout { - import RemoteRoundRobinMultiJvmSpec._ +class ArteryRemoteRoundRobinMultiJvmNode1 extends RemoteRoundRobinSpec(new RemoteRoundRobinConfig(artery = true)) +class ArteryRemoteRoundRobinMultiJvmNode2 extends RemoteRoundRobinSpec(new RemoteRoundRobinConfig(artery = true)) +class ArteryRemoteRoundRobinMultiJvmNode3 extends RemoteRoundRobinSpec(new RemoteRoundRobinConfig(artery = true)) +class ArteryRemoteRoundRobinMultiJvmNode4 extends RemoteRoundRobinSpec(new RemoteRoundRobinConfig(artery = true)) + +object RemoteRoundRobinSpec { + class SomeActor extends Actor { + def receive = { + case "hit" ⇒ sender() ! self + } + } + + class TestResizer extends Resizer { + override def isTimeForResize(messageCounter: Long): Boolean = messageCounter <= 10 + override def resize(currentRoutees: immutable.IndexedSeq[Routee]): Int = 1 + } +} + +class RemoteRoundRobinSpec(multiNodeConfig: RemoteRoundRobinConfig) extends RemotingMultiNodeSpec(multiNodeConfig) + with DefaultTimeout { + import RemoteRoundRobinSpec._ + import multiNodeConfig._ def initialParticipants = roles.size diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteScatterGatherSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteScatterGatherSpec.scala index 6f0fdb4412..62eb30e579 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteScatterGatherSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/routing/RemoteScatterGatherSpec.scala @@ -9,27 +9,26 @@ import akka.actor.ActorRef import akka.actor.Address import akka.actor.PoisonPill import akka.actor.Props -import akka.remote.testkit.{ STMultiNodeSpec, MultiNodeConfig, MultiNodeSpec } +import akka.remote.RemotingMultiNodeSpec +import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec } import akka.routing.Broadcast import akka.routing.ScatterGatherFirstCompletedPool import akka.routing.RoutedActorRef import akka.testkit._ import akka.testkit.TestEvent._ +import com.typesafe.config.ConfigFactory -object RemoteScatterGatherMultiJvmSpec extends MultiNodeConfig { - - class SomeActor extends Actor { - def receive = { - case "hit" ⇒ sender() ! self - } - } +class RemoteScatterGatherConfig(artery: Boolean) extends MultiNodeConfig { val first = role("first") val second = role("second") val third = role("third") val fourth = role("fourth") - commonConfig(debugConfig(on = false)) + commonConfig(debugConfig(on = false).withFallback( + ConfigFactory.parseString(s""" + akka.remote.artery.enabled = $artery + """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf)) deployOnAll(""" /service-hello { @@ -40,14 +39,28 @@ object RemoteScatterGatherMultiJvmSpec extends MultiNodeConfig { """) } -class RemoteScatterGatherMultiJvmNode1 extends RemoteScatterGatherSpec -class RemoteScatterGatherMultiJvmNode2 extends RemoteScatterGatherSpec -class RemoteScatterGatherMultiJvmNode3 extends RemoteScatterGatherSpec -class RemoteScatterGatherMultiJvmNode4 extends RemoteScatterGatherSpec +class RemoteScatterGatherMultiJvmNode1 extends RemoteScatterGatherSpec(new RemoteScatterGatherConfig(artery = false)) +class RemoteScatterGatherMultiJvmNode2 extends RemoteScatterGatherSpec(new RemoteScatterGatherConfig(artery = false)) +class RemoteScatterGatherMultiJvmNode3 extends RemoteScatterGatherSpec(new RemoteScatterGatherConfig(artery = false)) +class RemoteScatterGatherMultiJvmNode4 extends RemoteScatterGatherSpec(new RemoteScatterGatherConfig(artery = false)) -class RemoteScatterGatherSpec extends MultiNodeSpec(RemoteScatterGatherMultiJvmSpec) - with STMultiNodeSpec with ImplicitSender with DefaultTimeout { - import RemoteScatterGatherMultiJvmSpec._ +class ArteryRemoteScatterGatherMultiJvmNode1 extends RemoteScatterGatherSpec(new RemoteScatterGatherConfig(artery = true)) +class ArteryRemoteScatterGatherMultiJvmNode2 extends RemoteScatterGatherSpec(new RemoteScatterGatherConfig(artery = true)) +class ArteryRemoteScatterGatherMultiJvmNode3 extends RemoteScatterGatherSpec(new RemoteScatterGatherConfig(artery = true)) +class ArteryRemoteScatterGatherMultiJvmNode4 extends RemoteScatterGatherSpec(new RemoteScatterGatherConfig(artery = true)) + +object RemoteScatterGatherSpec { + class SomeActor extends Actor { + def receive = { + case "hit" ⇒ sender() ! self + } + } +} + +class RemoteScatterGatherSpec(multiNodeConfig: RemoteScatterGatherConfig) extends RemotingMultiNodeSpec(multiNodeConfig) + with DefaultTimeout { + import multiNodeConfig._ + import RemoteScatterGatherSpec._ def initialParticipants = roles.size diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/testconductor/TestConductorSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/testconductor/TestConductorSpec.scala index a52ee6f10f..8bbe6562d4 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/testconductor/TestConductorSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/testconductor/TestConductorSpec.scala @@ -5,7 +5,8 @@ package akka.remote.testconductor import language.postfixOps import com.typesafe.config.ConfigFactory -import akka.actor.{ Props, Actor, ActorIdentity, Identify, Deploy } +import akka.actor.{ Actor, ActorIdentity, Deploy, Identify, Props } + import scala.concurrent.Await import scala.concurrent.Awaitable import scala.concurrent.duration._ @@ -13,11 +14,13 @@ import akka.testkit.ImplicitSender import akka.testkit.LongRunningTest import java.net.InetSocketAddress import java.net.InetAddress -import akka.remote.testkit.{ STMultiNodeSpec, MultiNodeSpec, MultiNodeConfig } + +import akka.remote.RemotingMultiNodeSpec +import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec } import akka.remote.transport.ThrottlerTransportAdapter.Direction object TestConductorMultiJvmSpec extends MultiNodeConfig { - commonConfig(debugConfig(on = false)) + commonConfig(debugConfig(on = false).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf)) val master = role("master") val slave = role("slave") @@ -28,7 +31,7 @@ object TestConductorMultiJvmSpec extends MultiNodeConfig { class TestConductorMultiJvmNode1 extends TestConductorSpec class TestConductorMultiJvmNode2 extends TestConductorSpec -class TestConductorSpec extends MultiNodeSpec(TestConductorMultiJvmSpec) with STMultiNodeSpec with ImplicitSender { +class TestConductorSpec extends RemotingMultiNodeSpec(TestConductorMultiJvmSpec) { import TestConductorMultiJvmSpec._ diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/testkit/MultiNodeSpecSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/testkit/MultiNodeSpecSpec.scala index ba7b00a469..887f6c4a68 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/testkit/MultiNodeSpecSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/testkit/MultiNodeSpecSpec.scala @@ -3,10 +3,11 @@ */ package akka.remote.testkit +import akka.remote.RemotingMultiNodeSpec import akka.testkit.LongRunningTest object MultiNodeSpecMultiJvmSpec extends MultiNodeConfig { - commonConfig(debugConfig(on = false)) + commonConfig(debugConfig(on = false).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf)) val node1 = role("node1") val node2 = role("node2") @@ -19,7 +20,7 @@ class MultiNodeSpecSpecMultiJvmNode2 extends MultiNodeSpecSpec class MultiNodeSpecSpecMultiJvmNode3 extends MultiNodeSpecSpec class MultiNodeSpecSpecMultiJvmNode4 extends MultiNodeSpecSpec -class MultiNodeSpecSpec extends MultiNodeSpec(MultiNodeSpecMultiJvmSpec) with STMultiNodeSpec { +class MultiNodeSpecSpec extends RemotingMultiNodeSpec(MultiNodeSpecMultiJvmSpec) { import MultiNodeSpecMultiJvmSpec._ diff --git a/akka-remote-tests/src/test/java/akka/remote/artery/protobuf/TestMessages.java b/akka-remote-tests/src/test/java/akka/remote/artery/protobuf/TestMessages.java new file mode 100644 index 0000000000..bde5d4fd81 --- /dev/null +++ b/akka-remote-tests/src/test/java/akka/remote/artery/protobuf/TestMessages.java @@ -0,0 +1,1903 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: TestMessages.proto + +package akka.remote.artery.protobuf; + +public final class TestMessages { + private TestMessages() {} + public static void registerAllExtensions( + akka.protobuf.ExtensionRegistry registry) { + } + public interface TestMessageOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required uint64 id = 1; + /** + * required uint64 id = 1; + */ + boolean hasId(); + /** + * required uint64 id = 1; + */ + long getId(); + + // required string name = 2; + /** + * required string name = 2; + */ + boolean hasName(); + /** + * required string name = 2; + */ + java.lang.String getName(); + /** + * required string name = 2; + */ + akka.protobuf.ByteString + getNameBytes(); + + // required bool status = 3; + /** + * required bool status = 3; + */ + boolean hasStatus(); + /** + * required bool status = 3; + */ + boolean getStatus(); + + // optional string description = 4; + /** + * optional string description = 4; + */ + boolean hasDescription(); + /** + * optional string description = 4; + */ + java.lang.String getDescription(); + /** + * optional string description = 4; + */ + akka.protobuf.ByteString + getDescriptionBytes(); + + // optional bytes payload = 5; + /** + * optional bytes payload = 5; + */ + boolean hasPayload(); + /** + * optional bytes payload = 5; + */ + akka.protobuf.ByteString getPayload(); + + // repeated .Item items = 6; + /** + * repeated .Item items = 6; + */ + java.util.List + getItemsList(); + /** + * repeated .Item items = 6; + */ + akka.remote.artery.protobuf.TestMessages.Item getItems(int index); + /** + * repeated .Item items = 6; + */ + int getItemsCount(); + /** + * repeated .Item items = 6; + */ + java.util.List + getItemsOrBuilderList(); + /** + * repeated .Item items = 6; + */ + akka.remote.artery.protobuf.TestMessages.ItemOrBuilder getItemsOrBuilder( + int index); + } + /** + * Protobuf type {@code TestMessage} + */ + public static final class TestMessage extends + akka.protobuf.GeneratedMessage + implements TestMessageOrBuilder { + // Use TestMessage.newBuilder() to construct. + private TestMessage(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private TestMessage(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final TestMessage defaultInstance; + public static TestMessage getDefaultInstance() { + return defaultInstance; + } + + public TestMessage getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private TestMessage( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + id_ = input.readUInt64(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + name_ = input.readBytes(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + status_ = input.readBool(); + break; + } + case 34: { + bitField0_ |= 0x00000008; + description_ = input.readBytes(); + break; + } + case 42: { + bitField0_ |= 0x00000010; + payload_ = input.readBytes(); + break; + } + case 50: { + if (!((mutable_bitField0_ & 0x00000020) == 0x00000020)) { + items_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000020; + } + items_.add(input.readMessage(akka.remote.artery.protobuf.TestMessages.Item.PARSER, extensionRegistry)); + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000020) == 0x00000020)) { + items_ = java.util.Collections.unmodifiableList(items_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.artery.protobuf.TestMessages.internal_static_TestMessage_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.artery.protobuf.TestMessages.internal_static_TestMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.artery.protobuf.TestMessages.TestMessage.class, akka.remote.artery.protobuf.TestMessages.TestMessage.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public TestMessage parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new TestMessage(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required uint64 id = 1; + public static final int ID_FIELD_NUMBER = 1; + private long id_; + /** + * required uint64 id = 1; + */ + public boolean hasId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required uint64 id = 1; + */ + public long getId() { + return id_; + } + + // required string name = 2; + public static final int NAME_FIELD_NUMBER = 2; + private java.lang.Object name_; + /** + * required string name = 2; + */ + public boolean hasName() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required string name = 2; + */ + public java.lang.String getName() { + java.lang.Object ref = name_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + akka.protobuf.ByteString bs = + (akka.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + name_ = s; + } + return s; + } + } + /** + * required string name = 2; + */ + public akka.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = name_; + if (ref instanceof java.lang.String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + name_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + + // required bool status = 3; + public static final int STATUS_FIELD_NUMBER = 3; + private boolean status_; + /** + * required bool status = 3; + */ + public boolean hasStatus() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required bool status = 3; + */ + public boolean getStatus() { + return status_; + } + + // optional string description = 4; + public static final int DESCRIPTION_FIELD_NUMBER = 4; + private java.lang.Object description_; + /** + * optional string description = 4; + */ + public boolean hasDescription() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional string description = 4; + */ + public java.lang.String getDescription() { + java.lang.Object ref = description_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + akka.protobuf.ByteString bs = + (akka.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + description_ = s; + } + return s; + } + } + /** + * optional string description = 4; + */ + public akka.protobuf.ByteString + getDescriptionBytes() { + java.lang.Object ref = description_; + if (ref instanceof java.lang.String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + description_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + + // optional bytes payload = 5; + public static final int PAYLOAD_FIELD_NUMBER = 5; + private akka.protobuf.ByteString payload_; + /** + * optional bytes payload = 5; + */ + public boolean hasPayload() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + /** + * optional bytes payload = 5; + */ + public akka.protobuf.ByteString getPayload() { + return payload_; + } + + // repeated .Item items = 6; + public static final int ITEMS_FIELD_NUMBER = 6; + private java.util.List items_; + /** + * repeated .Item items = 6; + */ + public java.util.List getItemsList() { + return items_; + } + /** + * repeated .Item items = 6; + */ + public java.util.List + getItemsOrBuilderList() { + return items_; + } + /** + * repeated .Item items = 6; + */ + public int getItemsCount() { + return items_.size(); + } + /** + * repeated .Item items = 6; + */ + public akka.remote.artery.protobuf.TestMessages.Item getItems(int index) { + return items_.get(index); + } + /** + * repeated .Item items = 6; + */ + public akka.remote.artery.protobuf.TestMessages.ItemOrBuilder getItemsOrBuilder( + int index) { + return items_.get(index); + } + + private void initFields() { + id_ = 0L; + name_ = ""; + status_ = false; + description_ = ""; + payload_ = akka.protobuf.ByteString.EMPTY; + items_ = java.util.Collections.emptyList(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasName()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasStatus()) { + memoizedIsInitialized = 0; + return false; + } + for (int i = 0; i < getItemsCount(); i++) { + if (!getItems(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeUInt64(1, id_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getNameBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBool(3, status_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(4, getDescriptionBytes()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBytes(5, payload_); + } + for (int i = 0; i < items_.size(); i++) { + output.writeMessage(6, items_.get(i)); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeUInt64Size(1, id_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += akka.protobuf.CodedOutputStream + .computeBytesSize(2, getNameBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += akka.protobuf.CodedOutputStream + .computeBoolSize(3, status_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += akka.protobuf.CodedOutputStream + .computeBytesSize(4, getDescriptionBytes()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += akka.protobuf.CodedOutputStream + .computeBytesSize(5, payload_); + } + for (int i = 0; i < items_.size(); i++) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(6, items_.get(i)); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.artery.protobuf.TestMessages.TestMessage parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.artery.protobuf.TestMessages.TestMessage parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.artery.protobuf.TestMessages.TestMessage parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.artery.protobuf.TestMessages.TestMessage parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.artery.protobuf.TestMessages.TestMessage parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.artery.protobuf.TestMessages.TestMessage parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.artery.protobuf.TestMessages.TestMessage parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.artery.protobuf.TestMessages.TestMessage parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.artery.protobuf.TestMessages.TestMessage parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.artery.protobuf.TestMessages.TestMessage parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.artery.protobuf.TestMessages.TestMessage prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code TestMessage} + */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.artery.protobuf.TestMessages.TestMessageOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.artery.protobuf.TestMessages.internal_static_TestMessage_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.artery.protobuf.TestMessages.internal_static_TestMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.artery.protobuf.TestMessages.TestMessage.class, akka.remote.artery.protobuf.TestMessages.TestMessage.Builder.class); + } + + // Construct using akka.remote.artery.protobuf.TestMessages.TestMessage.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getItemsFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + id_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + name_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + status_ = false; + bitField0_ = (bitField0_ & ~0x00000004); + description_ = ""; + bitField0_ = (bitField0_ & ~0x00000008); + payload_ = akka.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000010); + if (itemsBuilder_ == null) { + items_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000020); + } else { + itemsBuilder_.clear(); + } + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.artery.protobuf.TestMessages.internal_static_TestMessage_descriptor; + } + + public akka.remote.artery.protobuf.TestMessages.TestMessage getDefaultInstanceForType() { + return akka.remote.artery.protobuf.TestMessages.TestMessage.getDefaultInstance(); + } + + public akka.remote.artery.protobuf.TestMessages.TestMessage build() { + akka.remote.artery.protobuf.TestMessages.TestMessage result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.artery.protobuf.TestMessages.TestMessage buildPartial() { + akka.remote.artery.protobuf.TestMessages.TestMessage result = new akka.remote.artery.protobuf.TestMessages.TestMessage(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.id_ = id_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.name_ = name_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.status_ = status_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.description_ = description_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.payload_ = payload_; + if (itemsBuilder_ == null) { + if (((bitField0_ & 0x00000020) == 0x00000020)) { + items_ = java.util.Collections.unmodifiableList(items_); + bitField0_ = (bitField0_ & ~0x00000020); + } + result.items_ = items_; + } else { + result.items_ = itemsBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.artery.protobuf.TestMessages.TestMessage) { + return mergeFrom((akka.remote.artery.protobuf.TestMessages.TestMessage)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.artery.protobuf.TestMessages.TestMessage other) { + if (other == akka.remote.artery.protobuf.TestMessages.TestMessage.getDefaultInstance()) return this; + if (other.hasId()) { + setId(other.getId()); + } + if (other.hasName()) { + bitField0_ |= 0x00000002; + name_ = other.name_; + onChanged(); + } + if (other.hasStatus()) { + setStatus(other.getStatus()); + } + if (other.hasDescription()) { + bitField0_ |= 0x00000008; + description_ = other.description_; + onChanged(); + } + if (other.hasPayload()) { + setPayload(other.getPayload()); + } + if (itemsBuilder_ == null) { + if (!other.items_.isEmpty()) { + if (items_.isEmpty()) { + items_ = other.items_; + bitField0_ = (bitField0_ & ~0x00000020); + } else { + ensureItemsIsMutable(); + items_.addAll(other.items_); + } + onChanged(); + } + } else { + if (!other.items_.isEmpty()) { + if (itemsBuilder_.isEmpty()) { + itemsBuilder_.dispose(); + itemsBuilder_ = null; + items_ = other.items_; + bitField0_ = (bitField0_ & ~0x00000020); + itemsBuilder_ = + akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getItemsFieldBuilder() : null; + } else { + itemsBuilder_.addAllMessages(other.items_); + } + } + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasId()) { + + return false; + } + if (!hasName()) { + + return false; + } + if (!hasStatus()) { + + return false; + } + for (int i = 0; i < getItemsCount(); i++) { + if (!getItems(i).isInitialized()) { + + return false; + } + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.artery.protobuf.TestMessages.TestMessage parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.artery.protobuf.TestMessages.TestMessage) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required uint64 id = 1; + private long id_ ; + /** + * required uint64 id = 1; + */ + public boolean hasId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required uint64 id = 1; + */ + public long getId() { + return id_; + } + /** + * required uint64 id = 1; + */ + public Builder setId(long value) { + bitField0_ |= 0x00000001; + id_ = value; + onChanged(); + return this; + } + /** + * required uint64 id = 1; + */ + public Builder clearId() { + bitField0_ = (bitField0_ & ~0x00000001); + id_ = 0L; + onChanged(); + return this; + } + + // required string name = 2; + private java.lang.Object name_ = ""; + /** + * required string name = 2; + */ + public boolean hasName() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required string name = 2; + */ + public java.lang.String getName() { + java.lang.Object ref = name_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((akka.protobuf.ByteString) ref) + .toStringUtf8(); + name_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string name = 2; + */ + public akka.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = name_; + if (ref instanceof String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + name_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + /** + * required string name = 2; + */ + public Builder setName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + name_ = value; + onChanged(); + return this; + } + /** + * required string name = 2; + */ + public Builder clearName() { + bitField0_ = (bitField0_ & ~0x00000002); + name_ = getDefaultInstance().getName(); + onChanged(); + return this; + } + /** + * required string name = 2; + */ + public Builder setNameBytes( + akka.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + name_ = value; + onChanged(); + return this; + } + + // required bool status = 3; + private boolean status_ ; + /** + * required bool status = 3; + */ + public boolean hasStatus() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required bool status = 3; + */ + public boolean getStatus() { + return status_; + } + /** + * required bool status = 3; + */ + public Builder setStatus(boolean value) { + bitField0_ |= 0x00000004; + status_ = value; + onChanged(); + return this; + } + /** + * required bool status = 3; + */ + public Builder clearStatus() { + bitField0_ = (bitField0_ & ~0x00000004); + status_ = false; + onChanged(); + return this; + } + + // optional string description = 4; + private java.lang.Object description_ = ""; + /** + * optional string description = 4; + */ + public boolean hasDescription() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional string description = 4; + */ + public java.lang.String getDescription() { + java.lang.Object ref = description_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((akka.protobuf.ByteString) ref) + .toStringUtf8(); + description_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * optional string description = 4; + */ + public akka.protobuf.ByteString + getDescriptionBytes() { + java.lang.Object ref = description_; + if (ref instanceof String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + description_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + /** + * optional string description = 4; + */ + public Builder setDescription( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + description_ = value; + onChanged(); + return this; + } + /** + * optional string description = 4; + */ + public Builder clearDescription() { + bitField0_ = (bitField0_ & ~0x00000008); + description_ = getDefaultInstance().getDescription(); + onChanged(); + return this; + } + /** + * optional string description = 4; + */ + public Builder setDescriptionBytes( + akka.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + description_ = value; + onChanged(); + return this; + } + + // optional bytes payload = 5; + private akka.protobuf.ByteString payload_ = akka.protobuf.ByteString.EMPTY; + /** + * optional bytes payload = 5; + */ + public boolean hasPayload() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + /** + * optional bytes payload = 5; + */ + public akka.protobuf.ByteString getPayload() { + return payload_; + } + /** + * optional bytes payload = 5; + */ + public Builder setPayload(akka.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + payload_ = value; + onChanged(); + return this; + } + /** + * optional bytes payload = 5; + */ + public Builder clearPayload() { + bitField0_ = (bitField0_ & ~0x00000010); + payload_ = getDefaultInstance().getPayload(); + onChanged(); + return this; + } + + // repeated .Item items = 6; + private java.util.List items_ = + java.util.Collections.emptyList(); + private void ensureItemsIsMutable() { + if (!((bitField0_ & 0x00000020) == 0x00000020)) { + items_ = new java.util.ArrayList(items_); + bitField0_ |= 0x00000020; + } + } + + private akka.protobuf.RepeatedFieldBuilder< + akka.remote.artery.protobuf.TestMessages.Item, akka.remote.artery.protobuf.TestMessages.Item.Builder, akka.remote.artery.protobuf.TestMessages.ItemOrBuilder> itemsBuilder_; + + /** + * repeated .Item items = 6; + */ + public java.util.List getItemsList() { + if (itemsBuilder_ == null) { + return java.util.Collections.unmodifiableList(items_); + } else { + return itemsBuilder_.getMessageList(); + } + } + /** + * repeated .Item items = 6; + */ + public int getItemsCount() { + if (itemsBuilder_ == null) { + return items_.size(); + } else { + return itemsBuilder_.getCount(); + } + } + /** + * repeated .Item items = 6; + */ + public akka.remote.artery.protobuf.TestMessages.Item getItems(int index) { + if (itemsBuilder_ == null) { + return items_.get(index); + } else { + return itemsBuilder_.getMessage(index); + } + } + /** + * repeated .Item items = 6; + */ + public Builder setItems( + int index, akka.remote.artery.protobuf.TestMessages.Item value) { + if (itemsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureItemsIsMutable(); + items_.set(index, value); + onChanged(); + } else { + itemsBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .Item items = 6; + */ + public Builder setItems( + int index, akka.remote.artery.protobuf.TestMessages.Item.Builder builderForValue) { + if (itemsBuilder_ == null) { + ensureItemsIsMutable(); + items_.set(index, builderForValue.build()); + onChanged(); + } else { + itemsBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .Item items = 6; + */ + public Builder addItems(akka.remote.artery.protobuf.TestMessages.Item value) { + if (itemsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureItemsIsMutable(); + items_.add(value); + onChanged(); + } else { + itemsBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .Item items = 6; + */ + public Builder addItems( + int index, akka.remote.artery.protobuf.TestMessages.Item value) { + if (itemsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureItemsIsMutable(); + items_.add(index, value); + onChanged(); + } else { + itemsBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .Item items = 6; + */ + public Builder addItems( + akka.remote.artery.protobuf.TestMessages.Item.Builder builderForValue) { + if (itemsBuilder_ == null) { + ensureItemsIsMutable(); + items_.add(builderForValue.build()); + onChanged(); + } else { + itemsBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .Item items = 6; + */ + public Builder addItems( + int index, akka.remote.artery.protobuf.TestMessages.Item.Builder builderForValue) { + if (itemsBuilder_ == null) { + ensureItemsIsMutable(); + items_.add(index, builderForValue.build()); + onChanged(); + } else { + itemsBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .Item items = 6; + */ + public Builder addAllItems( + java.lang.Iterable values) { + if (itemsBuilder_ == null) { + ensureItemsIsMutable(); + super.addAll(values, items_); + onChanged(); + } else { + itemsBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .Item items = 6; + */ + public Builder clearItems() { + if (itemsBuilder_ == null) { + items_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000020); + onChanged(); + } else { + itemsBuilder_.clear(); + } + return this; + } + /** + * repeated .Item items = 6; + */ + public Builder removeItems(int index) { + if (itemsBuilder_ == null) { + ensureItemsIsMutable(); + items_.remove(index); + onChanged(); + } else { + itemsBuilder_.remove(index); + } + return this; + } + /** + * repeated .Item items = 6; + */ + public akka.remote.artery.protobuf.TestMessages.Item.Builder getItemsBuilder( + int index) { + return getItemsFieldBuilder().getBuilder(index); + } + /** + * repeated .Item items = 6; + */ + public akka.remote.artery.protobuf.TestMessages.ItemOrBuilder getItemsOrBuilder( + int index) { + if (itemsBuilder_ == null) { + return items_.get(index); } else { + return itemsBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .Item items = 6; + */ + public java.util.List + getItemsOrBuilderList() { + if (itemsBuilder_ != null) { + return itemsBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(items_); + } + } + /** + * repeated .Item items = 6; + */ + public akka.remote.artery.protobuf.TestMessages.Item.Builder addItemsBuilder() { + return getItemsFieldBuilder().addBuilder( + akka.remote.artery.protobuf.TestMessages.Item.getDefaultInstance()); + } + /** + * repeated .Item items = 6; + */ + public akka.remote.artery.protobuf.TestMessages.Item.Builder addItemsBuilder( + int index) { + return getItemsFieldBuilder().addBuilder( + index, akka.remote.artery.protobuf.TestMessages.Item.getDefaultInstance()); + } + /** + * repeated .Item items = 6; + */ + public java.util.List + getItemsBuilderList() { + return getItemsFieldBuilder().getBuilderList(); + } + private akka.protobuf.RepeatedFieldBuilder< + akka.remote.artery.protobuf.TestMessages.Item, akka.remote.artery.protobuf.TestMessages.Item.Builder, akka.remote.artery.protobuf.TestMessages.ItemOrBuilder> + getItemsFieldBuilder() { + if (itemsBuilder_ == null) { + itemsBuilder_ = new akka.protobuf.RepeatedFieldBuilder< + akka.remote.artery.protobuf.TestMessages.Item, akka.remote.artery.protobuf.TestMessages.Item.Builder, akka.remote.artery.protobuf.TestMessages.ItemOrBuilder>( + items_, + ((bitField0_ & 0x00000020) == 0x00000020), + getParentForChildren(), + isClean()); + items_ = null; + } + return itemsBuilder_; + } + + // @@protoc_insertion_point(builder_scope:TestMessage) + } + + static { + defaultInstance = new TestMessage(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:TestMessage) + } + + public interface ItemOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required uint64 id = 1; + /** + * required uint64 id = 1; + */ + boolean hasId(); + /** + * required uint64 id = 1; + */ + long getId(); + + // required string name = 2; + /** + * required string name = 2; + */ + boolean hasName(); + /** + * required string name = 2; + */ + java.lang.String getName(); + /** + * required string name = 2; + */ + akka.protobuf.ByteString + getNameBytes(); + } + /** + * Protobuf type {@code Item} + */ + public static final class Item extends + akka.protobuf.GeneratedMessage + implements ItemOrBuilder { + // Use Item.newBuilder() to construct. + private Item(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private Item(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final Item defaultInstance; + public static Item getDefaultInstance() { + return defaultInstance; + } + + public Item getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private Item( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + id_ = input.readUInt64(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + name_ = input.readBytes(); + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.artery.protobuf.TestMessages.internal_static_Item_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.artery.protobuf.TestMessages.internal_static_Item_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.artery.protobuf.TestMessages.Item.class, akka.remote.artery.protobuf.TestMessages.Item.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public Item parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new Item(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required uint64 id = 1; + public static final int ID_FIELD_NUMBER = 1; + private long id_; + /** + * required uint64 id = 1; + */ + public boolean hasId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required uint64 id = 1; + */ + public long getId() { + return id_; + } + + // required string name = 2; + public static final int NAME_FIELD_NUMBER = 2; + private java.lang.Object name_; + /** + * required string name = 2; + */ + public boolean hasName() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required string name = 2; + */ + public java.lang.String getName() { + java.lang.Object ref = name_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + akka.protobuf.ByteString bs = + (akka.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + name_ = s; + } + return s; + } + } + /** + * required string name = 2; + */ + public akka.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = name_; + if (ref instanceof java.lang.String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + name_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + + private void initFields() { + id_ = 0L; + name_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasName()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeUInt64(1, id_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getNameBytes()); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeUInt64Size(1, id_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += akka.protobuf.CodedOutputStream + .computeBytesSize(2, getNameBytes()); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.artery.protobuf.TestMessages.Item parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.artery.protobuf.TestMessages.Item parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.artery.protobuf.TestMessages.Item parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.artery.protobuf.TestMessages.Item parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.artery.protobuf.TestMessages.Item parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.artery.protobuf.TestMessages.Item parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.artery.protobuf.TestMessages.Item parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.artery.protobuf.TestMessages.Item parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.artery.protobuf.TestMessages.Item parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.artery.protobuf.TestMessages.Item parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.artery.protobuf.TestMessages.Item prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code Item} + */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.artery.protobuf.TestMessages.ItemOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.artery.protobuf.TestMessages.internal_static_Item_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.artery.protobuf.TestMessages.internal_static_Item_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.artery.protobuf.TestMessages.Item.class, akka.remote.artery.protobuf.TestMessages.Item.Builder.class); + } + + // Construct using akka.remote.artery.protobuf.TestMessages.Item.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + id_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + name_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.artery.protobuf.TestMessages.internal_static_Item_descriptor; + } + + public akka.remote.artery.protobuf.TestMessages.Item getDefaultInstanceForType() { + return akka.remote.artery.protobuf.TestMessages.Item.getDefaultInstance(); + } + + public akka.remote.artery.protobuf.TestMessages.Item build() { + akka.remote.artery.protobuf.TestMessages.Item result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.artery.protobuf.TestMessages.Item buildPartial() { + akka.remote.artery.protobuf.TestMessages.Item result = new akka.remote.artery.protobuf.TestMessages.Item(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.id_ = id_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.name_ = name_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.artery.protobuf.TestMessages.Item) { + return mergeFrom((akka.remote.artery.protobuf.TestMessages.Item)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.artery.protobuf.TestMessages.Item other) { + if (other == akka.remote.artery.protobuf.TestMessages.Item.getDefaultInstance()) return this; + if (other.hasId()) { + setId(other.getId()); + } + if (other.hasName()) { + bitField0_ |= 0x00000002; + name_ = other.name_; + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasId()) { + + return false; + } + if (!hasName()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.artery.protobuf.TestMessages.Item parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.artery.protobuf.TestMessages.Item) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required uint64 id = 1; + private long id_ ; + /** + * required uint64 id = 1; + */ + public boolean hasId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required uint64 id = 1; + */ + public long getId() { + return id_; + } + /** + * required uint64 id = 1; + */ + public Builder setId(long value) { + bitField0_ |= 0x00000001; + id_ = value; + onChanged(); + return this; + } + /** + * required uint64 id = 1; + */ + public Builder clearId() { + bitField0_ = (bitField0_ & ~0x00000001); + id_ = 0L; + onChanged(); + return this; + } + + // required string name = 2; + private java.lang.Object name_ = ""; + /** + * required string name = 2; + */ + public boolean hasName() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required string name = 2; + */ + public java.lang.String getName() { + java.lang.Object ref = name_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((akka.protobuf.ByteString) ref) + .toStringUtf8(); + name_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string name = 2; + */ + public akka.protobuf.ByteString + getNameBytes() { + java.lang.Object ref = name_; + if (ref instanceof String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + name_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + /** + * required string name = 2; + */ + public Builder setName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + name_ = value; + onChanged(); + return this; + } + /** + * required string name = 2; + */ + public Builder clearName() { + bitField0_ = (bitField0_ & ~0x00000002); + name_ = getDefaultInstance().getName(); + onChanged(); + return this; + } + /** + * required string name = 2; + */ + public Builder setNameBytes( + akka.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + name_ = value; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:Item) + } + + static { + defaultInstance = new Item(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:Item) + } + + private static akka.protobuf.Descriptors.Descriptor + internal_static_TestMessage_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_TestMessage_fieldAccessorTable; + private static akka.protobuf.Descriptors.Descriptor + internal_static_Item_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_Item_fieldAccessorTable; + + public static akka.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static akka.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\022TestMessages.proto\"s\n\013TestMessage\022\n\n\002i" + + "d\030\001 \002(\004\022\014\n\004name\030\002 \002(\t\022\016\n\006status\030\003 \002(\010\022\023\n" + + "\013description\030\004 \001(\t\022\017\n\007payload\030\005 \001(\014\022\024\n\005i" + + "tems\030\006 \003(\0132\005.Item\" \n\004Item\022\n\n\002id\030\001 \002(\004\022\014\n" + + "\004name\030\002 \002(\tB\035\n\033akka.remote.artery.protob" + + "uf" + }; + akka.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = + new akka.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { + public akka.protobuf.ExtensionRegistry assignDescriptors( + akka.protobuf.Descriptors.FileDescriptor root) { + descriptor = root; + internal_static_TestMessage_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_TestMessage_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_TestMessage_descriptor, + new java.lang.String[] { "Id", "Name", "Status", "Description", "Payload", "Items", }); + internal_static_Item_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_Item_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_Item_descriptor, + new java.lang.String[] { "Id", "Name", }); + return null; + } + }; + akka.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new akka.protobuf.Descriptors.FileDescriptor[] { + }, assigner); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/akka-remote-tests/src/test/protobuf/TestMessages.proto b/akka-remote-tests/src/test/protobuf/TestMessages.proto new file mode 100644 index 0000000000..6715a0c0a0 --- /dev/null +++ b/akka-remote-tests/src/test/protobuf/TestMessages.proto @@ -0,0 +1,20 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ + +option java_package = "akka.remote.artery.protobuf"; + +message TestMessage { + required uint64 id = 1; + required string name = 2; + required bool status = 3; + optional string description = 4; + optional bytes payload = 5; + repeated Item items = 6; +} + +message Item { + required uint64 id = 1; + required string name = 2; +} + diff --git a/akka-remote-tests/src/test/scala/akka/remote/testconductor/BarrierSpec.scala b/akka-remote-tests/src/test/scala/akka/remote/testconductor/BarrierSpec.scala index de2b5d36f3..78a5a295f6 100644 --- a/akka-remote-tests/src/test/scala/akka/remote/testconductor/BarrierSpec.scala +++ b/akka-remote-tests/src/test/scala/akka/remote/testconductor/BarrierSpec.scala @@ -14,7 +14,7 @@ object BarrierSpec { final case class Failed(ref: ActorRef, thr: Throwable) val config = """ akka.testconductor.barrier-timeout = 5s - akka.actor.provider = akka.remote.RemoteActorRefProvider + akka.actor.provider = remote akka.actor.debug.fsm = on akka.actor.debug.lifecycle = on """ diff --git a/akka-remote-tests/src/test/scala/akka/remote/testconductor/ControllerSpec.scala b/akka-remote-tests/src/test/scala/akka/remote/testconductor/ControllerSpec.scala index 9ac87f6736..f904c46bdf 100644 --- a/akka-remote-tests/src/test/scala/akka/remote/testconductor/ControllerSpec.scala +++ b/akka-remote-tests/src/test/scala/akka/remote/testconductor/ControllerSpec.scala @@ -13,7 +13,7 @@ import java.net.InetAddress object ControllerSpec { val config = """ akka.testconductor.barrier-timeout = 5s - akka.actor.provider = akka.remote.RemoteActorRefProvider + akka.actor.provider = remote akka.actor.debug.fsm = on akka.actor.debug.lifecycle = on """ diff --git a/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java b/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java new file mode 100644 index 0000000000..f1ed38b3a9 --- /dev/null +++ b/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java @@ -0,0 +1,7395 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: ArteryControlFormats.proto + +package akka.remote; + +public final class ArteryControlFormats { + private ArteryControlFormats() {} + public static void registerAllExtensions( + akka.protobuf.ExtensionRegistry registry) { + } + public interface QuarantinedOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required .UniqueAddress from = 1; + /** + * required .UniqueAddress from = 1; + */ + boolean hasFrom(); + /** + * required .UniqueAddress from = 1; + */ + akka.remote.ArteryControlFormats.UniqueAddress getFrom(); + /** + * required .UniqueAddress from = 1; + */ + akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getFromOrBuilder(); + + // required .UniqueAddress to = 2; + /** + * required .UniqueAddress to = 2; + */ + boolean hasTo(); + /** + * required .UniqueAddress to = 2; + */ + akka.remote.ArteryControlFormats.UniqueAddress getTo(); + /** + * required .UniqueAddress to = 2; + */ + akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getToOrBuilder(); + } + /** + * Protobuf type {@code Quarantined} + */ + public static final class Quarantined extends + akka.protobuf.GeneratedMessage + implements QuarantinedOrBuilder { + // Use Quarantined.newBuilder() to construct. + private Quarantined(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private Quarantined(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final Quarantined defaultInstance; + public static Quarantined getDefaultInstance() { + return defaultInstance; + } + + public Quarantined getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private Quarantined( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + akka.remote.ArteryControlFormats.UniqueAddress.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = from_.toBuilder(); + } + from_ = input.readMessage(akka.remote.ArteryControlFormats.UniqueAddress.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(from_); + from_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + case 18: { + akka.remote.ArteryControlFormats.UniqueAddress.Builder subBuilder = null; + if (((bitField0_ & 0x00000002) == 0x00000002)) { + subBuilder = to_.toBuilder(); + } + to_ = input.readMessage(akka.remote.ArteryControlFormats.UniqueAddress.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(to_); + to_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000002; + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_Quarantined_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_Quarantined_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.Quarantined.class, akka.remote.ArteryControlFormats.Quarantined.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public Quarantined parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new Quarantined(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required .UniqueAddress from = 1; + public static final int FROM_FIELD_NUMBER = 1; + private akka.remote.ArteryControlFormats.UniqueAddress from_; + /** + * required .UniqueAddress from = 1; + */ + public boolean hasFrom() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddress getFrom() { + return from_; + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getFromOrBuilder() { + return from_; + } + + // required .UniqueAddress to = 2; + public static final int TO_FIELD_NUMBER = 2; + private akka.remote.ArteryControlFormats.UniqueAddress to_; + /** + * required .UniqueAddress to = 2; + */ + public boolean hasTo() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required .UniqueAddress to = 2; + */ + public akka.remote.ArteryControlFormats.UniqueAddress getTo() { + return to_; + } + /** + * required .UniqueAddress to = 2; + */ + public akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getToOrBuilder() { + return to_; + } + + private void initFields() { + from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + to_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasFrom()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasTo()) { + memoizedIsInitialized = 0; + return false; + } + if (!getFrom().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + if (!getTo().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, from_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, to_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(1, from_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(2, to_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.ArteryControlFormats.Quarantined parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.Quarantined parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.Quarantined parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.Quarantined parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.Quarantined parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.Quarantined parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.Quarantined parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.ArteryControlFormats.Quarantined parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.Quarantined parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.Quarantined parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.ArteryControlFormats.Quarantined prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code Quarantined} + */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.ArteryControlFormats.QuarantinedOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_Quarantined_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_Quarantined_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.Quarantined.class, akka.remote.ArteryControlFormats.Quarantined.Builder.class); + } + + // Construct using akka.remote.ArteryControlFormats.Quarantined.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getFromFieldBuilder(); + getToFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (fromBuilder_ == null) { + from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + } else { + fromBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + if (toBuilder_ == null) { + to_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + } else { + toBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.ArteryControlFormats.internal_static_Quarantined_descriptor; + } + + public akka.remote.ArteryControlFormats.Quarantined getDefaultInstanceForType() { + return akka.remote.ArteryControlFormats.Quarantined.getDefaultInstance(); + } + + public akka.remote.ArteryControlFormats.Quarantined build() { + akka.remote.ArteryControlFormats.Quarantined result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.ArteryControlFormats.Quarantined buildPartial() { + akka.remote.ArteryControlFormats.Quarantined result = new akka.remote.ArteryControlFormats.Quarantined(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (fromBuilder_ == null) { + result.from_ = from_; + } else { + result.from_ = fromBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (toBuilder_ == null) { + result.to_ = to_; + } else { + result.to_ = toBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.ArteryControlFormats.Quarantined) { + return mergeFrom((akka.remote.ArteryControlFormats.Quarantined)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.ArteryControlFormats.Quarantined other) { + if (other == akka.remote.ArteryControlFormats.Quarantined.getDefaultInstance()) return this; + if (other.hasFrom()) { + mergeFrom(other.getFrom()); + } + if (other.hasTo()) { + mergeTo(other.getTo()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasFrom()) { + + return false; + } + if (!hasTo()) { + + return false; + } + if (!getFrom().isInitialized()) { + + return false; + } + if (!getTo().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.ArteryControlFormats.Quarantined parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.ArteryControlFormats.Quarantined) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required .UniqueAddress from = 1; + private akka.remote.ArteryControlFormats.UniqueAddress from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder> fromBuilder_; + /** + * required .UniqueAddress from = 1; + */ + public boolean hasFrom() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddress getFrom() { + if (fromBuilder_ == null) { + return from_; + } else { + return fromBuilder_.getMessage(); + } + } + /** + * required .UniqueAddress from = 1; + */ + public Builder setFrom(akka.remote.ArteryControlFormats.UniqueAddress value) { + if (fromBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + from_ = value; + onChanged(); + } else { + fromBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .UniqueAddress from = 1; + */ + public Builder setFrom( + akka.remote.ArteryControlFormats.UniqueAddress.Builder builderForValue) { + if (fromBuilder_ == null) { + from_ = builderForValue.build(); + onChanged(); + } else { + fromBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .UniqueAddress from = 1; + */ + public Builder mergeFrom(akka.remote.ArteryControlFormats.UniqueAddress value) { + if (fromBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + from_ != akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance()) { + from_ = + akka.remote.ArteryControlFormats.UniqueAddress.newBuilder(from_).mergeFrom(value).buildPartial(); + } else { + from_ = value; + } + onChanged(); + } else { + fromBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .UniqueAddress from = 1; + */ + public Builder clearFrom() { + if (fromBuilder_ == null) { + from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + onChanged(); + } else { + fromBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddress.Builder getFromBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getFromFieldBuilder().getBuilder(); + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getFromOrBuilder() { + if (fromBuilder_ != null) { + return fromBuilder_.getMessageOrBuilder(); + } else { + return from_; + } + } + /** + * required .UniqueAddress from = 1; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder> + getFromFieldBuilder() { + if (fromBuilder_ == null) { + fromBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder>( + from_, + getParentForChildren(), + isClean()); + from_ = null; + } + return fromBuilder_; + } + + // required .UniqueAddress to = 2; + private akka.remote.ArteryControlFormats.UniqueAddress to_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder> toBuilder_; + /** + * required .UniqueAddress to = 2; + */ + public boolean hasTo() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required .UniqueAddress to = 2; + */ + public akka.remote.ArteryControlFormats.UniqueAddress getTo() { + if (toBuilder_ == null) { + return to_; + } else { + return toBuilder_.getMessage(); + } + } + /** + * required .UniqueAddress to = 2; + */ + public Builder setTo(akka.remote.ArteryControlFormats.UniqueAddress value) { + if (toBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + to_ = value; + onChanged(); + } else { + toBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .UniqueAddress to = 2; + */ + public Builder setTo( + akka.remote.ArteryControlFormats.UniqueAddress.Builder builderForValue) { + if (toBuilder_ == null) { + to_ = builderForValue.build(); + onChanged(); + } else { + toBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .UniqueAddress to = 2; + */ + public Builder mergeTo(akka.remote.ArteryControlFormats.UniqueAddress value) { + if (toBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + to_ != akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance()) { + to_ = + akka.remote.ArteryControlFormats.UniqueAddress.newBuilder(to_).mergeFrom(value).buildPartial(); + } else { + to_ = value; + } + onChanged(); + } else { + toBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .UniqueAddress to = 2; + */ + public Builder clearTo() { + if (toBuilder_ == null) { + to_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + onChanged(); + } else { + toBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + /** + * required .UniqueAddress to = 2; + */ + public akka.remote.ArteryControlFormats.UniqueAddress.Builder getToBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getToFieldBuilder().getBuilder(); + } + /** + * required .UniqueAddress to = 2; + */ + public akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getToOrBuilder() { + if (toBuilder_ != null) { + return toBuilder_.getMessageOrBuilder(); + } else { + return to_; + } + } + /** + * required .UniqueAddress to = 2; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder> + getToFieldBuilder() { + if (toBuilder_ == null) { + toBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder>( + to_, + getParentForChildren(), + isClean()); + to_ = null; + } + return toBuilder_; + } + + // @@protoc_insertion_point(builder_scope:Quarantined) + } + + static { + defaultInstance = new Quarantined(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:Quarantined) + } + + public interface MessageWithAddressOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required .UniqueAddress address = 1; + /** + * required .UniqueAddress address = 1; + */ + boolean hasAddress(); + /** + * required .UniqueAddress address = 1; + */ + akka.remote.ArteryControlFormats.UniqueAddress getAddress(); + /** + * required .UniqueAddress address = 1; + */ + akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getAddressOrBuilder(); + } + /** + * Protobuf type {@code MessageWithAddress} + * + *
+   * Generic message declaration that is used for all types of message that (currently) have a single
+   * address field. A message that needs to changed later can be cloned from this one and then adapted.
+   * ActorSystemTerminating
+   * ActorSystemTerminating.Ack
+   * OutboundHandshake.HandshakeRsp
+   * 
+ */ + public static final class MessageWithAddress extends + akka.protobuf.GeneratedMessage + implements MessageWithAddressOrBuilder { + // Use MessageWithAddress.newBuilder() to construct. + private MessageWithAddress(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private MessageWithAddress(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final MessageWithAddress defaultInstance; + public static MessageWithAddress getDefaultInstance() { + return defaultInstance; + } + + public MessageWithAddress getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private MessageWithAddress( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + akka.remote.ArteryControlFormats.UniqueAddress.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = address_.toBuilder(); + } + address_ = input.readMessage(akka.remote.ArteryControlFormats.UniqueAddress.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(address_); + address_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_MessageWithAddress_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_MessageWithAddress_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.MessageWithAddress.class, akka.remote.ArteryControlFormats.MessageWithAddress.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public MessageWithAddress parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new MessageWithAddress(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required .UniqueAddress address = 1; + public static final int ADDRESS_FIELD_NUMBER = 1; + private akka.remote.ArteryControlFormats.UniqueAddress address_; + /** + * required .UniqueAddress address = 1; + */ + public boolean hasAddress() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .UniqueAddress address = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddress getAddress() { + return address_; + } + /** + * required .UniqueAddress address = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getAddressOrBuilder() { + return address_; + } + + private void initFields() { + address_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasAddress()) { + memoizedIsInitialized = 0; + return false; + } + if (!getAddress().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, address_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(1, address_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.ArteryControlFormats.MessageWithAddress parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.MessageWithAddress parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.MessageWithAddress parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.MessageWithAddress parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.MessageWithAddress parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.MessageWithAddress parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.MessageWithAddress parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.ArteryControlFormats.MessageWithAddress parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.MessageWithAddress parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.MessageWithAddress parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.ArteryControlFormats.MessageWithAddress prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code MessageWithAddress} + * + *
+     * Generic message declaration that is used for all types of message that (currently) have a single
+     * address field. A message that needs to changed later can be cloned from this one and then adapted.
+     * ActorSystemTerminating
+     * ActorSystemTerminating.Ack
+     * OutboundHandshake.HandshakeRsp
+     * 
+ */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.ArteryControlFormats.MessageWithAddressOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_MessageWithAddress_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_MessageWithAddress_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.MessageWithAddress.class, akka.remote.ArteryControlFormats.MessageWithAddress.Builder.class); + } + + // Construct using akka.remote.ArteryControlFormats.MessageWithAddress.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getAddressFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (addressBuilder_ == null) { + address_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + } else { + addressBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.ArteryControlFormats.internal_static_MessageWithAddress_descriptor; + } + + public akka.remote.ArteryControlFormats.MessageWithAddress getDefaultInstanceForType() { + return akka.remote.ArteryControlFormats.MessageWithAddress.getDefaultInstance(); + } + + public akka.remote.ArteryControlFormats.MessageWithAddress build() { + akka.remote.ArteryControlFormats.MessageWithAddress result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.ArteryControlFormats.MessageWithAddress buildPartial() { + akka.remote.ArteryControlFormats.MessageWithAddress result = new akka.remote.ArteryControlFormats.MessageWithAddress(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (addressBuilder_ == null) { + result.address_ = address_; + } else { + result.address_ = addressBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.ArteryControlFormats.MessageWithAddress) { + return mergeFrom((akka.remote.ArteryControlFormats.MessageWithAddress)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.ArteryControlFormats.MessageWithAddress other) { + if (other == akka.remote.ArteryControlFormats.MessageWithAddress.getDefaultInstance()) return this; + if (other.hasAddress()) { + mergeAddress(other.getAddress()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasAddress()) { + + return false; + } + if (!getAddress().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.ArteryControlFormats.MessageWithAddress parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.ArteryControlFormats.MessageWithAddress) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required .UniqueAddress address = 1; + private akka.remote.ArteryControlFormats.UniqueAddress address_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder> addressBuilder_; + /** + * required .UniqueAddress address = 1; + */ + public boolean hasAddress() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .UniqueAddress address = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddress getAddress() { + if (addressBuilder_ == null) { + return address_; + } else { + return addressBuilder_.getMessage(); + } + } + /** + * required .UniqueAddress address = 1; + */ + public Builder setAddress(akka.remote.ArteryControlFormats.UniqueAddress value) { + if (addressBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + address_ = value; + onChanged(); + } else { + addressBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .UniqueAddress address = 1; + */ + public Builder setAddress( + akka.remote.ArteryControlFormats.UniqueAddress.Builder builderForValue) { + if (addressBuilder_ == null) { + address_ = builderForValue.build(); + onChanged(); + } else { + addressBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .UniqueAddress address = 1; + */ + public Builder mergeAddress(akka.remote.ArteryControlFormats.UniqueAddress value) { + if (addressBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + address_ != akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance()) { + address_ = + akka.remote.ArteryControlFormats.UniqueAddress.newBuilder(address_).mergeFrom(value).buildPartial(); + } else { + address_ = value; + } + onChanged(); + } else { + addressBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .UniqueAddress address = 1; + */ + public Builder clearAddress() { + if (addressBuilder_ == null) { + address_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + onChanged(); + } else { + addressBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * required .UniqueAddress address = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddress.Builder getAddressBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getAddressFieldBuilder().getBuilder(); + } + /** + * required .UniqueAddress address = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getAddressOrBuilder() { + if (addressBuilder_ != null) { + return addressBuilder_.getMessageOrBuilder(); + } else { + return address_; + } + } + /** + * required .UniqueAddress address = 1; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder> + getAddressFieldBuilder() { + if (addressBuilder_ == null) { + addressBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder>( + address_, + getParentForChildren(), + isClean()); + address_ = null; + } + return addressBuilder_; + } + + // @@protoc_insertion_point(builder_scope:MessageWithAddress) + } + + static { + defaultInstance = new MessageWithAddress(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:MessageWithAddress) + } + + public interface HandshakeReqOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required .UniqueAddress from = 1; + /** + * required .UniqueAddress from = 1; + */ + boolean hasFrom(); + /** + * required .UniqueAddress from = 1; + */ + akka.remote.ArteryControlFormats.UniqueAddress getFrom(); + /** + * required .UniqueAddress from = 1; + */ + akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getFromOrBuilder(); + + // required .Address to = 2; + /** + * required .Address to = 2; + */ + boolean hasTo(); + /** + * required .Address to = 2; + */ + akka.remote.ArteryControlFormats.Address getTo(); + /** + * required .Address to = 2; + */ + akka.remote.ArteryControlFormats.AddressOrBuilder getToOrBuilder(); + } + /** + * Protobuf type {@code HandshakeReq} + */ + public static final class HandshakeReq extends + akka.protobuf.GeneratedMessage + implements HandshakeReqOrBuilder { + // Use HandshakeReq.newBuilder() to construct. + private HandshakeReq(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private HandshakeReq(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final HandshakeReq defaultInstance; + public static HandshakeReq getDefaultInstance() { + return defaultInstance; + } + + public HandshakeReq getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private HandshakeReq( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + akka.remote.ArteryControlFormats.UniqueAddress.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = from_.toBuilder(); + } + from_ = input.readMessage(akka.remote.ArteryControlFormats.UniqueAddress.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(from_); + from_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + case 18: { + akka.remote.ArteryControlFormats.Address.Builder subBuilder = null; + if (((bitField0_ & 0x00000002) == 0x00000002)) { + subBuilder = to_.toBuilder(); + } + to_ = input.readMessage(akka.remote.ArteryControlFormats.Address.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(to_); + to_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000002; + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_HandshakeReq_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_HandshakeReq_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.HandshakeReq.class, akka.remote.ArteryControlFormats.HandshakeReq.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public HandshakeReq parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new HandshakeReq(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required .UniqueAddress from = 1; + public static final int FROM_FIELD_NUMBER = 1; + private akka.remote.ArteryControlFormats.UniqueAddress from_; + /** + * required .UniqueAddress from = 1; + */ + public boolean hasFrom() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddress getFrom() { + return from_; + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getFromOrBuilder() { + return from_; + } + + // required .Address to = 2; + public static final int TO_FIELD_NUMBER = 2; + private akka.remote.ArteryControlFormats.Address to_; + /** + * required .Address to = 2; + */ + public boolean hasTo() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required .Address to = 2; + */ + public akka.remote.ArteryControlFormats.Address getTo() { + return to_; + } + /** + * required .Address to = 2; + */ + public akka.remote.ArteryControlFormats.AddressOrBuilder getToOrBuilder() { + return to_; + } + + private void initFields() { + from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + to_ = akka.remote.ArteryControlFormats.Address.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasFrom()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasTo()) { + memoizedIsInitialized = 0; + return false; + } + if (!getFrom().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + if (!getTo().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, from_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, to_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(1, from_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(2, to_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.ArteryControlFormats.HandshakeReq parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.HandshakeReq parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.HandshakeReq parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.HandshakeReq parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.HandshakeReq parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.HandshakeReq parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.HandshakeReq parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.ArteryControlFormats.HandshakeReq parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.HandshakeReq parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.HandshakeReq parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.ArteryControlFormats.HandshakeReq prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code HandshakeReq} + */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.ArteryControlFormats.HandshakeReqOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_HandshakeReq_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_HandshakeReq_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.HandshakeReq.class, akka.remote.ArteryControlFormats.HandshakeReq.Builder.class); + } + + // Construct using akka.remote.ArteryControlFormats.HandshakeReq.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getFromFieldBuilder(); + getToFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (fromBuilder_ == null) { + from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + } else { + fromBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + if (toBuilder_ == null) { + to_ = akka.remote.ArteryControlFormats.Address.getDefaultInstance(); + } else { + toBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.ArteryControlFormats.internal_static_HandshakeReq_descriptor; + } + + public akka.remote.ArteryControlFormats.HandshakeReq getDefaultInstanceForType() { + return akka.remote.ArteryControlFormats.HandshakeReq.getDefaultInstance(); + } + + public akka.remote.ArteryControlFormats.HandshakeReq build() { + akka.remote.ArteryControlFormats.HandshakeReq result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.ArteryControlFormats.HandshakeReq buildPartial() { + akka.remote.ArteryControlFormats.HandshakeReq result = new akka.remote.ArteryControlFormats.HandshakeReq(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (fromBuilder_ == null) { + result.from_ = from_; + } else { + result.from_ = fromBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (toBuilder_ == null) { + result.to_ = to_; + } else { + result.to_ = toBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.ArteryControlFormats.HandshakeReq) { + return mergeFrom((akka.remote.ArteryControlFormats.HandshakeReq)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.ArteryControlFormats.HandshakeReq other) { + if (other == akka.remote.ArteryControlFormats.HandshakeReq.getDefaultInstance()) return this; + if (other.hasFrom()) { + mergeFrom(other.getFrom()); + } + if (other.hasTo()) { + mergeTo(other.getTo()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasFrom()) { + + return false; + } + if (!hasTo()) { + + return false; + } + if (!getFrom().isInitialized()) { + + return false; + } + if (!getTo().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.ArteryControlFormats.HandshakeReq parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.ArteryControlFormats.HandshakeReq) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required .UniqueAddress from = 1; + private akka.remote.ArteryControlFormats.UniqueAddress from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder> fromBuilder_; + /** + * required .UniqueAddress from = 1; + */ + public boolean hasFrom() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddress getFrom() { + if (fromBuilder_ == null) { + return from_; + } else { + return fromBuilder_.getMessage(); + } + } + /** + * required .UniqueAddress from = 1; + */ + public Builder setFrom(akka.remote.ArteryControlFormats.UniqueAddress value) { + if (fromBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + from_ = value; + onChanged(); + } else { + fromBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .UniqueAddress from = 1; + */ + public Builder setFrom( + akka.remote.ArteryControlFormats.UniqueAddress.Builder builderForValue) { + if (fromBuilder_ == null) { + from_ = builderForValue.build(); + onChanged(); + } else { + fromBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .UniqueAddress from = 1; + */ + public Builder mergeFrom(akka.remote.ArteryControlFormats.UniqueAddress value) { + if (fromBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + from_ != akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance()) { + from_ = + akka.remote.ArteryControlFormats.UniqueAddress.newBuilder(from_).mergeFrom(value).buildPartial(); + } else { + from_ = value; + } + onChanged(); + } else { + fromBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .UniqueAddress from = 1; + */ + public Builder clearFrom() { + if (fromBuilder_ == null) { + from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + onChanged(); + } else { + fromBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddress.Builder getFromBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getFromFieldBuilder().getBuilder(); + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getFromOrBuilder() { + if (fromBuilder_ != null) { + return fromBuilder_.getMessageOrBuilder(); + } else { + return from_; + } + } + /** + * required .UniqueAddress from = 1; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder> + getFromFieldBuilder() { + if (fromBuilder_ == null) { + fromBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder>( + from_, + getParentForChildren(), + isClean()); + from_ = null; + } + return fromBuilder_; + } + + // required .Address to = 2; + private akka.remote.ArteryControlFormats.Address to_ = akka.remote.ArteryControlFormats.Address.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.Address, akka.remote.ArteryControlFormats.Address.Builder, akka.remote.ArteryControlFormats.AddressOrBuilder> toBuilder_; + /** + * required .Address to = 2; + */ + public boolean hasTo() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required .Address to = 2; + */ + public akka.remote.ArteryControlFormats.Address getTo() { + if (toBuilder_ == null) { + return to_; + } else { + return toBuilder_.getMessage(); + } + } + /** + * required .Address to = 2; + */ + public Builder setTo(akka.remote.ArteryControlFormats.Address value) { + if (toBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + to_ = value; + onChanged(); + } else { + toBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .Address to = 2; + */ + public Builder setTo( + akka.remote.ArteryControlFormats.Address.Builder builderForValue) { + if (toBuilder_ == null) { + to_ = builderForValue.build(); + onChanged(); + } else { + toBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .Address to = 2; + */ + public Builder mergeTo(akka.remote.ArteryControlFormats.Address value) { + if (toBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + to_ != akka.remote.ArteryControlFormats.Address.getDefaultInstance()) { + to_ = + akka.remote.ArteryControlFormats.Address.newBuilder(to_).mergeFrom(value).buildPartial(); + } else { + to_ = value; + } + onChanged(); + } else { + toBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .Address to = 2; + */ + public Builder clearTo() { + if (toBuilder_ == null) { + to_ = akka.remote.ArteryControlFormats.Address.getDefaultInstance(); + onChanged(); + } else { + toBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + /** + * required .Address to = 2; + */ + public akka.remote.ArteryControlFormats.Address.Builder getToBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getToFieldBuilder().getBuilder(); + } + /** + * required .Address to = 2; + */ + public akka.remote.ArteryControlFormats.AddressOrBuilder getToOrBuilder() { + if (toBuilder_ != null) { + return toBuilder_.getMessageOrBuilder(); + } else { + return to_; + } + } + /** + * required .Address to = 2; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.Address, akka.remote.ArteryControlFormats.Address.Builder, akka.remote.ArteryControlFormats.AddressOrBuilder> + getToFieldBuilder() { + if (toBuilder_ == null) { + toBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.Address, akka.remote.ArteryControlFormats.Address.Builder, akka.remote.ArteryControlFormats.AddressOrBuilder>( + to_, + getParentForChildren(), + isClean()); + to_ = null; + } + return toBuilder_; + } + + // @@protoc_insertion_point(builder_scope:HandshakeReq) + } + + static { + defaultInstance = new HandshakeReq(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:HandshakeReq) + } + + public interface CompressionTableAdvertisementOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required .UniqueAddress from = 1; + /** + * required .UniqueAddress from = 1; + */ + boolean hasFrom(); + /** + * required .UniqueAddress from = 1; + */ + akka.remote.ArteryControlFormats.UniqueAddress getFrom(); + /** + * required .UniqueAddress from = 1; + */ + akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getFromOrBuilder(); + + // required uint64 originUid = 2; + /** + * required uint64 originUid = 2; + */ + boolean hasOriginUid(); + /** + * required uint64 originUid = 2; + */ + long getOriginUid(); + + // required uint32 tableVersion = 3; + /** + * required uint32 tableVersion = 3; + */ + boolean hasTableVersion(); + /** + * required uint32 tableVersion = 3; + */ + int getTableVersion(); + + // repeated string keys = 4; + /** + * repeated string keys = 4; + * + *
+     * actual Map is represented by separate sequences of keys and values,
+     * relies on both sequences using the same order so that corresponding entries can be
+     * associated again when deserializing
+     * 
+ */ + java.util.List + getKeysList(); + /** + * repeated string keys = 4; + * + *
+     * actual Map is represented by separate sequences of keys and values,
+     * relies on both sequences using the same order so that corresponding entries can be
+     * associated again when deserializing
+     * 
+ */ + int getKeysCount(); + /** + * repeated string keys = 4; + * + *
+     * actual Map is represented by separate sequences of keys and values,
+     * relies on both sequences using the same order so that corresponding entries can be
+     * associated again when deserializing
+     * 
+ */ + java.lang.String getKeys(int index); + /** + * repeated string keys = 4; + * + *
+     * actual Map is represented by separate sequences of keys and values,
+     * relies on both sequences using the same order so that corresponding entries can be
+     * associated again when deserializing
+     * 
+ */ + akka.protobuf.ByteString + getKeysBytes(int index); + + // repeated uint32 values = 5; + /** + * repeated uint32 values = 5; + */ + java.util.List getValuesList(); + /** + * repeated uint32 values = 5; + */ + int getValuesCount(); + /** + * repeated uint32 values = 5; + */ + int getValues(int index); + } + /** + * Protobuf type {@code CompressionTableAdvertisement} + * + *
+   * CompressionProtocol.ActorRefCompressionAdvertisement
+   * CompressionProtocol.ClassManifestCompressionAdvertisement
+   * 
+ */ + public static final class CompressionTableAdvertisement extends + akka.protobuf.GeneratedMessage + implements CompressionTableAdvertisementOrBuilder { + // Use CompressionTableAdvertisement.newBuilder() to construct. + private CompressionTableAdvertisement(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private CompressionTableAdvertisement(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final CompressionTableAdvertisement defaultInstance; + public static CompressionTableAdvertisement getDefaultInstance() { + return defaultInstance; + } + + public CompressionTableAdvertisement getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private CompressionTableAdvertisement( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + akka.remote.ArteryControlFormats.UniqueAddress.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = from_.toBuilder(); + } + from_ = input.readMessage(akka.remote.ArteryControlFormats.UniqueAddress.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(from_); + from_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + case 16: { + bitField0_ |= 0x00000002; + originUid_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + tableVersion_ = input.readUInt32(); + break; + } + case 34: { + if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) { + keys_ = new akka.protobuf.LazyStringArrayList(); + mutable_bitField0_ |= 0x00000008; + } + keys_.add(input.readBytes()); + break; + } + case 40: { + if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) { + values_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000010; + } + values_.add(input.readUInt32()); + break; + } + case 42: { + int length = input.readRawVarint32(); + int limit = input.pushLimit(length); + if (!((mutable_bitField0_ & 0x00000010) == 0x00000010) && input.getBytesUntilLimit() > 0) { + values_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000010; + } + while (input.getBytesUntilLimit() > 0) { + values_.add(input.readUInt32()); + } + input.popLimit(limit); + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) { + keys_ = new akka.protobuf.UnmodifiableLazyStringList(keys_); + } + if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) { + values_ = java.util.Collections.unmodifiableList(values_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_CompressionTableAdvertisement_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_CompressionTableAdvertisement_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.CompressionTableAdvertisement.class, akka.remote.ArteryControlFormats.CompressionTableAdvertisement.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public CompressionTableAdvertisement parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new CompressionTableAdvertisement(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required .UniqueAddress from = 1; + public static final int FROM_FIELD_NUMBER = 1; + private akka.remote.ArteryControlFormats.UniqueAddress from_; + /** + * required .UniqueAddress from = 1; + */ + public boolean hasFrom() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddress getFrom() { + return from_; + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getFromOrBuilder() { + return from_; + } + + // required uint64 originUid = 2; + public static final int ORIGINUID_FIELD_NUMBER = 2; + private long originUid_; + /** + * required uint64 originUid = 2; + */ + public boolean hasOriginUid() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required uint64 originUid = 2; + */ + public long getOriginUid() { + return originUid_; + } + + // required uint32 tableVersion = 3; + public static final int TABLEVERSION_FIELD_NUMBER = 3; + private int tableVersion_; + /** + * required uint32 tableVersion = 3; + */ + public boolean hasTableVersion() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required uint32 tableVersion = 3; + */ + public int getTableVersion() { + return tableVersion_; + } + + // repeated string keys = 4; + public static final int KEYS_FIELD_NUMBER = 4; + private akka.protobuf.LazyStringList keys_; + /** + * repeated string keys = 4; + * + *
+     * actual Map is represented by separate sequences of keys and values,
+     * relies on both sequences using the same order so that corresponding entries can be
+     * associated again when deserializing
+     * 
+ */ + public java.util.List + getKeysList() { + return keys_; + } + /** + * repeated string keys = 4; + * + *
+     * actual Map is represented by separate sequences of keys and values,
+     * relies on both sequences using the same order so that corresponding entries can be
+     * associated again when deserializing
+     * 
+ */ + public int getKeysCount() { + return keys_.size(); + } + /** + * repeated string keys = 4; + * + *
+     * actual Map is represented by separate sequences of keys and values,
+     * relies on both sequences using the same order so that corresponding entries can be
+     * associated again when deserializing
+     * 
+ */ + public java.lang.String getKeys(int index) { + return keys_.get(index); + } + /** + * repeated string keys = 4; + * + *
+     * actual Map is represented by separate sequences of keys and values,
+     * relies on both sequences using the same order so that corresponding entries can be
+     * associated again when deserializing
+     * 
+ */ + public akka.protobuf.ByteString + getKeysBytes(int index) { + return keys_.getByteString(index); + } + + // repeated uint32 values = 5; + public static final int VALUES_FIELD_NUMBER = 5; + private java.util.List values_; + /** + * repeated uint32 values = 5; + */ + public java.util.List + getValuesList() { + return values_; + } + /** + * repeated uint32 values = 5; + */ + public int getValuesCount() { + return values_.size(); + } + /** + * repeated uint32 values = 5; + */ + public int getValues(int index) { + return values_.get(index); + } + + private void initFields() { + from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + originUid_ = 0L; + tableVersion_ = 0; + keys_ = akka.protobuf.LazyStringArrayList.EMPTY; + values_ = java.util.Collections.emptyList(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasFrom()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasOriginUid()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasTableVersion()) { + memoizedIsInitialized = 0; + return false; + } + if (!getFrom().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, from_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, originUid_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt32(3, tableVersion_); + } + for (int i = 0; i < keys_.size(); i++) { + output.writeBytes(4, keys_.getByteString(i)); + } + for (int i = 0; i < values_.size(); i++) { + output.writeUInt32(5, values_.get(i)); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(1, from_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += akka.protobuf.CodedOutputStream + .computeUInt64Size(2, originUid_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += akka.protobuf.CodedOutputStream + .computeUInt32Size(3, tableVersion_); + } + { + int dataSize = 0; + for (int i = 0; i < keys_.size(); i++) { + dataSize += akka.protobuf.CodedOutputStream + .computeBytesSizeNoTag(keys_.getByteString(i)); + } + size += dataSize; + size += 1 * getKeysList().size(); + } + { + int dataSize = 0; + for (int i = 0; i < values_.size(); i++) { + dataSize += akka.protobuf.CodedOutputStream + .computeUInt32SizeNoTag(values_.get(i)); + } + size += dataSize; + size += 1 * getValuesList().size(); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisement parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisement parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisement parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisement parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisement parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisement parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisement parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisement parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisement parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisement parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.ArteryControlFormats.CompressionTableAdvertisement prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code CompressionTableAdvertisement} + * + *
+     * CompressionProtocol.ActorRefCompressionAdvertisement
+     * CompressionProtocol.ClassManifestCompressionAdvertisement
+     * 
+ */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.ArteryControlFormats.CompressionTableAdvertisementOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_CompressionTableAdvertisement_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_CompressionTableAdvertisement_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.CompressionTableAdvertisement.class, akka.remote.ArteryControlFormats.CompressionTableAdvertisement.Builder.class); + } + + // Construct using akka.remote.ArteryControlFormats.CompressionTableAdvertisement.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getFromFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (fromBuilder_ == null) { + from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + } else { + fromBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + originUid_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + tableVersion_ = 0; + bitField0_ = (bitField0_ & ~0x00000004); + keys_ = akka.protobuf.LazyStringArrayList.EMPTY; + bitField0_ = (bitField0_ & ~0x00000008); + values_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.ArteryControlFormats.internal_static_CompressionTableAdvertisement_descriptor; + } + + public akka.remote.ArteryControlFormats.CompressionTableAdvertisement getDefaultInstanceForType() { + return akka.remote.ArteryControlFormats.CompressionTableAdvertisement.getDefaultInstance(); + } + + public akka.remote.ArteryControlFormats.CompressionTableAdvertisement build() { + akka.remote.ArteryControlFormats.CompressionTableAdvertisement result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.ArteryControlFormats.CompressionTableAdvertisement buildPartial() { + akka.remote.ArteryControlFormats.CompressionTableAdvertisement result = new akka.remote.ArteryControlFormats.CompressionTableAdvertisement(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (fromBuilder_ == null) { + result.from_ = from_; + } else { + result.from_ = fromBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.originUid_ = originUid_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.tableVersion_ = tableVersion_; + if (((bitField0_ & 0x00000008) == 0x00000008)) { + keys_ = new akka.protobuf.UnmodifiableLazyStringList( + keys_); + bitField0_ = (bitField0_ & ~0x00000008); + } + result.keys_ = keys_; + if (((bitField0_ & 0x00000010) == 0x00000010)) { + values_ = java.util.Collections.unmodifiableList(values_); + bitField0_ = (bitField0_ & ~0x00000010); + } + result.values_ = values_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.ArteryControlFormats.CompressionTableAdvertisement) { + return mergeFrom((akka.remote.ArteryControlFormats.CompressionTableAdvertisement)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.ArteryControlFormats.CompressionTableAdvertisement other) { + if (other == akka.remote.ArteryControlFormats.CompressionTableAdvertisement.getDefaultInstance()) return this; + if (other.hasFrom()) { + mergeFrom(other.getFrom()); + } + if (other.hasOriginUid()) { + setOriginUid(other.getOriginUid()); + } + if (other.hasTableVersion()) { + setTableVersion(other.getTableVersion()); + } + if (!other.keys_.isEmpty()) { + if (keys_.isEmpty()) { + keys_ = other.keys_; + bitField0_ = (bitField0_ & ~0x00000008); + } else { + ensureKeysIsMutable(); + keys_.addAll(other.keys_); + } + onChanged(); + } + if (!other.values_.isEmpty()) { + if (values_.isEmpty()) { + values_ = other.values_; + bitField0_ = (bitField0_ & ~0x00000010); + } else { + ensureValuesIsMutable(); + values_.addAll(other.values_); + } + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasFrom()) { + + return false; + } + if (!hasOriginUid()) { + + return false; + } + if (!hasTableVersion()) { + + return false; + } + if (!getFrom().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.ArteryControlFormats.CompressionTableAdvertisement parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.ArteryControlFormats.CompressionTableAdvertisement) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required .UniqueAddress from = 1; + private akka.remote.ArteryControlFormats.UniqueAddress from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder> fromBuilder_; + /** + * required .UniqueAddress from = 1; + */ + public boolean hasFrom() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddress getFrom() { + if (fromBuilder_ == null) { + return from_; + } else { + return fromBuilder_.getMessage(); + } + } + /** + * required .UniqueAddress from = 1; + */ + public Builder setFrom(akka.remote.ArteryControlFormats.UniqueAddress value) { + if (fromBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + from_ = value; + onChanged(); + } else { + fromBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .UniqueAddress from = 1; + */ + public Builder setFrom( + akka.remote.ArteryControlFormats.UniqueAddress.Builder builderForValue) { + if (fromBuilder_ == null) { + from_ = builderForValue.build(); + onChanged(); + } else { + fromBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .UniqueAddress from = 1; + */ + public Builder mergeFrom(akka.remote.ArteryControlFormats.UniqueAddress value) { + if (fromBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + from_ != akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance()) { + from_ = + akka.remote.ArteryControlFormats.UniqueAddress.newBuilder(from_).mergeFrom(value).buildPartial(); + } else { + from_ = value; + } + onChanged(); + } else { + fromBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .UniqueAddress from = 1; + */ + public Builder clearFrom() { + if (fromBuilder_ == null) { + from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + onChanged(); + } else { + fromBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddress.Builder getFromBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getFromFieldBuilder().getBuilder(); + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getFromOrBuilder() { + if (fromBuilder_ != null) { + return fromBuilder_.getMessageOrBuilder(); + } else { + return from_; + } + } + /** + * required .UniqueAddress from = 1; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder> + getFromFieldBuilder() { + if (fromBuilder_ == null) { + fromBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder>( + from_, + getParentForChildren(), + isClean()); + from_ = null; + } + return fromBuilder_; + } + + // required uint64 originUid = 2; + private long originUid_ ; + /** + * required uint64 originUid = 2; + */ + public boolean hasOriginUid() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required uint64 originUid = 2; + */ + public long getOriginUid() { + return originUid_; + } + /** + * required uint64 originUid = 2; + */ + public Builder setOriginUid(long value) { + bitField0_ |= 0x00000002; + originUid_ = value; + onChanged(); + return this; + } + /** + * required uint64 originUid = 2; + */ + public Builder clearOriginUid() { + bitField0_ = (bitField0_ & ~0x00000002); + originUid_ = 0L; + onChanged(); + return this; + } + + // required uint32 tableVersion = 3; + private int tableVersion_ ; + /** + * required uint32 tableVersion = 3; + */ + public boolean hasTableVersion() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required uint32 tableVersion = 3; + */ + public int getTableVersion() { + return tableVersion_; + } + /** + * required uint32 tableVersion = 3; + */ + public Builder setTableVersion(int value) { + bitField0_ |= 0x00000004; + tableVersion_ = value; + onChanged(); + return this; + } + /** + * required uint32 tableVersion = 3; + */ + public Builder clearTableVersion() { + bitField0_ = (bitField0_ & ~0x00000004); + tableVersion_ = 0; + onChanged(); + return this; + } + + // repeated string keys = 4; + private akka.protobuf.LazyStringList keys_ = akka.protobuf.LazyStringArrayList.EMPTY; + private void ensureKeysIsMutable() { + if (!((bitField0_ & 0x00000008) == 0x00000008)) { + keys_ = new akka.protobuf.LazyStringArrayList(keys_); + bitField0_ |= 0x00000008; + } + } + /** + * repeated string keys = 4; + * + *
+       * actual Map is represented by separate sequences of keys and values,
+       * relies on both sequences using the same order so that corresponding entries can be
+       * associated again when deserializing
+       * 
+ */ + public java.util.List + getKeysList() { + return java.util.Collections.unmodifiableList(keys_); + } + /** + * repeated string keys = 4; + * + *
+       * actual Map is represented by separate sequences of keys and values,
+       * relies on both sequences using the same order so that corresponding entries can be
+       * associated again when deserializing
+       * 
+ */ + public int getKeysCount() { + return keys_.size(); + } + /** + * repeated string keys = 4; + * + *
+       * actual Map is represented by separate sequences of keys and values,
+       * relies on both sequences using the same order so that corresponding entries can be
+       * associated again when deserializing
+       * 
+ */ + public java.lang.String getKeys(int index) { + return keys_.get(index); + } + /** + * repeated string keys = 4; + * + *
+       * actual Map is represented by separate sequences of keys and values,
+       * relies on both sequences using the same order so that corresponding entries can be
+       * associated again when deserializing
+       * 
+ */ + public akka.protobuf.ByteString + getKeysBytes(int index) { + return keys_.getByteString(index); + } + /** + * repeated string keys = 4; + * + *
+       * actual Map is represented by separate sequences of keys and values,
+       * relies on both sequences using the same order so that corresponding entries can be
+       * associated again when deserializing
+       * 
+ */ + public Builder setKeys( + int index, java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + ensureKeysIsMutable(); + keys_.set(index, value); + onChanged(); + return this; + } + /** + * repeated string keys = 4; + * + *
+       * actual Map is represented by separate sequences of keys and values,
+       * relies on both sequences using the same order so that corresponding entries can be
+       * associated again when deserializing
+       * 
+ */ + public Builder addKeys( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + ensureKeysIsMutable(); + keys_.add(value); + onChanged(); + return this; + } + /** + * repeated string keys = 4; + * + *
+       * actual Map is represented by separate sequences of keys and values,
+       * relies on both sequences using the same order so that corresponding entries can be
+       * associated again when deserializing
+       * 
+ */ + public Builder addAllKeys( + java.lang.Iterable values) { + ensureKeysIsMutable(); + super.addAll(values, keys_); + onChanged(); + return this; + } + /** + * repeated string keys = 4; + * + *
+       * actual Map is represented by separate sequences of keys and values,
+       * relies on both sequences using the same order so that corresponding entries can be
+       * associated again when deserializing
+       * 
+ */ + public Builder clearKeys() { + keys_ = akka.protobuf.LazyStringArrayList.EMPTY; + bitField0_ = (bitField0_ & ~0x00000008); + onChanged(); + return this; + } + /** + * repeated string keys = 4; + * + *
+       * actual Map is represented by separate sequences of keys and values,
+       * relies on both sequences using the same order so that corresponding entries can be
+       * associated again when deserializing
+       * 
+ */ + public Builder addKeysBytes( + akka.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + ensureKeysIsMutable(); + keys_.add(value); + onChanged(); + return this; + } + + // repeated uint32 values = 5; + private java.util.List values_ = java.util.Collections.emptyList(); + private void ensureValuesIsMutable() { + if (!((bitField0_ & 0x00000010) == 0x00000010)) { + values_ = new java.util.ArrayList(values_); + bitField0_ |= 0x00000010; + } + } + /** + * repeated uint32 values = 5; + */ + public java.util.List + getValuesList() { + return java.util.Collections.unmodifiableList(values_); + } + /** + * repeated uint32 values = 5; + */ + public int getValuesCount() { + return values_.size(); + } + /** + * repeated uint32 values = 5; + */ + public int getValues(int index) { + return values_.get(index); + } + /** + * repeated uint32 values = 5; + */ + public Builder setValues( + int index, int value) { + ensureValuesIsMutable(); + values_.set(index, value); + onChanged(); + return this; + } + /** + * repeated uint32 values = 5; + */ + public Builder addValues(int value) { + ensureValuesIsMutable(); + values_.add(value); + onChanged(); + return this; + } + /** + * repeated uint32 values = 5; + */ + public Builder addAllValues( + java.lang.Iterable values) { + ensureValuesIsMutable(); + super.addAll(values, values_); + onChanged(); + return this; + } + /** + * repeated uint32 values = 5; + */ + public Builder clearValues() { + values_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000010); + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:CompressionTableAdvertisement) + } + + static { + defaultInstance = new CompressionTableAdvertisement(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:CompressionTableAdvertisement) + } + + public interface CompressionTableAdvertisementAckOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required .UniqueAddress from = 1; + /** + * required .UniqueAddress from = 1; + */ + boolean hasFrom(); + /** + * required .UniqueAddress from = 1; + */ + akka.remote.ArteryControlFormats.UniqueAddress getFrom(); + /** + * required .UniqueAddress from = 1; + */ + akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getFromOrBuilder(); + + // required uint32 version = 2; + /** + * required uint32 version = 2; + */ + boolean hasVersion(); + /** + * required uint32 version = 2; + */ + int getVersion(); + } + /** + * Protobuf type {@code CompressionTableAdvertisementAck} + * + *
+   * CompressionProtocol.ActorRefCompressionAdvertisementAck
+   * CompressionProtocol.ClassManifestCompressionAdvertisementAck
+   * 
+ */ + public static final class CompressionTableAdvertisementAck extends + akka.protobuf.GeneratedMessage + implements CompressionTableAdvertisementAckOrBuilder { + // Use CompressionTableAdvertisementAck.newBuilder() to construct. + private CompressionTableAdvertisementAck(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private CompressionTableAdvertisementAck(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final CompressionTableAdvertisementAck defaultInstance; + public static CompressionTableAdvertisementAck getDefaultInstance() { + return defaultInstance; + } + + public CompressionTableAdvertisementAck getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private CompressionTableAdvertisementAck( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + akka.remote.ArteryControlFormats.UniqueAddress.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = from_.toBuilder(); + } + from_ = input.readMessage(akka.remote.ArteryControlFormats.UniqueAddress.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(from_); + from_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + case 16: { + bitField0_ |= 0x00000002; + version_ = input.readUInt32(); + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_CompressionTableAdvertisementAck_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_CompressionTableAdvertisementAck_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck.class, akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public CompressionTableAdvertisementAck parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new CompressionTableAdvertisementAck(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required .UniqueAddress from = 1; + public static final int FROM_FIELD_NUMBER = 1; + private akka.remote.ArteryControlFormats.UniqueAddress from_; + /** + * required .UniqueAddress from = 1; + */ + public boolean hasFrom() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddress getFrom() { + return from_; + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getFromOrBuilder() { + return from_; + } + + // required uint32 version = 2; + public static final int VERSION_FIELD_NUMBER = 2; + private int version_; + /** + * required uint32 version = 2; + */ + public boolean hasVersion() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required uint32 version = 2; + */ + public int getVersion() { + return version_; + } + + private void initFields() { + from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + version_ = 0; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasFrom()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasVersion()) { + memoizedIsInitialized = 0; + return false; + } + if (!getFrom().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, from_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt32(2, version_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(1, from_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += akka.protobuf.CodedOutputStream + .computeUInt32Size(2, version_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code CompressionTableAdvertisementAck} + * + *
+     * CompressionProtocol.ActorRefCompressionAdvertisementAck
+     * CompressionProtocol.ClassManifestCompressionAdvertisementAck
+     * 
+ */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.ArteryControlFormats.CompressionTableAdvertisementAckOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_CompressionTableAdvertisementAck_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_CompressionTableAdvertisementAck_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck.class, akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck.Builder.class); + } + + // Construct using akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getFromFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (fromBuilder_ == null) { + from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + } else { + fromBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + version_ = 0; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.ArteryControlFormats.internal_static_CompressionTableAdvertisementAck_descriptor; + } + + public akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck getDefaultInstanceForType() { + return akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck.getDefaultInstance(); + } + + public akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck build() { + akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck buildPartial() { + akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck result = new akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (fromBuilder_ == null) { + result.from_ = from_; + } else { + result.from_ = fromBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.version_ = version_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck) { + return mergeFrom((akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck other) { + if (other == akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck.getDefaultInstance()) return this; + if (other.hasFrom()) { + mergeFrom(other.getFrom()); + } + if (other.hasVersion()) { + setVersion(other.getVersion()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasFrom()) { + + return false; + } + if (!hasVersion()) { + + return false; + } + if (!getFrom().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.ArteryControlFormats.CompressionTableAdvertisementAck) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required .UniqueAddress from = 1; + private akka.remote.ArteryControlFormats.UniqueAddress from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder> fromBuilder_; + /** + * required .UniqueAddress from = 1; + */ + public boolean hasFrom() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddress getFrom() { + if (fromBuilder_ == null) { + return from_; + } else { + return fromBuilder_.getMessage(); + } + } + /** + * required .UniqueAddress from = 1; + */ + public Builder setFrom(akka.remote.ArteryControlFormats.UniqueAddress value) { + if (fromBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + from_ = value; + onChanged(); + } else { + fromBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .UniqueAddress from = 1; + */ + public Builder setFrom( + akka.remote.ArteryControlFormats.UniqueAddress.Builder builderForValue) { + if (fromBuilder_ == null) { + from_ = builderForValue.build(); + onChanged(); + } else { + fromBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .UniqueAddress from = 1; + */ + public Builder mergeFrom(akka.remote.ArteryControlFormats.UniqueAddress value) { + if (fromBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + from_ != akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance()) { + from_ = + akka.remote.ArteryControlFormats.UniqueAddress.newBuilder(from_).mergeFrom(value).buildPartial(); + } else { + from_ = value; + } + onChanged(); + } else { + fromBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .UniqueAddress from = 1; + */ + public Builder clearFrom() { + if (fromBuilder_ == null) { + from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + onChanged(); + } else { + fromBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddress.Builder getFromBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getFromFieldBuilder().getBuilder(); + } + /** + * required .UniqueAddress from = 1; + */ + public akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getFromOrBuilder() { + if (fromBuilder_ != null) { + return fromBuilder_.getMessageOrBuilder(); + } else { + return from_; + } + } + /** + * required .UniqueAddress from = 1; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder> + getFromFieldBuilder() { + if (fromBuilder_ == null) { + fromBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder>( + from_, + getParentForChildren(), + isClean()); + from_ = null; + } + return fromBuilder_; + } + + // required uint32 version = 2; + private int version_ ; + /** + * required uint32 version = 2; + */ + public boolean hasVersion() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required uint32 version = 2; + */ + public int getVersion() { + return version_; + } + /** + * required uint32 version = 2; + */ + public Builder setVersion(int value) { + bitField0_ |= 0x00000002; + version_ = value; + onChanged(); + return this; + } + /** + * required uint32 version = 2; + */ + public Builder clearVersion() { + bitField0_ = (bitField0_ & ~0x00000002); + version_ = 0; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:CompressionTableAdvertisementAck) + } + + static { + defaultInstance = new CompressionTableAdvertisementAck(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:CompressionTableAdvertisementAck) + } + + public interface SystemMessageEnvelopeOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required bytes message = 1; + /** + * required bytes message = 1; + */ + boolean hasMessage(); + /** + * required bytes message = 1; + */ + akka.protobuf.ByteString getMessage(); + + // required int32 serializerId = 2; + /** + * required int32 serializerId = 2; + */ + boolean hasSerializerId(); + /** + * required int32 serializerId = 2; + */ + int getSerializerId(); + + // optional bytes messageManifest = 3; + /** + * optional bytes messageManifest = 3; + */ + boolean hasMessageManifest(); + /** + * optional bytes messageManifest = 3; + */ + akka.protobuf.ByteString getMessageManifest(); + + // required uint64 seqNo = 4; + /** + * required uint64 seqNo = 4; + */ + boolean hasSeqNo(); + /** + * required uint64 seqNo = 4; + */ + long getSeqNo(); + + // required .UniqueAddress ackReplyTo = 5; + /** + * required .UniqueAddress ackReplyTo = 5; + */ + boolean hasAckReplyTo(); + /** + * required .UniqueAddress ackReplyTo = 5; + */ + akka.remote.ArteryControlFormats.UniqueAddress getAckReplyTo(); + /** + * required .UniqueAddress ackReplyTo = 5; + */ + akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getAckReplyToOrBuilder(); + } + /** + * Protobuf type {@code SystemMessageEnvelope} + * + *
+   * SystemMessageDelivery.SystemMessageEnvelope
+   * 
+ */ + public static final class SystemMessageEnvelope extends + akka.protobuf.GeneratedMessage + implements SystemMessageEnvelopeOrBuilder { + // Use SystemMessageEnvelope.newBuilder() to construct. + private SystemMessageEnvelope(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private SystemMessageEnvelope(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final SystemMessageEnvelope defaultInstance; + public static SystemMessageEnvelope getDefaultInstance() { + return defaultInstance; + } + + public SystemMessageEnvelope getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private SystemMessageEnvelope( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + message_ = input.readBytes(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + serializerId_ = input.readInt32(); + break; + } + case 26: { + bitField0_ |= 0x00000004; + messageManifest_ = input.readBytes(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + seqNo_ = input.readUInt64(); + break; + } + case 42: { + akka.remote.ArteryControlFormats.UniqueAddress.Builder subBuilder = null; + if (((bitField0_ & 0x00000010) == 0x00000010)) { + subBuilder = ackReplyTo_.toBuilder(); + } + ackReplyTo_ = input.readMessage(akka.remote.ArteryControlFormats.UniqueAddress.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(ackReplyTo_); + ackReplyTo_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000010; + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_SystemMessageEnvelope_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_SystemMessageEnvelope_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.SystemMessageEnvelope.class, akka.remote.ArteryControlFormats.SystemMessageEnvelope.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public SystemMessageEnvelope parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new SystemMessageEnvelope(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required bytes message = 1; + public static final int MESSAGE_FIELD_NUMBER = 1; + private akka.protobuf.ByteString message_; + /** + * required bytes message = 1; + */ + public boolean hasMessage() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required bytes message = 1; + */ + public akka.protobuf.ByteString getMessage() { + return message_; + } + + // required int32 serializerId = 2; + public static final int SERIALIZERID_FIELD_NUMBER = 2; + private int serializerId_; + /** + * required int32 serializerId = 2; + */ + public boolean hasSerializerId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required int32 serializerId = 2; + */ + public int getSerializerId() { + return serializerId_; + } + + // optional bytes messageManifest = 3; + public static final int MESSAGEMANIFEST_FIELD_NUMBER = 3; + private akka.protobuf.ByteString messageManifest_; + /** + * optional bytes messageManifest = 3; + */ + public boolean hasMessageManifest() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional bytes messageManifest = 3; + */ + public akka.protobuf.ByteString getMessageManifest() { + return messageManifest_; + } + + // required uint64 seqNo = 4; + public static final int SEQNO_FIELD_NUMBER = 4; + private long seqNo_; + /** + * required uint64 seqNo = 4; + */ + public boolean hasSeqNo() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * required uint64 seqNo = 4; + */ + public long getSeqNo() { + return seqNo_; + } + + // required .UniqueAddress ackReplyTo = 5; + public static final int ACKREPLYTO_FIELD_NUMBER = 5; + private akka.remote.ArteryControlFormats.UniqueAddress ackReplyTo_; + /** + * required .UniqueAddress ackReplyTo = 5; + */ + public boolean hasAckReplyTo() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + /** + * required .UniqueAddress ackReplyTo = 5; + */ + public akka.remote.ArteryControlFormats.UniqueAddress getAckReplyTo() { + return ackReplyTo_; + } + /** + * required .UniqueAddress ackReplyTo = 5; + */ + public akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getAckReplyToOrBuilder() { + return ackReplyTo_; + } + + private void initFields() { + message_ = akka.protobuf.ByteString.EMPTY; + serializerId_ = 0; + messageManifest_ = akka.protobuf.ByteString.EMPTY; + seqNo_ = 0L; + ackReplyTo_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasMessage()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasSerializerId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasSeqNo()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasAckReplyTo()) { + memoizedIsInitialized = 0; + return false; + } + if (!getAckReplyTo().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, message_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeInt32(2, serializerId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, messageManifest_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt64(4, seqNo_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeMessage(5, ackReplyTo_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeBytesSize(1, message_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += akka.protobuf.CodedOutputStream + .computeInt32Size(2, serializerId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += akka.protobuf.CodedOutputStream + .computeBytesSize(3, messageManifest_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += akka.protobuf.CodedOutputStream + .computeUInt64Size(4, seqNo_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(5, ackReplyTo_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.ArteryControlFormats.SystemMessageEnvelope parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.SystemMessageEnvelope parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.SystemMessageEnvelope parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.SystemMessageEnvelope parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.SystemMessageEnvelope parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.SystemMessageEnvelope parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.SystemMessageEnvelope parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.ArteryControlFormats.SystemMessageEnvelope parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.SystemMessageEnvelope parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.SystemMessageEnvelope parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.ArteryControlFormats.SystemMessageEnvelope prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code SystemMessageEnvelope} + * + *
+     * SystemMessageDelivery.SystemMessageEnvelope
+     * 
+ */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.ArteryControlFormats.SystemMessageEnvelopeOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_SystemMessageEnvelope_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_SystemMessageEnvelope_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.SystemMessageEnvelope.class, akka.remote.ArteryControlFormats.SystemMessageEnvelope.Builder.class); + } + + // Construct using akka.remote.ArteryControlFormats.SystemMessageEnvelope.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getAckReplyToFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + message_ = akka.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000001); + serializerId_ = 0; + bitField0_ = (bitField0_ & ~0x00000002); + messageManifest_ = akka.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000004); + seqNo_ = 0L; + bitField0_ = (bitField0_ & ~0x00000008); + if (ackReplyToBuilder_ == null) { + ackReplyTo_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + } else { + ackReplyToBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.ArteryControlFormats.internal_static_SystemMessageEnvelope_descriptor; + } + + public akka.remote.ArteryControlFormats.SystemMessageEnvelope getDefaultInstanceForType() { + return akka.remote.ArteryControlFormats.SystemMessageEnvelope.getDefaultInstance(); + } + + public akka.remote.ArteryControlFormats.SystemMessageEnvelope build() { + akka.remote.ArteryControlFormats.SystemMessageEnvelope result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.ArteryControlFormats.SystemMessageEnvelope buildPartial() { + akka.remote.ArteryControlFormats.SystemMessageEnvelope result = new akka.remote.ArteryControlFormats.SystemMessageEnvelope(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.message_ = message_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.serializerId_ = serializerId_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.messageManifest_ = messageManifest_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.seqNo_ = seqNo_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + if (ackReplyToBuilder_ == null) { + result.ackReplyTo_ = ackReplyTo_; + } else { + result.ackReplyTo_ = ackReplyToBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.ArteryControlFormats.SystemMessageEnvelope) { + return mergeFrom((akka.remote.ArteryControlFormats.SystemMessageEnvelope)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.ArteryControlFormats.SystemMessageEnvelope other) { + if (other == akka.remote.ArteryControlFormats.SystemMessageEnvelope.getDefaultInstance()) return this; + if (other.hasMessage()) { + setMessage(other.getMessage()); + } + if (other.hasSerializerId()) { + setSerializerId(other.getSerializerId()); + } + if (other.hasMessageManifest()) { + setMessageManifest(other.getMessageManifest()); + } + if (other.hasSeqNo()) { + setSeqNo(other.getSeqNo()); + } + if (other.hasAckReplyTo()) { + mergeAckReplyTo(other.getAckReplyTo()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasMessage()) { + + return false; + } + if (!hasSerializerId()) { + + return false; + } + if (!hasSeqNo()) { + + return false; + } + if (!hasAckReplyTo()) { + + return false; + } + if (!getAckReplyTo().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.ArteryControlFormats.SystemMessageEnvelope parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.ArteryControlFormats.SystemMessageEnvelope) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required bytes message = 1; + private akka.protobuf.ByteString message_ = akka.protobuf.ByteString.EMPTY; + /** + * required bytes message = 1; + */ + public boolean hasMessage() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required bytes message = 1; + */ + public akka.protobuf.ByteString getMessage() { + return message_; + } + /** + * required bytes message = 1; + */ + public Builder setMessage(akka.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + message_ = value; + onChanged(); + return this; + } + /** + * required bytes message = 1; + */ + public Builder clearMessage() { + bitField0_ = (bitField0_ & ~0x00000001); + message_ = getDefaultInstance().getMessage(); + onChanged(); + return this; + } + + // required int32 serializerId = 2; + private int serializerId_ ; + /** + * required int32 serializerId = 2; + */ + public boolean hasSerializerId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required int32 serializerId = 2; + */ + public int getSerializerId() { + return serializerId_; + } + /** + * required int32 serializerId = 2; + */ + public Builder setSerializerId(int value) { + bitField0_ |= 0x00000002; + serializerId_ = value; + onChanged(); + return this; + } + /** + * required int32 serializerId = 2; + */ + public Builder clearSerializerId() { + bitField0_ = (bitField0_ & ~0x00000002); + serializerId_ = 0; + onChanged(); + return this; + } + + // optional bytes messageManifest = 3; + private akka.protobuf.ByteString messageManifest_ = akka.protobuf.ByteString.EMPTY; + /** + * optional bytes messageManifest = 3; + */ + public boolean hasMessageManifest() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional bytes messageManifest = 3; + */ + public akka.protobuf.ByteString getMessageManifest() { + return messageManifest_; + } + /** + * optional bytes messageManifest = 3; + */ + public Builder setMessageManifest(akka.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + messageManifest_ = value; + onChanged(); + return this; + } + /** + * optional bytes messageManifest = 3; + */ + public Builder clearMessageManifest() { + bitField0_ = (bitField0_ & ~0x00000004); + messageManifest_ = getDefaultInstance().getMessageManifest(); + onChanged(); + return this; + } + + // required uint64 seqNo = 4; + private long seqNo_ ; + /** + * required uint64 seqNo = 4; + */ + public boolean hasSeqNo() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * required uint64 seqNo = 4; + */ + public long getSeqNo() { + return seqNo_; + } + /** + * required uint64 seqNo = 4; + */ + public Builder setSeqNo(long value) { + bitField0_ |= 0x00000008; + seqNo_ = value; + onChanged(); + return this; + } + /** + * required uint64 seqNo = 4; + */ + public Builder clearSeqNo() { + bitField0_ = (bitField0_ & ~0x00000008); + seqNo_ = 0L; + onChanged(); + return this; + } + + // required .UniqueAddress ackReplyTo = 5; + private akka.remote.ArteryControlFormats.UniqueAddress ackReplyTo_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder> ackReplyToBuilder_; + /** + * required .UniqueAddress ackReplyTo = 5; + */ + public boolean hasAckReplyTo() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + /** + * required .UniqueAddress ackReplyTo = 5; + */ + public akka.remote.ArteryControlFormats.UniqueAddress getAckReplyTo() { + if (ackReplyToBuilder_ == null) { + return ackReplyTo_; + } else { + return ackReplyToBuilder_.getMessage(); + } + } + /** + * required .UniqueAddress ackReplyTo = 5; + */ + public Builder setAckReplyTo(akka.remote.ArteryControlFormats.UniqueAddress value) { + if (ackReplyToBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ackReplyTo_ = value; + onChanged(); + } else { + ackReplyToBuilder_.setMessage(value); + } + bitField0_ |= 0x00000010; + return this; + } + /** + * required .UniqueAddress ackReplyTo = 5; + */ + public Builder setAckReplyTo( + akka.remote.ArteryControlFormats.UniqueAddress.Builder builderForValue) { + if (ackReplyToBuilder_ == null) { + ackReplyTo_ = builderForValue.build(); + onChanged(); + } else { + ackReplyToBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000010; + return this; + } + /** + * required .UniqueAddress ackReplyTo = 5; + */ + public Builder mergeAckReplyTo(akka.remote.ArteryControlFormats.UniqueAddress value) { + if (ackReplyToBuilder_ == null) { + if (((bitField0_ & 0x00000010) == 0x00000010) && + ackReplyTo_ != akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance()) { + ackReplyTo_ = + akka.remote.ArteryControlFormats.UniqueAddress.newBuilder(ackReplyTo_).mergeFrom(value).buildPartial(); + } else { + ackReplyTo_ = value; + } + onChanged(); + } else { + ackReplyToBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000010; + return this; + } + /** + * required .UniqueAddress ackReplyTo = 5; + */ + public Builder clearAckReplyTo() { + if (ackReplyToBuilder_ == null) { + ackReplyTo_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + onChanged(); + } else { + ackReplyToBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + /** + * required .UniqueAddress ackReplyTo = 5; + */ + public akka.remote.ArteryControlFormats.UniqueAddress.Builder getAckReplyToBuilder() { + bitField0_ |= 0x00000010; + onChanged(); + return getAckReplyToFieldBuilder().getBuilder(); + } + /** + * required .UniqueAddress ackReplyTo = 5; + */ + public akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getAckReplyToOrBuilder() { + if (ackReplyToBuilder_ != null) { + return ackReplyToBuilder_.getMessageOrBuilder(); + } else { + return ackReplyTo_; + } + } + /** + * required .UniqueAddress ackReplyTo = 5; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder> + getAckReplyToFieldBuilder() { + if (ackReplyToBuilder_ == null) { + ackReplyToBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder>( + ackReplyTo_, + getParentForChildren(), + isClean()); + ackReplyTo_ = null; + } + return ackReplyToBuilder_; + } + + // @@protoc_insertion_point(builder_scope:SystemMessageEnvelope) + } + + static { + defaultInstance = new SystemMessageEnvelope(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:SystemMessageEnvelope) + } + + public interface SystemMessageDeliveryAckOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required uint64 seqNo = 1; + /** + * required uint64 seqNo = 1; + */ + boolean hasSeqNo(); + /** + * required uint64 seqNo = 1; + */ + long getSeqNo(); + + // required .UniqueAddress from = 2; + /** + * required .UniqueAddress from = 2; + */ + boolean hasFrom(); + /** + * required .UniqueAddress from = 2; + */ + akka.remote.ArteryControlFormats.UniqueAddress getFrom(); + /** + * required .UniqueAddress from = 2; + */ + akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getFromOrBuilder(); + } + /** + * Protobuf type {@code SystemMessageDeliveryAck} + * + *
+   * SystemMessageDelivery.Ack
+   * SystemMessageDelivery.Nack
+   * 
+ */ + public static final class SystemMessageDeliveryAck extends + akka.protobuf.GeneratedMessage + implements SystemMessageDeliveryAckOrBuilder { + // Use SystemMessageDeliveryAck.newBuilder() to construct. + private SystemMessageDeliveryAck(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private SystemMessageDeliveryAck(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final SystemMessageDeliveryAck defaultInstance; + public static SystemMessageDeliveryAck getDefaultInstance() { + return defaultInstance; + } + + public SystemMessageDeliveryAck getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private SystemMessageDeliveryAck( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + seqNo_ = input.readUInt64(); + break; + } + case 18: { + akka.remote.ArteryControlFormats.UniqueAddress.Builder subBuilder = null; + if (((bitField0_ & 0x00000002) == 0x00000002)) { + subBuilder = from_.toBuilder(); + } + from_ = input.readMessage(akka.remote.ArteryControlFormats.UniqueAddress.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(from_); + from_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000002; + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_SystemMessageDeliveryAck_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_SystemMessageDeliveryAck_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.SystemMessageDeliveryAck.class, akka.remote.ArteryControlFormats.SystemMessageDeliveryAck.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public SystemMessageDeliveryAck parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new SystemMessageDeliveryAck(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required uint64 seqNo = 1; + public static final int SEQNO_FIELD_NUMBER = 1; + private long seqNo_; + /** + * required uint64 seqNo = 1; + */ + public boolean hasSeqNo() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required uint64 seqNo = 1; + */ + public long getSeqNo() { + return seqNo_; + } + + // required .UniqueAddress from = 2; + public static final int FROM_FIELD_NUMBER = 2; + private akka.remote.ArteryControlFormats.UniqueAddress from_; + /** + * required .UniqueAddress from = 2; + */ + public boolean hasFrom() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required .UniqueAddress from = 2; + */ + public akka.remote.ArteryControlFormats.UniqueAddress getFrom() { + return from_; + } + /** + * required .UniqueAddress from = 2; + */ + public akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getFromOrBuilder() { + return from_; + } + + private void initFields() { + seqNo_ = 0L; + from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasSeqNo()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasFrom()) { + memoizedIsInitialized = 0; + return false; + } + if (!getFrom().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeUInt64(1, seqNo_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, from_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeUInt64Size(1, seqNo_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(2, from_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.ArteryControlFormats.SystemMessageDeliveryAck parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.SystemMessageDeliveryAck parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.SystemMessageDeliveryAck parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.SystemMessageDeliveryAck parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.SystemMessageDeliveryAck parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.SystemMessageDeliveryAck parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.SystemMessageDeliveryAck parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.ArteryControlFormats.SystemMessageDeliveryAck parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.SystemMessageDeliveryAck parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.SystemMessageDeliveryAck parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.ArteryControlFormats.SystemMessageDeliveryAck prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code SystemMessageDeliveryAck} + * + *
+     * SystemMessageDelivery.Ack
+     * SystemMessageDelivery.Nack
+     * 
+ */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.ArteryControlFormats.SystemMessageDeliveryAckOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_SystemMessageDeliveryAck_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_SystemMessageDeliveryAck_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.SystemMessageDeliveryAck.class, akka.remote.ArteryControlFormats.SystemMessageDeliveryAck.Builder.class); + } + + // Construct using akka.remote.ArteryControlFormats.SystemMessageDeliveryAck.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getFromFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + seqNo_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + if (fromBuilder_ == null) { + from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + } else { + fromBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.ArteryControlFormats.internal_static_SystemMessageDeliveryAck_descriptor; + } + + public akka.remote.ArteryControlFormats.SystemMessageDeliveryAck getDefaultInstanceForType() { + return akka.remote.ArteryControlFormats.SystemMessageDeliveryAck.getDefaultInstance(); + } + + public akka.remote.ArteryControlFormats.SystemMessageDeliveryAck build() { + akka.remote.ArteryControlFormats.SystemMessageDeliveryAck result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.ArteryControlFormats.SystemMessageDeliveryAck buildPartial() { + akka.remote.ArteryControlFormats.SystemMessageDeliveryAck result = new akka.remote.ArteryControlFormats.SystemMessageDeliveryAck(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.seqNo_ = seqNo_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (fromBuilder_ == null) { + result.from_ = from_; + } else { + result.from_ = fromBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.ArteryControlFormats.SystemMessageDeliveryAck) { + return mergeFrom((akka.remote.ArteryControlFormats.SystemMessageDeliveryAck)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.ArteryControlFormats.SystemMessageDeliveryAck other) { + if (other == akka.remote.ArteryControlFormats.SystemMessageDeliveryAck.getDefaultInstance()) return this; + if (other.hasSeqNo()) { + setSeqNo(other.getSeqNo()); + } + if (other.hasFrom()) { + mergeFrom(other.getFrom()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasSeqNo()) { + + return false; + } + if (!hasFrom()) { + + return false; + } + if (!getFrom().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.ArteryControlFormats.SystemMessageDeliveryAck parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.ArteryControlFormats.SystemMessageDeliveryAck) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required uint64 seqNo = 1; + private long seqNo_ ; + /** + * required uint64 seqNo = 1; + */ + public boolean hasSeqNo() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required uint64 seqNo = 1; + */ + public long getSeqNo() { + return seqNo_; + } + /** + * required uint64 seqNo = 1; + */ + public Builder setSeqNo(long value) { + bitField0_ |= 0x00000001; + seqNo_ = value; + onChanged(); + return this; + } + /** + * required uint64 seqNo = 1; + */ + public Builder clearSeqNo() { + bitField0_ = (bitField0_ & ~0x00000001); + seqNo_ = 0L; + onChanged(); + return this; + } + + // required .UniqueAddress from = 2; + private akka.remote.ArteryControlFormats.UniqueAddress from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder> fromBuilder_; + /** + * required .UniqueAddress from = 2; + */ + public boolean hasFrom() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required .UniqueAddress from = 2; + */ + public akka.remote.ArteryControlFormats.UniqueAddress getFrom() { + if (fromBuilder_ == null) { + return from_; + } else { + return fromBuilder_.getMessage(); + } + } + /** + * required .UniqueAddress from = 2; + */ + public Builder setFrom(akka.remote.ArteryControlFormats.UniqueAddress value) { + if (fromBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + from_ = value; + onChanged(); + } else { + fromBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .UniqueAddress from = 2; + */ + public Builder setFrom( + akka.remote.ArteryControlFormats.UniqueAddress.Builder builderForValue) { + if (fromBuilder_ == null) { + from_ = builderForValue.build(); + onChanged(); + } else { + fromBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .UniqueAddress from = 2; + */ + public Builder mergeFrom(akka.remote.ArteryControlFormats.UniqueAddress value) { + if (fromBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + from_ != akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance()) { + from_ = + akka.remote.ArteryControlFormats.UniqueAddress.newBuilder(from_).mergeFrom(value).buildPartial(); + } else { + from_ = value; + } + onChanged(); + } else { + fromBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .UniqueAddress from = 2; + */ + public Builder clearFrom() { + if (fromBuilder_ == null) { + from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + onChanged(); + } else { + fromBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + /** + * required .UniqueAddress from = 2; + */ + public akka.remote.ArteryControlFormats.UniqueAddress.Builder getFromBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getFromFieldBuilder().getBuilder(); + } + /** + * required .UniqueAddress from = 2; + */ + public akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getFromOrBuilder() { + if (fromBuilder_ != null) { + return fromBuilder_.getMessageOrBuilder(); + } else { + return from_; + } + } + /** + * required .UniqueAddress from = 2; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder> + getFromFieldBuilder() { + if (fromBuilder_ == null) { + fromBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.UniqueAddress, akka.remote.ArteryControlFormats.UniqueAddress.Builder, akka.remote.ArteryControlFormats.UniqueAddressOrBuilder>( + from_, + getParentForChildren(), + isClean()); + from_ = null; + } + return fromBuilder_; + } + + // @@protoc_insertion_point(builder_scope:SystemMessageDeliveryAck) + } + + static { + defaultInstance = new SystemMessageDeliveryAck(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:SystemMessageDeliveryAck) + } + + public interface AddressOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required string protocol = 1; + /** + * required string protocol = 1; + */ + boolean hasProtocol(); + /** + * required string protocol = 1; + */ + java.lang.String getProtocol(); + /** + * required string protocol = 1; + */ + akka.protobuf.ByteString + getProtocolBytes(); + + // required string system = 2; + /** + * required string system = 2; + */ + boolean hasSystem(); + /** + * required string system = 2; + */ + java.lang.String getSystem(); + /** + * required string system = 2; + */ + akka.protobuf.ByteString + getSystemBytes(); + + // required string hostname = 3; + /** + * required string hostname = 3; + */ + boolean hasHostname(); + /** + * required string hostname = 3; + */ + java.lang.String getHostname(); + /** + * required string hostname = 3; + */ + akka.protobuf.ByteString + getHostnameBytes(); + + // required uint32 port = 4; + /** + * required uint32 port = 4; + */ + boolean hasPort(); + /** + * required uint32 port = 4; + */ + int getPort(); + } + /** + * Protobuf type {@code Address} + * + *
+   **
+   * Defines a remote address.
+   * 
+ */ + public static final class Address extends + akka.protobuf.GeneratedMessage + implements AddressOrBuilder { + // Use Address.newBuilder() to construct. + private Address(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private Address(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final Address defaultInstance; + public static Address getDefaultInstance() { + return defaultInstance; + } + + public Address getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private Address( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + protocol_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + system_ = input.readBytes(); + break; + } + case 26: { + bitField0_ |= 0x00000004; + hostname_ = input.readBytes(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + port_ = input.readUInt32(); + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_Address_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_Address_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.Address.class, akka.remote.ArteryControlFormats.Address.Builder.class); + } + + public static akka.protobuf.Parser
PARSER = + new akka.protobuf.AbstractParser
() { + public Address parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new Address(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser
getParserForType() { + return PARSER; + } + + private int bitField0_; + // required string protocol = 1; + public static final int PROTOCOL_FIELD_NUMBER = 1; + private java.lang.Object protocol_; + /** + * required string protocol = 1; + */ + public boolean hasProtocol() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required string protocol = 1; + */ + public java.lang.String getProtocol() { + java.lang.Object ref = protocol_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + akka.protobuf.ByteString bs = + (akka.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + protocol_ = s; + } + return s; + } + } + /** + * required string protocol = 1; + */ + public akka.protobuf.ByteString + getProtocolBytes() { + java.lang.Object ref = protocol_; + if (ref instanceof java.lang.String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + protocol_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + + // required string system = 2; + public static final int SYSTEM_FIELD_NUMBER = 2; + private java.lang.Object system_; + /** + * required string system = 2; + */ + public boolean hasSystem() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required string system = 2; + */ + public java.lang.String getSystem() { + java.lang.Object ref = system_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + akka.protobuf.ByteString bs = + (akka.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + system_ = s; + } + return s; + } + } + /** + * required string system = 2; + */ + public akka.protobuf.ByteString + getSystemBytes() { + java.lang.Object ref = system_; + if (ref instanceof java.lang.String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + system_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + + // required string hostname = 3; + public static final int HOSTNAME_FIELD_NUMBER = 3; + private java.lang.Object hostname_; + /** + * required string hostname = 3; + */ + public boolean hasHostname() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required string hostname = 3; + */ + public java.lang.String getHostname() { + java.lang.Object ref = hostname_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + akka.protobuf.ByteString bs = + (akka.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + hostname_ = s; + } + return s; + } + } + /** + * required string hostname = 3; + */ + public akka.protobuf.ByteString + getHostnameBytes() { + java.lang.Object ref = hostname_; + if (ref instanceof java.lang.String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + hostname_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + + // required uint32 port = 4; + public static final int PORT_FIELD_NUMBER = 4; + private int port_; + /** + * required uint32 port = 4; + */ + public boolean hasPort() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * required uint32 port = 4; + */ + public int getPort() { + return port_; + } + + private void initFields() { + protocol_ = ""; + system_ = ""; + hostname_ = ""; + port_ = 0; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasProtocol()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasSystem()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasHostname()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasPort()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getProtocolBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getSystemBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, getHostnameBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt32(4, port_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeBytesSize(1, getProtocolBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += akka.protobuf.CodedOutputStream + .computeBytesSize(2, getSystemBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += akka.protobuf.CodedOutputStream + .computeBytesSize(3, getHostnameBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += akka.protobuf.CodedOutputStream + .computeUInt32Size(4, port_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.ArteryControlFormats.Address parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.Address parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.Address parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.Address parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.Address parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.Address parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.Address parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.ArteryControlFormats.Address parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.Address parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.Address parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.ArteryControlFormats.Address prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code Address} + * + *
+     **
+     * Defines a remote address.
+     * 
+ */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.ArteryControlFormats.AddressOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_Address_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_Address_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.Address.class, akka.remote.ArteryControlFormats.Address.Builder.class); + } + + // Construct using akka.remote.ArteryControlFormats.Address.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + protocol_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + system_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + hostname_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + port_ = 0; + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.ArteryControlFormats.internal_static_Address_descriptor; + } + + public akka.remote.ArteryControlFormats.Address getDefaultInstanceForType() { + return akka.remote.ArteryControlFormats.Address.getDefaultInstance(); + } + + public akka.remote.ArteryControlFormats.Address build() { + akka.remote.ArteryControlFormats.Address result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.ArteryControlFormats.Address buildPartial() { + akka.remote.ArteryControlFormats.Address result = new akka.remote.ArteryControlFormats.Address(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.protocol_ = protocol_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.system_ = system_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.hostname_ = hostname_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.port_ = port_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.ArteryControlFormats.Address) { + return mergeFrom((akka.remote.ArteryControlFormats.Address)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.ArteryControlFormats.Address other) { + if (other == akka.remote.ArteryControlFormats.Address.getDefaultInstance()) return this; + if (other.hasProtocol()) { + bitField0_ |= 0x00000001; + protocol_ = other.protocol_; + onChanged(); + } + if (other.hasSystem()) { + bitField0_ |= 0x00000002; + system_ = other.system_; + onChanged(); + } + if (other.hasHostname()) { + bitField0_ |= 0x00000004; + hostname_ = other.hostname_; + onChanged(); + } + if (other.hasPort()) { + setPort(other.getPort()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasProtocol()) { + + return false; + } + if (!hasSystem()) { + + return false; + } + if (!hasHostname()) { + + return false; + } + if (!hasPort()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.ArteryControlFormats.Address parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.ArteryControlFormats.Address) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required string protocol = 1; + private java.lang.Object protocol_ = ""; + /** + * required string protocol = 1; + */ + public boolean hasProtocol() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required string protocol = 1; + */ + public java.lang.String getProtocol() { + java.lang.Object ref = protocol_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((akka.protobuf.ByteString) ref) + .toStringUtf8(); + protocol_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string protocol = 1; + */ + public akka.protobuf.ByteString + getProtocolBytes() { + java.lang.Object ref = protocol_; + if (ref instanceof String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + protocol_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + /** + * required string protocol = 1; + */ + public Builder setProtocol( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + protocol_ = value; + onChanged(); + return this; + } + /** + * required string protocol = 1; + */ + public Builder clearProtocol() { + bitField0_ = (bitField0_ & ~0x00000001); + protocol_ = getDefaultInstance().getProtocol(); + onChanged(); + return this; + } + /** + * required string protocol = 1; + */ + public Builder setProtocolBytes( + akka.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + protocol_ = value; + onChanged(); + return this; + } + + // required string system = 2; + private java.lang.Object system_ = ""; + /** + * required string system = 2; + */ + public boolean hasSystem() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required string system = 2; + */ + public java.lang.String getSystem() { + java.lang.Object ref = system_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((akka.protobuf.ByteString) ref) + .toStringUtf8(); + system_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string system = 2; + */ + public akka.protobuf.ByteString + getSystemBytes() { + java.lang.Object ref = system_; + if (ref instanceof String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + system_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + /** + * required string system = 2; + */ + public Builder setSystem( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + system_ = value; + onChanged(); + return this; + } + /** + * required string system = 2; + */ + public Builder clearSystem() { + bitField0_ = (bitField0_ & ~0x00000002); + system_ = getDefaultInstance().getSystem(); + onChanged(); + return this; + } + /** + * required string system = 2; + */ + public Builder setSystemBytes( + akka.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + system_ = value; + onChanged(); + return this; + } + + // required string hostname = 3; + private java.lang.Object hostname_ = ""; + /** + * required string hostname = 3; + */ + public boolean hasHostname() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required string hostname = 3; + */ + public java.lang.String getHostname() { + java.lang.Object ref = hostname_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((akka.protobuf.ByteString) ref) + .toStringUtf8(); + hostname_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string hostname = 3; + */ + public akka.protobuf.ByteString + getHostnameBytes() { + java.lang.Object ref = hostname_; + if (ref instanceof String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + hostname_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + /** + * required string hostname = 3; + */ + public Builder setHostname( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + hostname_ = value; + onChanged(); + return this; + } + /** + * required string hostname = 3; + */ + public Builder clearHostname() { + bitField0_ = (bitField0_ & ~0x00000004); + hostname_ = getDefaultInstance().getHostname(); + onChanged(); + return this; + } + /** + * required string hostname = 3; + */ + public Builder setHostnameBytes( + akka.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + hostname_ = value; + onChanged(); + return this; + } + + // required uint32 port = 4; + private int port_ ; + /** + * required uint32 port = 4; + */ + public boolean hasPort() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * required uint32 port = 4; + */ + public int getPort() { + return port_; + } + /** + * required uint32 port = 4; + */ + public Builder setPort(int value) { + bitField0_ |= 0x00000008; + port_ = value; + onChanged(); + return this; + } + /** + * required uint32 port = 4; + */ + public Builder clearPort() { + bitField0_ = (bitField0_ & ~0x00000008); + port_ = 0; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:Address) + } + + static { + defaultInstance = new Address(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:Address) + } + + public interface UniqueAddressOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required .Address address = 1; + /** + * required .Address address = 1; + */ + boolean hasAddress(); + /** + * required .Address address = 1; + */ + akka.remote.ArteryControlFormats.Address getAddress(); + /** + * required .Address address = 1; + */ + akka.remote.ArteryControlFormats.AddressOrBuilder getAddressOrBuilder(); + + // required uint64 uid = 2; + /** + * required uint64 uid = 2; + */ + boolean hasUid(); + /** + * required uint64 uid = 2; + */ + long getUid(); + } + /** + * Protobuf type {@code UniqueAddress} + * + *
+   **
+   * Defines a remote address with uid.
+   * 
+ */ + public static final class UniqueAddress extends + akka.protobuf.GeneratedMessage + implements UniqueAddressOrBuilder { + // Use UniqueAddress.newBuilder() to construct. + private UniqueAddress(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private UniqueAddress(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final UniqueAddress defaultInstance; + public static UniqueAddress getDefaultInstance() { + return defaultInstance; + } + + public UniqueAddress getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private UniqueAddress( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + akka.remote.ArteryControlFormats.Address.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = address_.toBuilder(); + } + address_ = input.readMessage(akka.remote.ArteryControlFormats.Address.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(address_); + address_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + case 16: { + bitField0_ |= 0x00000002; + uid_ = input.readUInt64(); + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_UniqueAddress_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_UniqueAddress_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.UniqueAddress.class, akka.remote.ArteryControlFormats.UniqueAddress.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public UniqueAddress parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new UniqueAddress(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required .Address address = 1; + public static final int ADDRESS_FIELD_NUMBER = 1; + private akka.remote.ArteryControlFormats.Address address_; + /** + * required .Address address = 1; + */ + public boolean hasAddress() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .Address address = 1; + */ + public akka.remote.ArteryControlFormats.Address getAddress() { + return address_; + } + /** + * required .Address address = 1; + */ + public akka.remote.ArteryControlFormats.AddressOrBuilder getAddressOrBuilder() { + return address_; + } + + // required uint64 uid = 2; + public static final int UID_FIELD_NUMBER = 2; + private long uid_; + /** + * required uint64 uid = 2; + */ + public boolean hasUid() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required uint64 uid = 2; + */ + public long getUid() { + return uid_; + } + + private void initFields() { + address_ = akka.remote.ArteryControlFormats.Address.getDefaultInstance(); + uid_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasAddress()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasUid()) { + memoizedIsInitialized = 0; + return false; + } + if (!getAddress().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, address_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, uid_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(1, address_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += akka.protobuf.CodedOutputStream + .computeUInt64Size(2, uid_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.ArteryControlFormats.UniqueAddress parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.UniqueAddress parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.UniqueAddress parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.UniqueAddress parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.UniqueAddress parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.UniqueAddress parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.UniqueAddress parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.ArteryControlFormats.UniqueAddress parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.UniqueAddress parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.UniqueAddress parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.ArteryControlFormats.UniqueAddress prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code UniqueAddress} + * + *
+     **
+     * Defines a remote address with uid.
+     * 
+ */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.ArteryControlFormats.UniqueAddressOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_UniqueAddress_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_UniqueAddress_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.UniqueAddress.class, akka.remote.ArteryControlFormats.UniqueAddress.Builder.class); + } + + // Construct using akka.remote.ArteryControlFormats.UniqueAddress.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getAddressFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (addressBuilder_ == null) { + address_ = akka.remote.ArteryControlFormats.Address.getDefaultInstance(); + } else { + addressBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + uid_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.ArteryControlFormats.internal_static_UniqueAddress_descriptor; + } + + public akka.remote.ArteryControlFormats.UniqueAddress getDefaultInstanceForType() { + return akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + } + + public akka.remote.ArteryControlFormats.UniqueAddress build() { + akka.remote.ArteryControlFormats.UniqueAddress result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.ArteryControlFormats.UniqueAddress buildPartial() { + akka.remote.ArteryControlFormats.UniqueAddress result = new akka.remote.ArteryControlFormats.UniqueAddress(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (addressBuilder_ == null) { + result.address_ = address_; + } else { + result.address_ = addressBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.uid_ = uid_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.ArteryControlFormats.UniqueAddress) { + return mergeFrom((akka.remote.ArteryControlFormats.UniqueAddress)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.ArteryControlFormats.UniqueAddress other) { + if (other == akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance()) return this; + if (other.hasAddress()) { + mergeAddress(other.getAddress()); + } + if (other.hasUid()) { + setUid(other.getUid()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasAddress()) { + + return false; + } + if (!hasUid()) { + + return false; + } + if (!getAddress().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.ArteryControlFormats.UniqueAddress parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.ArteryControlFormats.UniqueAddress) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required .Address address = 1; + private akka.remote.ArteryControlFormats.Address address_ = akka.remote.ArteryControlFormats.Address.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.Address, akka.remote.ArteryControlFormats.Address.Builder, akka.remote.ArteryControlFormats.AddressOrBuilder> addressBuilder_; + /** + * required .Address address = 1; + */ + public boolean hasAddress() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .Address address = 1; + */ + public akka.remote.ArteryControlFormats.Address getAddress() { + if (addressBuilder_ == null) { + return address_; + } else { + return addressBuilder_.getMessage(); + } + } + /** + * required .Address address = 1; + */ + public Builder setAddress(akka.remote.ArteryControlFormats.Address value) { + if (addressBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + address_ = value; + onChanged(); + } else { + addressBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .Address address = 1; + */ + public Builder setAddress( + akka.remote.ArteryControlFormats.Address.Builder builderForValue) { + if (addressBuilder_ == null) { + address_ = builderForValue.build(); + onChanged(); + } else { + addressBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .Address address = 1; + */ + public Builder mergeAddress(akka.remote.ArteryControlFormats.Address value) { + if (addressBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + address_ != akka.remote.ArteryControlFormats.Address.getDefaultInstance()) { + address_ = + akka.remote.ArteryControlFormats.Address.newBuilder(address_).mergeFrom(value).buildPartial(); + } else { + address_ = value; + } + onChanged(); + } else { + addressBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .Address address = 1; + */ + public Builder clearAddress() { + if (addressBuilder_ == null) { + address_ = akka.remote.ArteryControlFormats.Address.getDefaultInstance(); + onChanged(); + } else { + addressBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * required .Address address = 1; + */ + public akka.remote.ArteryControlFormats.Address.Builder getAddressBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getAddressFieldBuilder().getBuilder(); + } + /** + * required .Address address = 1; + */ + public akka.remote.ArteryControlFormats.AddressOrBuilder getAddressOrBuilder() { + if (addressBuilder_ != null) { + return addressBuilder_.getMessageOrBuilder(); + } else { + return address_; + } + } + /** + * required .Address address = 1; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.Address, akka.remote.ArteryControlFormats.Address.Builder, akka.remote.ArteryControlFormats.AddressOrBuilder> + getAddressFieldBuilder() { + if (addressBuilder_ == null) { + addressBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.ArteryControlFormats.Address, akka.remote.ArteryControlFormats.Address.Builder, akka.remote.ArteryControlFormats.AddressOrBuilder>( + address_, + getParentForChildren(), + isClean()); + address_ = null; + } + return addressBuilder_; + } + + // required uint64 uid = 2; + private long uid_ ; + /** + * required uint64 uid = 2; + */ + public boolean hasUid() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required uint64 uid = 2; + */ + public long getUid() { + return uid_; + } + /** + * required uint64 uid = 2; + */ + public Builder setUid(long value) { + bitField0_ |= 0x00000002; + uid_ = value; + onChanged(); + return this; + } + /** + * required uint64 uid = 2; + */ + public Builder clearUid() { + bitField0_ = (bitField0_ & ~0x00000002); + uid_ = 0L; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:UniqueAddress) + } + + static { + defaultInstance = new UniqueAddress(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:UniqueAddress) + } + + public interface ArteryHeartbeatRspOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required uint64 uid = 1; + /** + * required uint64 uid = 1; + */ + boolean hasUid(); + /** + * required uint64 uid = 1; + */ + long getUid(); + } + /** + * Protobuf type {@code ArteryHeartbeatRsp} + * + *
+   * RemoteWatcher.ArteryHeartbeat is empty array
+   * RemoteWatcher.ArteryHeartbeatRsp
+   * 
+ */ + public static final class ArteryHeartbeatRsp extends + akka.protobuf.GeneratedMessage + implements ArteryHeartbeatRspOrBuilder { + // Use ArteryHeartbeatRsp.newBuilder() to construct. + private ArteryHeartbeatRsp(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private ArteryHeartbeatRsp(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final ArteryHeartbeatRsp defaultInstance; + public static ArteryHeartbeatRsp getDefaultInstance() { + return defaultInstance; + } + + public ArteryHeartbeatRsp getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private ArteryHeartbeatRsp( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + uid_ = input.readUInt64(); + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_ArteryHeartbeatRsp_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_ArteryHeartbeatRsp_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.ArteryHeartbeatRsp.class, akka.remote.ArteryControlFormats.ArteryHeartbeatRsp.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public ArteryHeartbeatRsp parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new ArteryHeartbeatRsp(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required uint64 uid = 1; + public static final int UID_FIELD_NUMBER = 1; + private long uid_; + /** + * required uint64 uid = 1; + */ + public boolean hasUid() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required uint64 uid = 1; + */ + public long getUid() { + return uid_; + } + + private void initFields() { + uid_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasUid()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeUInt64(1, uid_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeUInt64Size(1, uid_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.ArteryControlFormats.ArteryHeartbeatRsp parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.ArteryHeartbeatRsp parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.ArteryHeartbeatRsp parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ArteryControlFormats.ArteryHeartbeatRsp parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.ArteryHeartbeatRsp parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.ArteryHeartbeatRsp parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.ArteryHeartbeatRsp parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.ArteryControlFormats.ArteryHeartbeatRsp parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.ArteryControlFormats.ArteryHeartbeatRsp parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ArteryControlFormats.ArteryHeartbeatRsp parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.ArteryControlFormats.ArteryHeartbeatRsp prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code ArteryHeartbeatRsp} + * + *
+     * RemoteWatcher.ArteryHeartbeat is empty array
+     * RemoteWatcher.ArteryHeartbeatRsp
+     * 
+ */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.ArteryControlFormats.ArteryHeartbeatRspOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ArteryControlFormats.internal_static_ArteryHeartbeatRsp_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ArteryControlFormats.internal_static_ArteryHeartbeatRsp_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ArteryControlFormats.ArteryHeartbeatRsp.class, akka.remote.ArteryControlFormats.ArteryHeartbeatRsp.Builder.class); + } + + // Construct using akka.remote.ArteryControlFormats.ArteryHeartbeatRsp.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + uid_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.ArteryControlFormats.internal_static_ArteryHeartbeatRsp_descriptor; + } + + public akka.remote.ArteryControlFormats.ArteryHeartbeatRsp getDefaultInstanceForType() { + return akka.remote.ArteryControlFormats.ArteryHeartbeatRsp.getDefaultInstance(); + } + + public akka.remote.ArteryControlFormats.ArteryHeartbeatRsp build() { + akka.remote.ArteryControlFormats.ArteryHeartbeatRsp result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.ArteryControlFormats.ArteryHeartbeatRsp buildPartial() { + akka.remote.ArteryControlFormats.ArteryHeartbeatRsp result = new akka.remote.ArteryControlFormats.ArteryHeartbeatRsp(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.uid_ = uid_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.ArteryControlFormats.ArteryHeartbeatRsp) { + return mergeFrom((akka.remote.ArteryControlFormats.ArteryHeartbeatRsp)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.ArteryControlFormats.ArteryHeartbeatRsp other) { + if (other == akka.remote.ArteryControlFormats.ArteryHeartbeatRsp.getDefaultInstance()) return this; + if (other.hasUid()) { + setUid(other.getUid()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasUid()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.ArteryControlFormats.ArteryHeartbeatRsp parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.ArteryControlFormats.ArteryHeartbeatRsp) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required uint64 uid = 1; + private long uid_ ; + /** + * required uint64 uid = 1; + */ + public boolean hasUid() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required uint64 uid = 1; + */ + public long getUid() { + return uid_; + } + /** + * required uint64 uid = 1; + */ + public Builder setUid(long value) { + bitField0_ |= 0x00000001; + uid_ = value; + onChanged(); + return this; + } + /** + * required uint64 uid = 1; + */ + public Builder clearUid() { + bitField0_ = (bitField0_ & ~0x00000001); + uid_ = 0L; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:ArteryHeartbeatRsp) + } + + static { + defaultInstance = new ArteryHeartbeatRsp(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:ArteryHeartbeatRsp) + } + + private static akka.protobuf.Descriptors.Descriptor + internal_static_Quarantined_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_Quarantined_fieldAccessorTable; + private static akka.protobuf.Descriptors.Descriptor + internal_static_MessageWithAddress_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_MessageWithAddress_fieldAccessorTable; + private static akka.protobuf.Descriptors.Descriptor + internal_static_HandshakeReq_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_HandshakeReq_fieldAccessorTable; + private static akka.protobuf.Descriptors.Descriptor + internal_static_CompressionTableAdvertisement_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_CompressionTableAdvertisement_fieldAccessorTable; + private static akka.protobuf.Descriptors.Descriptor + internal_static_CompressionTableAdvertisementAck_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_CompressionTableAdvertisementAck_fieldAccessorTable; + private static akka.protobuf.Descriptors.Descriptor + internal_static_SystemMessageEnvelope_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_SystemMessageEnvelope_fieldAccessorTable; + private static akka.protobuf.Descriptors.Descriptor + internal_static_SystemMessageDeliveryAck_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_SystemMessageDeliveryAck_fieldAccessorTable; + private static akka.protobuf.Descriptors.Descriptor + internal_static_Address_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_Address_fieldAccessorTable; + private static akka.protobuf.Descriptors.Descriptor + internal_static_UniqueAddress_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_UniqueAddress_fieldAccessorTable; + private static akka.protobuf.Descriptors.Descriptor + internal_static_ArteryHeartbeatRsp_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_ArteryHeartbeatRsp_fieldAccessorTable; + + public static akka.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static akka.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\032ArteryControlFormats.proto\"G\n\013Quaranti" + + "ned\022\034\n\004from\030\001 \002(\0132\016.UniqueAddress\022\032\n\002to\030" + + "\002 \002(\0132\016.UniqueAddress\"5\n\022MessageWithAddr" + + "ess\022\037\n\007address\030\001 \002(\0132\016.UniqueAddress\"B\n\014" + + "HandshakeReq\022\034\n\004from\030\001 \002(\0132\016.UniqueAddre" + + "ss\022\024\n\002to\030\002 \002(\0132\010.Address\"\204\001\n\035Compression" + + "TableAdvertisement\022\034\n\004from\030\001 \002(\0132\016.Uniqu" + + "eAddress\022\021\n\toriginUid\030\002 \002(\004\022\024\n\014tableVers" + + "ion\030\003 \002(\r\022\014\n\004keys\030\004 \003(\t\022\016\n\006values\030\005 \003(\r\"" + + "Q\n CompressionTableAdvertisementAck\022\034\n\004f", + "rom\030\001 \002(\0132\016.UniqueAddress\022\017\n\007version\030\002 \002" + + "(\r\"\212\001\n\025SystemMessageEnvelope\022\017\n\007message\030" + + "\001 \002(\014\022\024\n\014serializerId\030\002 \002(\005\022\027\n\017messageMa" + + "nifest\030\003 \001(\014\022\r\n\005seqNo\030\004 \002(\004\022\"\n\nackReplyT" + + "o\030\005 \002(\0132\016.UniqueAddress\"G\n\030SystemMessage" + + "DeliveryAck\022\r\n\005seqNo\030\001 \002(\004\022\034\n\004from\030\002 \002(\013" + + "2\016.UniqueAddress\"K\n\007Address\022\020\n\010protocol\030" + + "\001 \002(\t\022\016\n\006system\030\002 \002(\t\022\020\n\010hostname\030\003 \002(\t\022" + + "\014\n\004port\030\004 \002(\r\"7\n\rUniqueAddress\022\031\n\007addres" + + "s\030\001 \002(\0132\010.Address\022\013\n\003uid\030\002 \002(\004\"!\n\022Artery", + "HeartbeatRsp\022\013\n\003uid\030\001 \002(\004B\017\n\013akka.remote" + + "H\001" + }; + akka.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = + new akka.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { + public akka.protobuf.ExtensionRegistry assignDescriptors( + akka.protobuf.Descriptors.FileDescriptor root) { + descriptor = root; + internal_static_Quarantined_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_Quarantined_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_Quarantined_descriptor, + new java.lang.String[] { "From", "To", }); + internal_static_MessageWithAddress_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_MessageWithAddress_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_MessageWithAddress_descriptor, + new java.lang.String[] { "Address", }); + internal_static_HandshakeReq_descriptor = + getDescriptor().getMessageTypes().get(2); + internal_static_HandshakeReq_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_HandshakeReq_descriptor, + new java.lang.String[] { "From", "To", }); + internal_static_CompressionTableAdvertisement_descriptor = + getDescriptor().getMessageTypes().get(3); + internal_static_CompressionTableAdvertisement_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_CompressionTableAdvertisement_descriptor, + new java.lang.String[] { "From", "OriginUid", "TableVersion", "Keys", "Values", }); + internal_static_CompressionTableAdvertisementAck_descriptor = + getDescriptor().getMessageTypes().get(4); + internal_static_CompressionTableAdvertisementAck_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_CompressionTableAdvertisementAck_descriptor, + new java.lang.String[] { "From", "Version", }); + internal_static_SystemMessageEnvelope_descriptor = + getDescriptor().getMessageTypes().get(5); + internal_static_SystemMessageEnvelope_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_SystemMessageEnvelope_descriptor, + new java.lang.String[] { "Message", "SerializerId", "MessageManifest", "SeqNo", "AckReplyTo", }); + internal_static_SystemMessageDeliveryAck_descriptor = + getDescriptor().getMessageTypes().get(6); + internal_static_SystemMessageDeliveryAck_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_SystemMessageDeliveryAck_descriptor, + new java.lang.String[] { "SeqNo", "From", }); + internal_static_Address_descriptor = + getDescriptor().getMessageTypes().get(7); + internal_static_Address_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_Address_descriptor, + new java.lang.String[] { "Protocol", "System", "Hostname", "Port", }); + internal_static_UniqueAddress_descriptor = + getDescriptor().getMessageTypes().get(8); + internal_static_UniqueAddress_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_UniqueAddress_descriptor, + new java.lang.String[] { "Address", "Uid", }); + internal_static_ArteryHeartbeatRsp_descriptor = + getDescriptor().getMessageTypes().get(9); + internal_static_ArteryHeartbeatRsp_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_ArteryHeartbeatRsp_descriptor, + new java.lang.String[] { "Uid", }); + return null; + } + }; + akka.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new akka.protobuf.Descriptors.FileDescriptor[] { + }, assigner); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/akka-remote/src/main/java/akka/remote/ContainerFormats.java b/akka-remote/src/main/java/akka/remote/ContainerFormats.java index 1e63dd0f4b..c22fd48253 100644 --- a/akka-remote/src/main/java/akka/remote/ContainerFormats.java +++ b/akka-remote/src/main/java/akka/remote/ContainerFormats.java @@ -4563,6 +4563,3410 @@ public final class ContainerFormats { // @@protoc_insertion_point(class_scope:Payload) } + public interface WatcherHeartbeatResponseOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required uint64 uid = 1; + /** + * required uint64 uid = 1; + */ + boolean hasUid(); + /** + * required uint64 uid = 1; + */ + long getUid(); + } + /** + * Protobuf type {@code WatcherHeartbeatResponse} + */ + public static final class WatcherHeartbeatResponse extends + akka.protobuf.GeneratedMessage + implements WatcherHeartbeatResponseOrBuilder { + // Use WatcherHeartbeatResponse.newBuilder() to construct. + private WatcherHeartbeatResponse(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private WatcherHeartbeatResponse(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final WatcherHeartbeatResponse defaultInstance; + public static WatcherHeartbeatResponse getDefaultInstance() { + return defaultInstance; + } + + public WatcherHeartbeatResponse getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private WatcherHeartbeatResponse( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + uid_ = input.readUInt64(); + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ContainerFormats.internal_static_WatcherHeartbeatResponse_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ContainerFormats.internal_static_WatcherHeartbeatResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ContainerFormats.WatcherHeartbeatResponse.class, akka.remote.ContainerFormats.WatcherHeartbeatResponse.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public WatcherHeartbeatResponse parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new WatcherHeartbeatResponse(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required uint64 uid = 1; + public static final int UID_FIELD_NUMBER = 1; + private long uid_; + /** + * required uint64 uid = 1; + */ + public boolean hasUid() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required uint64 uid = 1; + */ + public long getUid() { + return uid_; + } + + private void initFields() { + uid_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasUid()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeUInt64(1, uid_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeUInt64Size(1, uid_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.ContainerFormats.WatcherHeartbeatResponse parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ContainerFormats.WatcherHeartbeatResponse parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ContainerFormats.WatcherHeartbeatResponse parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ContainerFormats.WatcherHeartbeatResponse parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ContainerFormats.WatcherHeartbeatResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ContainerFormats.WatcherHeartbeatResponse parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.ContainerFormats.WatcherHeartbeatResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.ContainerFormats.WatcherHeartbeatResponse parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.ContainerFormats.WatcherHeartbeatResponse parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ContainerFormats.WatcherHeartbeatResponse parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.ContainerFormats.WatcherHeartbeatResponse prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code WatcherHeartbeatResponse} + */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.ContainerFormats.WatcherHeartbeatResponseOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ContainerFormats.internal_static_WatcherHeartbeatResponse_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ContainerFormats.internal_static_WatcherHeartbeatResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ContainerFormats.WatcherHeartbeatResponse.class, akka.remote.ContainerFormats.WatcherHeartbeatResponse.Builder.class); + } + + // Construct using akka.remote.ContainerFormats.WatcherHeartbeatResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + uid_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.ContainerFormats.internal_static_WatcherHeartbeatResponse_descriptor; + } + + public akka.remote.ContainerFormats.WatcherHeartbeatResponse getDefaultInstanceForType() { + return akka.remote.ContainerFormats.WatcherHeartbeatResponse.getDefaultInstance(); + } + + public akka.remote.ContainerFormats.WatcherHeartbeatResponse build() { + akka.remote.ContainerFormats.WatcherHeartbeatResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.ContainerFormats.WatcherHeartbeatResponse buildPartial() { + akka.remote.ContainerFormats.WatcherHeartbeatResponse result = new akka.remote.ContainerFormats.WatcherHeartbeatResponse(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.uid_ = uid_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.ContainerFormats.WatcherHeartbeatResponse) { + return mergeFrom((akka.remote.ContainerFormats.WatcherHeartbeatResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.ContainerFormats.WatcherHeartbeatResponse other) { + if (other == akka.remote.ContainerFormats.WatcherHeartbeatResponse.getDefaultInstance()) return this; + if (other.hasUid()) { + setUid(other.getUid()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasUid()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.ContainerFormats.WatcherHeartbeatResponse parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.ContainerFormats.WatcherHeartbeatResponse) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required uint64 uid = 1; + private long uid_ ; + /** + * required uint64 uid = 1; + */ + public boolean hasUid() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required uint64 uid = 1; + */ + public long getUid() { + return uid_; + } + /** + * required uint64 uid = 1; + */ + public Builder setUid(long value) { + bitField0_ |= 0x00000001; + uid_ = value; + onChanged(); + return this; + } + /** + * required uint64 uid = 1; + */ + public Builder clearUid() { + bitField0_ = (bitField0_ & ~0x00000001); + uid_ = 0L; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:WatcherHeartbeatResponse) + } + + static { + defaultInstance = new WatcherHeartbeatResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:WatcherHeartbeatResponse) + } + + public interface ThrowableOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required string className = 1; + /** + * required string className = 1; + */ + boolean hasClassName(); + /** + * required string className = 1; + */ + java.lang.String getClassName(); + /** + * required string className = 1; + */ + akka.protobuf.ByteString + getClassNameBytes(); + + // optional string message = 2; + /** + * optional string message = 2; + */ + boolean hasMessage(); + /** + * optional string message = 2; + */ + java.lang.String getMessage(); + /** + * optional string message = 2; + */ + akka.protobuf.ByteString + getMessageBytes(); + + // optional .Payload cause = 3; + /** + * optional .Payload cause = 3; + */ + boolean hasCause(); + /** + * optional .Payload cause = 3; + */ + akka.remote.ContainerFormats.Payload getCause(); + /** + * optional .Payload cause = 3; + */ + akka.remote.ContainerFormats.PayloadOrBuilder getCauseOrBuilder(); + + // repeated .StackTraceElement stackTrace = 4; + /** + * repeated .StackTraceElement stackTrace = 4; + */ + java.util.List + getStackTraceList(); + /** + * repeated .StackTraceElement stackTrace = 4; + */ + akka.remote.ContainerFormats.StackTraceElement getStackTrace(int index); + /** + * repeated .StackTraceElement stackTrace = 4; + */ + int getStackTraceCount(); + /** + * repeated .StackTraceElement stackTrace = 4; + */ + java.util.List + getStackTraceOrBuilderList(); + /** + * repeated .StackTraceElement stackTrace = 4; + */ + akka.remote.ContainerFormats.StackTraceElementOrBuilder getStackTraceOrBuilder( + int index); + } + /** + * Protobuf type {@code Throwable} + */ + public static final class Throwable extends + akka.protobuf.GeneratedMessage + implements ThrowableOrBuilder { + // Use Throwable.newBuilder() to construct. + private Throwable(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private Throwable(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final Throwable defaultInstance; + public static Throwable getDefaultInstance() { + return defaultInstance; + } + + public Throwable getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private Throwable( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + className_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + message_ = input.readBytes(); + break; + } + case 26: { + akka.remote.ContainerFormats.Payload.Builder subBuilder = null; + if (((bitField0_ & 0x00000004) == 0x00000004)) { + subBuilder = cause_.toBuilder(); + } + cause_ = input.readMessage(akka.remote.ContainerFormats.Payload.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(cause_); + cause_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000004; + break; + } + case 34: { + if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) { + stackTrace_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000008; + } + stackTrace_.add(input.readMessage(akka.remote.ContainerFormats.StackTraceElement.PARSER, extensionRegistry)); + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) { + stackTrace_ = java.util.Collections.unmodifiableList(stackTrace_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ContainerFormats.internal_static_Throwable_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ContainerFormats.internal_static_Throwable_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ContainerFormats.Throwable.class, akka.remote.ContainerFormats.Throwable.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public Throwable parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new Throwable(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required string className = 1; + public static final int CLASSNAME_FIELD_NUMBER = 1; + private java.lang.Object className_; + /** + * required string className = 1; + */ + public boolean hasClassName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required string className = 1; + */ + public java.lang.String getClassName() { + java.lang.Object ref = className_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + akka.protobuf.ByteString bs = + (akka.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + className_ = s; + } + return s; + } + } + /** + * required string className = 1; + */ + public akka.protobuf.ByteString + getClassNameBytes() { + java.lang.Object ref = className_; + if (ref instanceof java.lang.String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + className_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + + // optional string message = 2; + public static final int MESSAGE_FIELD_NUMBER = 2; + private java.lang.Object message_; + /** + * optional string message = 2; + */ + public boolean hasMessage() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional string message = 2; + */ + public java.lang.String getMessage() { + java.lang.Object ref = message_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + akka.protobuf.ByteString bs = + (akka.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + message_ = s; + } + return s; + } + } + /** + * optional string message = 2; + */ + public akka.protobuf.ByteString + getMessageBytes() { + java.lang.Object ref = message_; + if (ref instanceof java.lang.String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + message_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + + // optional .Payload cause = 3; + public static final int CAUSE_FIELD_NUMBER = 3; + private akka.remote.ContainerFormats.Payload cause_; + /** + * optional .Payload cause = 3; + */ + public boolean hasCause() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional .Payload cause = 3; + */ + public akka.remote.ContainerFormats.Payload getCause() { + return cause_; + } + /** + * optional .Payload cause = 3; + */ + public akka.remote.ContainerFormats.PayloadOrBuilder getCauseOrBuilder() { + return cause_; + } + + // repeated .StackTraceElement stackTrace = 4; + public static final int STACKTRACE_FIELD_NUMBER = 4; + private java.util.List stackTrace_; + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public java.util.List getStackTraceList() { + return stackTrace_; + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public java.util.List + getStackTraceOrBuilderList() { + return stackTrace_; + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public int getStackTraceCount() { + return stackTrace_.size(); + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public akka.remote.ContainerFormats.StackTraceElement getStackTrace(int index) { + return stackTrace_.get(index); + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public akka.remote.ContainerFormats.StackTraceElementOrBuilder getStackTraceOrBuilder( + int index) { + return stackTrace_.get(index); + } + + private void initFields() { + className_ = ""; + message_ = ""; + cause_ = akka.remote.ContainerFormats.Payload.getDefaultInstance(); + stackTrace_ = java.util.Collections.emptyList(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasClassName()) { + memoizedIsInitialized = 0; + return false; + } + if (hasCause()) { + if (!getCause().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + for (int i = 0; i < getStackTraceCount(); i++) { + if (!getStackTrace(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getClassNameBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getMessageBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeMessage(3, cause_); + } + for (int i = 0; i < stackTrace_.size(); i++) { + output.writeMessage(4, stackTrace_.get(i)); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeBytesSize(1, getClassNameBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += akka.protobuf.CodedOutputStream + .computeBytesSize(2, getMessageBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(3, cause_); + } + for (int i = 0; i < stackTrace_.size(); i++) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(4, stackTrace_.get(i)); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.ContainerFormats.Throwable parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ContainerFormats.Throwable parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ContainerFormats.Throwable parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ContainerFormats.Throwable parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ContainerFormats.Throwable parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ContainerFormats.Throwable parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.ContainerFormats.Throwable parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.ContainerFormats.Throwable parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.ContainerFormats.Throwable parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ContainerFormats.Throwable parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.ContainerFormats.Throwable prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code Throwable} + */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.ContainerFormats.ThrowableOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ContainerFormats.internal_static_Throwable_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ContainerFormats.internal_static_Throwable_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ContainerFormats.Throwable.class, akka.remote.ContainerFormats.Throwable.Builder.class); + } + + // Construct using akka.remote.ContainerFormats.Throwable.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getCauseFieldBuilder(); + getStackTraceFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + className_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + message_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + if (causeBuilder_ == null) { + cause_ = akka.remote.ContainerFormats.Payload.getDefaultInstance(); + } else { + causeBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + if (stackTraceBuilder_ == null) { + stackTrace_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000008); + } else { + stackTraceBuilder_.clear(); + } + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.ContainerFormats.internal_static_Throwable_descriptor; + } + + public akka.remote.ContainerFormats.Throwable getDefaultInstanceForType() { + return akka.remote.ContainerFormats.Throwable.getDefaultInstance(); + } + + public akka.remote.ContainerFormats.Throwable build() { + akka.remote.ContainerFormats.Throwable result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.ContainerFormats.Throwable buildPartial() { + akka.remote.ContainerFormats.Throwable result = new akka.remote.ContainerFormats.Throwable(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.className_ = className_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.message_ = message_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + if (causeBuilder_ == null) { + result.cause_ = cause_; + } else { + result.cause_ = causeBuilder_.build(); + } + if (stackTraceBuilder_ == null) { + if (((bitField0_ & 0x00000008) == 0x00000008)) { + stackTrace_ = java.util.Collections.unmodifiableList(stackTrace_); + bitField0_ = (bitField0_ & ~0x00000008); + } + result.stackTrace_ = stackTrace_; + } else { + result.stackTrace_ = stackTraceBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.ContainerFormats.Throwable) { + return mergeFrom((akka.remote.ContainerFormats.Throwable)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.ContainerFormats.Throwable other) { + if (other == akka.remote.ContainerFormats.Throwable.getDefaultInstance()) return this; + if (other.hasClassName()) { + bitField0_ |= 0x00000001; + className_ = other.className_; + onChanged(); + } + if (other.hasMessage()) { + bitField0_ |= 0x00000002; + message_ = other.message_; + onChanged(); + } + if (other.hasCause()) { + mergeCause(other.getCause()); + } + if (stackTraceBuilder_ == null) { + if (!other.stackTrace_.isEmpty()) { + if (stackTrace_.isEmpty()) { + stackTrace_ = other.stackTrace_; + bitField0_ = (bitField0_ & ~0x00000008); + } else { + ensureStackTraceIsMutable(); + stackTrace_.addAll(other.stackTrace_); + } + onChanged(); + } + } else { + if (!other.stackTrace_.isEmpty()) { + if (stackTraceBuilder_.isEmpty()) { + stackTraceBuilder_.dispose(); + stackTraceBuilder_ = null; + stackTrace_ = other.stackTrace_; + bitField0_ = (bitField0_ & ~0x00000008); + stackTraceBuilder_ = + akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getStackTraceFieldBuilder() : null; + } else { + stackTraceBuilder_.addAllMessages(other.stackTrace_); + } + } + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasClassName()) { + + return false; + } + if (hasCause()) { + if (!getCause().isInitialized()) { + + return false; + } + } + for (int i = 0; i < getStackTraceCount(); i++) { + if (!getStackTrace(i).isInitialized()) { + + return false; + } + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.ContainerFormats.Throwable parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.ContainerFormats.Throwable) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required string className = 1; + private java.lang.Object className_ = ""; + /** + * required string className = 1; + */ + public boolean hasClassName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required string className = 1; + */ + public java.lang.String getClassName() { + java.lang.Object ref = className_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((akka.protobuf.ByteString) ref) + .toStringUtf8(); + className_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string className = 1; + */ + public akka.protobuf.ByteString + getClassNameBytes() { + java.lang.Object ref = className_; + if (ref instanceof String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + className_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + /** + * required string className = 1; + */ + public Builder setClassName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + className_ = value; + onChanged(); + return this; + } + /** + * required string className = 1; + */ + public Builder clearClassName() { + bitField0_ = (bitField0_ & ~0x00000001); + className_ = getDefaultInstance().getClassName(); + onChanged(); + return this; + } + /** + * required string className = 1; + */ + public Builder setClassNameBytes( + akka.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + className_ = value; + onChanged(); + return this; + } + + // optional string message = 2; + private java.lang.Object message_ = ""; + /** + * optional string message = 2; + */ + public boolean hasMessage() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional string message = 2; + */ + public java.lang.String getMessage() { + java.lang.Object ref = message_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((akka.protobuf.ByteString) ref) + .toStringUtf8(); + message_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * optional string message = 2; + */ + public akka.protobuf.ByteString + getMessageBytes() { + java.lang.Object ref = message_; + if (ref instanceof String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + message_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + /** + * optional string message = 2; + */ + public Builder setMessage( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + message_ = value; + onChanged(); + return this; + } + /** + * optional string message = 2; + */ + public Builder clearMessage() { + bitField0_ = (bitField0_ & ~0x00000002); + message_ = getDefaultInstance().getMessage(); + onChanged(); + return this; + } + /** + * optional string message = 2; + */ + public Builder setMessageBytes( + akka.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + message_ = value; + onChanged(); + return this; + } + + // optional .Payload cause = 3; + private akka.remote.ContainerFormats.Payload cause_ = akka.remote.ContainerFormats.Payload.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.Payload, akka.remote.ContainerFormats.Payload.Builder, akka.remote.ContainerFormats.PayloadOrBuilder> causeBuilder_; + /** + * optional .Payload cause = 3; + */ + public boolean hasCause() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional .Payload cause = 3; + */ + public akka.remote.ContainerFormats.Payload getCause() { + if (causeBuilder_ == null) { + return cause_; + } else { + return causeBuilder_.getMessage(); + } + } + /** + * optional .Payload cause = 3; + */ + public Builder setCause(akka.remote.ContainerFormats.Payload value) { + if (causeBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + cause_ = value; + onChanged(); + } else { + causeBuilder_.setMessage(value); + } + bitField0_ |= 0x00000004; + return this; + } + /** + * optional .Payload cause = 3; + */ + public Builder setCause( + akka.remote.ContainerFormats.Payload.Builder builderForValue) { + if (causeBuilder_ == null) { + cause_ = builderForValue.build(); + onChanged(); + } else { + causeBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000004; + return this; + } + /** + * optional .Payload cause = 3; + */ + public Builder mergeCause(akka.remote.ContainerFormats.Payload value) { + if (causeBuilder_ == null) { + if (((bitField0_ & 0x00000004) == 0x00000004) && + cause_ != akka.remote.ContainerFormats.Payload.getDefaultInstance()) { + cause_ = + akka.remote.ContainerFormats.Payload.newBuilder(cause_).mergeFrom(value).buildPartial(); + } else { + cause_ = value; + } + onChanged(); + } else { + causeBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000004; + return this; + } + /** + * optional .Payload cause = 3; + */ + public Builder clearCause() { + if (causeBuilder_ == null) { + cause_ = akka.remote.ContainerFormats.Payload.getDefaultInstance(); + onChanged(); + } else { + causeBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + /** + * optional .Payload cause = 3; + */ + public akka.remote.ContainerFormats.Payload.Builder getCauseBuilder() { + bitField0_ |= 0x00000004; + onChanged(); + return getCauseFieldBuilder().getBuilder(); + } + /** + * optional .Payload cause = 3; + */ + public akka.remote.ContainerFormats.PayloadOrBuilder getCauseOrBuilder() { + if (causeBuilder_ != null) { + return causeBuilder_.getMessageOrBuilder(); + } else { + return cause_; + } + } + /** + * optional .Payload cause = 3; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.Payload, akka.remote.ContainerFormats.Payload.Builder, akka.remote.ContainerFormats.PayloadOrBuilder> + getCauseFieldBuilder() { + if (causeBuilder_ == null) { + causeBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.Payload, akka.remote.ContainerFormats.Payload.Builder, akka.remote.ContainerFormats.PayloadOrBuilder>( + cause_, + getParentForChildren(), + isClean()); + cause_ = null; + } + return causeBuilder_; + } + + // repeated .StackTraceElement stackTrace = 4; + private java.util.List stackTrace_ = + java.util.Collections.emptyList(); + private void ensureStackTraceIsMutable() { + if (!((bitField0_ & 0x00000008) == 0x00000008)) { + stackTrace_ = new java.util.ArrayList(stackTrace_); + bitField0_ |= 0x00000008; + } + } + + private akka.protobuf.RepeatedFieldBuilder< + akka.remote.ContainerFormats.StackTraceElement, akka.remote.ContainerFormats.StackTraceElement.Builder, akka.remote.ContainerFormats.StackTraceElementOrBuilder> stackTraceBuilder_; + + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public java.util.List getStackTraceList() { + if (stackTraceBuilder_ == null) { + return java.util.Collections.unmodifiableList(stackTrace_); + } else { + return stackTraceBuilder_.getMessageList(); + } + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public int getStackTraceCount() { + if (stackTraceBuilder_ == null) { + return stackTrace_.size(); + } else { + return stackTraceBuilder_.getCount(); + } + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public akka.remote.ContainerFormats.StackTraceElement getStackTrace(int index) { + if (stackTraceBuilder_ == null) { + return stackTrace_.get(index); + } else { + return stackTraceBuilder_.getMessage(index); + } + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public Builder setStackTrace( + int index, akka.remote.ContainerFormats.StackTraceElement value) { + if (stackTraceBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureStackTraceIsMutable(); + stackTrace_.set(index, value); + onChanged(); + } else { + stackTraceBuilder_.setMessage(index, value); + } + return this; + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public Builder setStackTrace( + int index, akka.remote.ContainerFormats.StackTraceElement.Builder builderForValue) { + if (stackTraceBuilder_ == null) { + ensureStackTraceIsMutable(); + stackTrace_.set(index, builderForValue.build()); + onChanged(); + } else { + stackTraceBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public Builder addStackTrace(akka.remote.ContainerFormats.StackTraceElement value) { + if (stackTraceBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureStackTraceIsMutable(); + stackTrace_.add(value); + onChanged(); + } else { + stackTraceBuilder_.addMessage(value); + } + return this; + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public Builder addStackTrace( + int index, akka.remote.ContainerFormats.StackTraceElement value) { + if (stackTraceBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureStackTraceIsMutable(); + stackTrace_.add(index, value); + onChanged(); + } else { + stackTraceBuilder_.addMessage(index, value); + } + return this; + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public Builder addStackTrace( + akka.remote.ContainerFormats.StackTraceElement.Builder builderForValue) { + if (stackTraceBuilder_ == null) { + ensureStackTraceIsMutable(); + stackTrace_.add(builderForValue.build()); + onChanged(); + } else { + stackTraceBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public Builder addStackTrace( + int index, akka.remote.ContainerFormats.StackTraceElement.Builder builderForValue) { + if (stackTraceBuilder_ == null) { + ensureStackTraceIsMutable(); + stackTrace_.add(index, builderForValue.build()); + onChanged(); + } else { + stackTraceBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public Builder addAllStackTrace( + java.lang.Iterable values) { + if (stackTraceBuilder_ == null) { + ensureStackTraceIsMutable(); + super.addAll(values, stackTrace_); + onChanged(); + } else { + stackTraceBuilder_.addAllMessages(values); + } + return this; + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public Builder clearStackTrace() { + if (stackTraceBuilder_ == null) { + stackTrace_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000008); + onChanged(); + } else { + stackTraceBuilder_.clear(); + } + return this; + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public Builder removeStackTrace(int index) { + if (stackTraceBuilder_ == null) { + ensureStackTraceIsMutable(); + stackTrace_.remove(index); + onChanged(); + } else { + stackTraceBuilder_.remove(index); + } + return this; + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public akka.remote.ContainerFormats.StackTraceElement.Builder getStackTraceBuilder( + int index) { + return getStackTraceFieldBuilder().getBuilder(index); + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public akka.remote.ContainerFormats.StackTraceElementOrBuilder getStackTraceOrBuilder( + int index) { + if (stackTraceBuilder_ == null) { + return stackTrace_.get(index); } else { + return stackTraceBuilder_.getMessageOrBuilder(index); + } + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public java.util.List + getStackTraceOrBuilderList() { + if (stackTraceBuilder_ != null) { + return stackTraceBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(stackTrace_); + } + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public akka.remote.ContainerFormats.StackTraceElement.Builder addStackTraceBuilder() { + return getStackTraceFieldBuilder().addBuilder( + akka.remote.ContainerFormats.StackTraceElement.getDefaultInstance()); + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public akka.remote.ContainerFormats.StackTraceElement.Builder addStackTraceBuilder( + int index) { + return getStackTraceFieldBuilder().addBuilder( + index, akka.remote.ContainerFormats.StackTraceElement.getDefaultInstance()); + } + /** + * repeated .StackTraceElement stackTrace = 4; + */ + public java.util.List + getStackTraceBuilderList() { + return getStackTraceFieldBuilder().getBuilderList(); + } + private akka.protobuf.RepeatedFieldBuilder< + akka.remote.ContainerFormats.StackTraceElement, akka.remote.ContainerFormats.StackTraceElement.Builder, akka.remote.ContainerFormats.StackTraceElementOrBuilder> + getStackTraceFieldBuilder() { + if (stackTraceBuilder_ == null) { + stackTraceBuilder_ = new akka.protobuf.RepeatedFieldBuilder< + akka.remote.ContainerFormats.StackTraceElement, akka.remote.ContainerFormats.StackTraceElement.Builder, akka.remote.ContainerFormats.StackTraceElementOrBuilder>( + stackTrace_, + ((bitField0_ & 0x00000008) == 0x00000008), + getParentForChildren(), + isClean()); + stackTrace_ = null; + } + return stackTraceBuilder_; + } + + // @@protoc_insertion_point(builder_scope:Throwable) + } + + static { + defaultInstance = new Throwable(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:Throwable) + } + + public interface ActorInitializationExceptionOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // optional .ActorRef actor = 1; + /** + * optional .ActorRef actor = 1; + */ + boolean hasActor(); + /** + * optional .ActorRef actor = 1; + */ + akka.remote.ContainerFormats.ActorRef getActor(); + /** + * optional .ActorRef actor = 1; + */ + akka.remote.ContainerFormats.ActorRefOrBuilder getActorOrBuilder(); + + // required string message = 2; + /** + * required string message = 2; + */ + boolean hasMessage(); + /** + * required string message = 2; + */ + java.lang.String getMessage(); + /** + * required string message = 2; + */ + akka.protobuf.ByteString + getMessageBytes(); + + // required .Payload cause = 3; + /** + * required .Payload cause = 3; + */ + boolean hasCause(); + /** + * required .Payload cause = 3; + */ + akka.remote.ContainerFormats.Payload getCause(); + /** + * required .Payload cause = 3; + */ + akka.remote.ContainerFormats.PayloadOrBuilder getCauseOrBuilder(); + } + /** + * Protobuf type {@code ActorInitializationException} + */ + public static final class ActorInitializationException extends + akka.protobuf.GeneratedMessage + implements ActorInitializationExceptionOrBuilder { + // Use ActorInitializationException.newBuilder() to construct. + private ActorInitializationException(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private ActorInitializationException(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final ActorInitializationException defaultInstance; + public static ActorInitializationException getDefaultInstance() { + return defaultInstance; + } + + public ActorInitializationException getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private ActorInitializationException( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + akka.remote.ContainerFormats.ActorRef.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = actor_.toBuilder(); + } + actor_ = input.readMessage(akka.remote.ContainerFormats.ActorRef.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(actor_); + actor_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + case 18: { + bitField0_ |= 0x00000002; + message_ = input.readBytes(); + break; + } + case 26: { + akka.remote.ContainerFormats.Payload.Builder subBuilder = null; + if (((bitField0_ & 0x00000004) == 0x00000004)) { + subBuilder = cause_.toBuilder(); + } + cause_ = input.readMessage(akka.remote.ContainerFormats.Payload.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(cause_); + cause_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000004; + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ContainerFormats.internal_static_ActorInitializationException_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ContainerFormats.internal_static_ActorInitializationException_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ContainerFormats.ActorInitializationException.class, akka.remote.ContainerFormats.ActorInitializationException.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public ActorInitializationException parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new ActorInitializationException(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // optional .ActorRef actor = 1; + public static final int ACTOR_FIELD_NUMBER = 1; + private akka.remote.ContainerFormats.ActorRef actor_; + /** + * optional .ActorRef actor = 1; + */ + public boolean hasActor() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional .ActorRef actor = 1; + */ + public akka.remote.ContainerFormats.ActorRef getActor() { + return actor_; + } + /** + * optional .ActorRef actor = 1; + */ + public akka.remote.ContainerFormats.ActorRefOrBuilder getActorOrBuilder() { + return actor_; + } + + // required string message = 2; + public static final int MESSAGE_FIELD_NUMBER = 2; + private java.lang.Object message_; + /** + * required string message = 2; + */ + public boolean hasMessage() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required string message = 2; + */ + public java.lang.String getMessage() { + java.lang.Object ref = message_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + akka.protobuf.ByteString bs = + (akka.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + message_ = s; + } + return s; + } + } + /** + * required string message = 2; + */ + public akka.protobuf.ByteString + getMessageBytes() { + java.lang.Object ref = message_; + if (ref instanceof java.lang.String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + message_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + + // required .Payload cause = 3; + public static final int CAUSE_FIELD_NUMBER = 3; + private akka.remote.ContainerFormats.Payload cause_; + /** + * required .Payload cause = 3; + */ + public boolean hasCause() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required .Payload cause = 3; + */ + public akka.remote.ContainerFormats.Payload getCause() { + return cause_; + } + /** + * required .Payload cause = 3; + */ + public akka.remote.ContainerFormats.PayloadOrBuilder getCauseOrBuilder() { + return cause_; + } + + private void initFields() { + actor_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + message_ = ""; + cause_ = akka.remote.ContainerFormats.Payload.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasMessage()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasCause()) { + memoizedIsInitialized = 0; + return false; + } + if (hasActor()) { + if (!getActor().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (!getCause().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, actor_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getMessageBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeMessage(3, cause_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(1, actor_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += akka.protobuf.CodedOutputStream + .computeBytesSize(2, getMessageBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(3, cause_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.ContainerFormats.ActorInitializationException parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ContainerFormats.ActorInitializationException parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ContainerFormats.ActorInitializationException parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ContainerFormats.ActorInitializationException parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ContainerFormats.ActorInitializationException parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ContainerFormats.ActorInitializationException parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.ContainerFormats.ActorInitializationException parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.ContainerFormats.ActorInitializationException parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.ContainerFormats.ActorInitializationException parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ContainerFormats.ActorInitializationException parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.ContainerFormats.ActorInitializationException prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code ActorInitializationException} + */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.ContainerFormats.ActorInitializationExceptionOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ContainerFormats.internal_static_ActorInitializationException_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ContainerFormats.internal_static_ActorInitializationException_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ContainerFormats.ActorInitializationException.class, akka.remote.ContainerFormats.ActorInitializationException.Builder.class); + } + + // Construct using akka.remote.ContainerFormats.ActorInitializationException.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getActorFieldBuilder(); + getCauseFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (actorBuilder_ == null) { + actor_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + } else { + actorBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + message_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + if (causeBuilder_ == null) { + cause_ = akka.remote.ContainerFormats.Payload.getDefaultInstance(); + } else { + causeBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.ContainerFormats.internal_static_ActorInitializationException_descriptor; + } + + public akka.remote.ContainerFormats.ActorInitializationException getDefaultInstanceForType() { + return akka.remote.ContainerFormats.ActorInitializationException.getDefaultInstance(); + } + + public akka.remote.ContainerFormats.ActorInitializationException build() { + akka.remote.ContainerFormats.ActorInitializationException result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.ContainerFormats.ActorInitializationException buildPartial() { + akka.remote.ContainerFormats.ActorInitializationException result = new akka.remote.ContainerFormats.ActorInitializationException(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (actorBuilder_ == null) { + result.actor_ = actor_; + } else { + result.actor_ = actorBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.message_ = message_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + if (causeBuilder_ == null) { + result.cause_ = cause_; + } else { + result.cause_ = causeBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.ContainerFormats.ActorInitializationException) { + return mergeFrom((akka.remote.ContainerFormats.ActorInitializationException)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.ContainerFormats.ActorInitializationException other) { + if (other == akka.remote.ContainerFormats.ActorInitializationException.getDefaultInstance()) return this; + if (other.hasActor()) { + mergeActor(other.getActor()); + } + if (other.hasMessage()) { + bitField0_ |= 0x00000002; + message_ = other.message_; + onChanged(); + } + if (other.hasCause()) { + mergeCause(other.getCause()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasMessage()) { + + return false; + } + if (!hasCause()) { + + return false; + } + if (hasActor()) { + if (!getActor().isInitialized()) { + + return false; + } + } + if (!getCause().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.ContainerFormats.ActorInitializationException parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.ContainerFormats.ActorInitializationException) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // optional .ActorRef actor = 1; + private akka.remote.ContainerFormats.ActorRef actor_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.ActorRef, akka.remote.ContainerFormats.ActorRef.Builder, akka.remote.ContainerFormats.ActorRefOrBuilder> actorBuilder_; + /** + * optional .ActorRef actor = 1; + */ + public boolean hasActor() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional .ActorRef actor = 1; + */ + public akka.remote.ContainerFormats.ActorRef getActor() { + if (actorBuilder_ == null) { + return actor_; + } else { + return actorBuilder_.getMessage(); + } + } + /** + * optional .ActorRef actor = 1; + */ + public Builder setActor(akka.remote.ContainerFormats.ActorRef value) { + if (actorBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + actor_ = value; + onChanged(); + } else { + actorBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .ActorRef actor = 1; + */ + public Builder setActor( + akka.remote.ContainerFormats.ActorRef.Builder builderForValue) { + if (actorBuilder_ == null) { + actor_ = builderForValue.build(); + onChanged(); + } else { + actorBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .ActorRef actor = 1; + */ + public Builder mergeActor(akka.remote.ContainerFormats.ActorRef value) { + if (actorBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + actor_ != akka.remote.ContainerFormats.ActorRef.getDefaultInstance()) { + actor_ = + akka.remote.ContainerFormats.ActorRef.newBuilder(actor_).mergeFrom(value).buildPartial(); + } else { + actor_ = value; + } + onChanged(); + } else { + actorBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .ActorRef actor = 1; + */ + public Builder clearActor() { + if (actorBuilder_ == null) { + actor_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + onChanged(); + } else { + actorBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * optional .ActorRef actor = 1; + */ + public akka.remote.ContainerFormats.ActorRef.Builder getActorBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getActorFieldBuilder().getBuilder(); + } + /** + * optional .ActorRef actor = 1; + */ + public akka.remote.ContainerFormats.ActorRefOrBuilder getActorOrBuilder() { + if (actorBuilder_ != null) { + return actorBuilder_.getMessageOrBuilder(); + } else { + return actor_; + } + } + /** + * optional .ActorRef actor = 1; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.ActorRef, akka.remote.ContainerFormats.ActorRef.Builder, akka.remote.ContainerFormats.ActorRefOrBuilder> + getActorFieldBuilder() { + if (actorBuilder_ == null) { + actorBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.ActorRef, akka.remote.ContainerFormats.ActorRef.Builder, akka.remote.ContainerFormats.ActorRefOrBuilder>( + actor_, + getParentForChildren(), + isClean()); + actor_ = null; + } + return actorBuilder_; + } + + // required string message = 2; + private java.lang.Object message_ = ""; + /** + * required string message = 2; + */ + public boolean hasMessage() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required string message = 2; + */ + public java.lang.String getMessage() { + java.lang.Object ref = message_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((akka.protobuf.ByteString) ref) + .toStringUtf8(); + message_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string message = 2; + */ + public akka.protobuf.ByteString + getMessageBytes() { + java.lang.Object ref = message_; + if (ref instanceof String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + message_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + /** + * required string message = 2; + */ + public Builder setMessage( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + message_ = value; + onChanged(); + return this; + } + /** + * required string message = 2; + */ + public Builder clearMessage() { + bitField0_ = (bitField0_ & ~0x00000002); + message_ = getDefaultInstance().getMessage(); + onChanged(); + return this; + } + /** + * required string message = 2; + */ + public Builder setMessageBytes( + akka.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + message_ = value; + onChanged(); + return this; + } + + // required .Payload cause = 3; + private akka.remote.ContainerFormats.Payload cause_ = akka.remote.ContainerFormats.Payload.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.Payload, akka.remote.ContainerFormats.Payload.Builder, akka.remote.ContainerFormats.PayloadOrBuilder> causeBuilder_; + /** + * required .Payload cause = 3; + */ + public boolean hasCause() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required .Payload cause = 3; + */ + public akka.remote.ContainerFormats.Payload getCause() { + if (causeBuilder_ == null) { + return cause_; + } else { + return causeBuilder_.getMessage(); + } + } + /** + * required .Payload cause = 3; + */ + public Builder setCause(akka.remote.ContainerFormats.Payload value) { + if (causeBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + cause_ = value; + onChanged(); + } else { + causeBuilder_.setMessage(value); + } + bitField0_ |= 0x00000004; + return this; + } + /** + * required .Payload cause = 3; + */ + public Builder setCause( + akka.remote.ContainerFormats.Payload.Builder builderForValue) { + if (causeBuilder_ == null) { + cause_ = builderForValue.build(); + onChanged(); + } else { + causeBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000004; + return this; + } + /** + * required .Payload cause = 3; + */ + public Builder mergeCause(akka.remote.ContainerFormats.Payload value) { + if (causeBuilder_ == null) { + if (((bitField0_ & 0x00000004) == 0x00000004) && + cause_ != akka.remote.ContainerFormats.Payload.getDefaultInstance()) { + cause_ = + akka.remote.ContainerFormats.Payload.newBuilder(cause_).mergeFrom(value).buildPartial(); + } else { + cause_ = value; + } + onChanged(); + } else { + causeBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000004; + return this; + } + /** + * required .Payload cause = 3; + */ + public Builder clearCause() { + if (causeBuilder_ == null) { + cause_ = akka.remote.ContainerFormats.Payload.getDefaultInstance(); + onChanged(); + } else { + causeBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + /** + * required .Payload cause = 3; + */ + public akka.remote.ContainerFormats.Payload.Builder getCauseBuilder() { + bitField0_ |= 0x00000004; + onChanged(); + return getCauseFieldBuilder().getBuilder(); + } + /** + * required .Payload cause = 3; + */ + public akka.remote.ContainerFormats.PayloadOrBuilder getCauseOrBuilder() { + if (causeBuilder_ != null) { + return causeBuilder_.getMessageOrBuilder(); + } else { + return cause_; + } + } + /** + * required .Payload cause = 3; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.Payload, akka.remote.ContainerFormats.Payload.Builder, akka.remote.ContainerFormats.PayloadOrBuilder> + getCauseFieldBuilder() { + if (causeBuilder_ == null) { + causeBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.Payload, akka.remote.ContainerFormats.Payload.Builder, akka.remote.ContainerFormats.PayloadOrBuilder>( + cause_, + getParentForChildren(), + isClean()); + cause_ = null; + } + return causeBuilder_; + } + + // @@protoc_insertion_point(builder_scope:ActorInitializationException) + } + + static { + defaultInstance = new ActorInitializationException(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:ActorInitializationException) + } + + public interface StackTraceElementOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required string className = 1; + /** + * required string className = 1; + */ + boolean hasClassName(); + /** + * required string className = 1; + */ + java.lang.String getClassName(); + /** + * required string className = 1; + */ + akka.protobuf.ByteString + getClassNameBytes(); + + // required string methodName = 2; + /** + * required string methodName = 2; + */ + boolean hasMethodName(); + /** + * required string methodName = 2; + */ + java.lang.String getMethodName(); + /** + * required string methodName = 2; + */ + akka.protobuf.ByteString + getMethodNameBytes(); + + // required string fileName = 3; + /** + * required string fileName = 3; + */ + boolean hasFileName(); + /** + * required string fileName = 3; + */ + java.lang.String getFileName(); + /** + * required string fileName = 3; + */ + akka.protobuf.ByteString + getFileNameBytes(); + + // required int32 lineNumber = 4; + /** + * required int32 lineNumber = 4; + */ + boolean hasLineNumber(); + /** + * required int32 lineNumber = 4; + */ + int getLineNumber(); + } + /** + * Protobuf type {@code StackTraceElement} + */ + public static final class StackTraceElement extends + akka.protobuf.GeneratedMessage + implements StackTraceElementOrBuilder { + // Use StackTraceElement.newBuilder() to construct. + private StackTraceElement(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private StackTraceElement(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final StackTraceElement defaultInstance; + public static StackTraceElement getDefaultInstance() { + return defaultInstance; + } + + public StackTraceElement getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private StackTraceElement( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + className_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + methodName_ = input.readBytes(); + break; + } + case 26: { + bitField0_ |= 0x00000004; + fileName_ = input.readBytes(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + lineNumber_ = input.readInt32(); + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ContainerFormats.internal_static_StackTraceElement_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ContainerFormats.internal_static_StackTraceElement_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ContainerFormats.StackTraceElement.class, akka.remote.ContainerFormats.StackTraceElement.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public StackTraceElement parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new StackTraceElement(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required string className = 1; + public static final int CLASSNAME_FIELD_NUMBER = 1; + private java.lang.Object className_; + /** + * required string className = 1; + */ + public boolean hasClassName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required string className = 1; + */ + public java.lang.String getClassName() { + java.lang.Object ref = className_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + akka.protobuf.ByteString bs = + (akka.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + className_ = s; + } + return s; + } + } + /** + * required string className = 1; + */ + public akka.protobuf.ByteString + getClassNameBytes() { + java.lang.Object ref = className_; + if (ref instanceof java.lang.String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + className_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + + // required string methodName = 2; + public static final int METHODNAME_FIELD_NUMBER = 2; + private java.lang.Object methodName_; + /** + * required string methodName = 2; + */ + public boolean hasMethodName() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required string methodName = 2; + */ + public java.lang.String getMethodName() { + java.lang.Object ref = methodName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + akka.protobuf.ByteString bs = + (akka.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + methodName_ = s; + } + return s; + } + } + /** + * required string methodName = 2; + */ + public akka.protobuf.ByteString + getMethodNameBytes() { + java.lang.Object ref = methodName_; + if (ref instanceof java.lang.String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + methodName_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + + // required string fileName = 3; + public static final int FILENAME_FIELD_NUMBER = 3; + private java.lang.Object fileName_; + /** + * required string fileName = 3; + */ + public boolean hasFileName() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required string fileName = 3; + */ + public java.lang.String getFileName() { + java.lang.Object ref = fileName_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + akka.protobuf.ByteString bs = + (akka.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + fileName_ = s; + } + return s; + } + } + /** + * required string fileName = 3; + */ + public akka.protobuf.ByteString + getFileNameBytes() { + java.lang.Object ref = fileName_; + if (ref instanceof java.lang.String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + fileName_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + + // required int32 lineNumber = 4; + public static final int LINENUMBER_FIELD_NUMBER = 4; + private int lineNumber_; + /** + * required int32 lineNumber = 4; + */ + public boolean hasLineNumber() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * required int32 lineNumber = 4; + */ + public int getLineNumber() { + return lineNumber_; + } + + private void initFields() { + className_ = ""; + methodName_ = ""; + fileName_ = ""; + lineNumber_ = 0; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasClassName()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasMethodName()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasFileName()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasLineNumber()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getClassNameBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getMethodNameBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, getFileNameBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeInt32(4, lineNumber_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeBytesSize(1, getClassNameBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += akka.protobuf.CodedOutputStream + .computeBytesSize(2, getMethodNameBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += akka.protobuf.CodedOutputStream + .computeBytesSize(3, getFileNameBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += akka.protobuf.CodedOutputStream + .computeInt32Size(4, lineNumber_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.ContainerFormats.StackTraceElement parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ContainerFormats.StackTraceElement parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ContainerFormats.StackTraceElement parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.ContainerFormats.StackTraceElement parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.ContainerFormats.StackTraceElement parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ContainerFormats.StackTraceElement parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.ContainerFormats.StackTraceElement parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.ContainerFormats.StackTraceElement parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.ContainerFormats.StackTraceElement parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.ContainerFormats.StackTraceElement parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.ContainerFormats.StackTraceElement prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code StackTraceElement} + */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.ContainerFormats.StackTraceElementOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.ContainerFormats.internal_static_StackTraceElement_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.ContainerFormats.internal_static_StackTraceElement_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.ContainerFormats.StackTraceElement.class, akka.remote.ContainerFormats.StackTraceElement.Builder.class); + } + + // Construct using akka.remote.ContainerFormats.StackTraceElement.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + className_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + methodName_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + fileName_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + lineNumber_ = 0; + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.ContainerFormats.internal_static_StackTraceElement_descriptor; + } + + public akka.remote.ContainerFormats.StackTraceElement getDefaultInstanceForType() { + return akka.remote.ContainerFormats.StackTraceElement.getDefaultInstance(); + } + + public akka.remote.ContainerFormats.StackTraceElement build() { + akka.remote.ContainerFormats.StackTraceElement result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.ContainerFormats.StackTraceElement buildPartial() { + akka.remote.ContainerFormats.StackTraceElement result = new akka.remote.ContainerFormats.StackTraceElement(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.className_ = className_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.methodName_ = methodName_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.fileName_ = fileName_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.lineNumber_ = lineNumber_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.ContainerFormats.StackTraceElement) { + return mergeFrom((akka.remote.ContainerFormats.StackTraceElement)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.ContainerFormats.StackTraceElement other) { + if (other == akka.remote.ContainerFormats.StackTraceElement.getDefaultInstance()) return this; + if (other.hasClassName()) { + bitField0_ |= 0x00000001; + className_ = other.className_; + onChanged(); + } + if (other.hasMethodName()) { + bitField0_ |= 0x00000002; + methodName_ = other.methodName_; + onChanged(); + } + if (other.hasFileName()) { + bitField0_ |= 0x00000004; + fileName_ = other.fileName_; + onChanged(); + } + if (other.hasLineNumber()) { + setLineNumber(other.getLineNumber()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasClassName()) { + + return false; + } + if (!hasMethodName()) { + + return false; + } + if (!hasFileName()) { + + return false; + } + if (!hasLineNumber()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.ContainerFormats.StackTraceElement parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.ContainerFormats.StackTraceElement) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required string className = 1; + private java.lang.Object className_ = ""; + /** + * required string className = 1; + */ + public boolean hasClassName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required string className = 1; + */ + public java.lang.String getClassName() { + java.lang.Object ref = className_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((akka.protobuf.ByteString) ref) + .toStringUtf8(); + className_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string className = 1; + */ + public akka.protobuf.ByteString + getClassNameBytes() { + java.lang.Object ref = className_; + if (ref instanceof String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + className_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + /** + * required string className = 1; + */ + public Builder setClassName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + className_ = value; + onChanged(); + return this; + } + /** + * required string className = 1; + */ + public Builder clearClassName() { + bitField0_ = (bitField0_ & ~0x00000001); + className_ = getDefaultInstance().getClassName(); + onChanged(); + return this; + } + /** + * required string className = 1; + */ + public Builder setClassNameBytes( + akka.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + className_ = value; + onChanged(); + return this; + } + + // required string methodName = 2; + private java.lang.Object methodName_ = ""; + /** + * required string methodName = 2; + */ + public boolean hasMethodName() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required string methodName = 2; + */ + public java.lang.String getMethodName() { + java.lang.Object ref = methodName_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((akka.protobuf.ByteString) ref) + .toStringUtf8(); + methodName_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string methodName = 2; + */ + public akka.protobuf.ByteString + getMethodNameBytes() { + java.lang.Object ref = methodName_; + if (ref instanceof String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + methodName_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + /** + * required string methodName = 2; + */ + public Builder setMethodName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + methodName_ = value; + onChanged(); + return this; + } + /** + * required string methodName = 2; + */ + public Builder clearMethodName() { + bitField0_ = (bitField0_ & ~0x00000002); + methodName_ = getDefaultInstance().getMethodName(); + onChanged(); + return this; + } + /** + * required string methodName = 2; + */ + public Builder setMethodNameBytes( + akka.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + methodName_ = value; + onChanged(); + return this; + } + + // required string fileName = 3; + private java.lang.Object fileName_ = ""; + /** + * required string fileName = 3; + */ + public boolean hasFileName() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required string fileName = 3; + */ + public java.lang.String getFileName() { + java.lang.Object ref = fileName_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((akka.protobuf.ByteString) ref) + .toStringUtf8(); + fileName_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string fileName = 3; + */ + public akka.protobuf.ByteString + getFileNameBytes() { + java.lang.Object ref = fileName_; + if (ref instanceof String) { + akka.protobuf.ByteString b = + akka.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + fileName_ = b; + return b; + } else { + return (akka.protobuf.ByteString) ref; + } + } + /** + * required string fileName = 3; + */ + public Builder setFileName( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + fileName_ = value; + onChanged(); + return this; + } + /** + * required string fileName = 3; + */ + public Builder clearFileName() { + bitField0_ = (bitField0_ & ~0x00000004); + fileName_ = getDefaultInstance().getFileName(); + onChanged(); + return this; + } + /** + * required string fileName = 3; + */ + public Builder setFileNameBytes( + akka.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + fileName_ = value; + onChanged(); + return this; + } + + // required int32 lineNumber = 4; + private int lineNumber_ ; + /** + * required int32 lineNumber = 4; + */ + public boolean hasLineNumber() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * required int32 lineNumber = 4; + */ + public int getLineNumber() { + return lineNumber_; + } + /** + * required int32 lineNumber = 4; + */ + public Builder setLineNumber(int value) { + bitField0_ |= 0x00000008; + lineNumber_ = value; + onChanged(); + return this; + } + /** + * required int32 lineNumber = 4; + */ + public Builder clearLineNumber() { + bitField0_ = (bitField0_ & ~0x00000008); + lineNumber_ = 0; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:StackTraceElement) + } + + static { + defaultInstance = new StackTraceElement(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:StackTraceElement) + } + private static akka.protobuf.Descriptors.Descriptor internal_static_SelectionEnvelope_descriptor; private static @@ -4598,6 +8002,26 @@ public final class ContainerFormats { private static akka.protobuf.GeneratedMessage.FieldAccessorTable internal_static_Payload_fieldAccessorTable; + private static akka.protobuf.Descriptors.Descriptor + internal_static_WatcherHeartbeatResponse_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_WatcherHeartbeatResponse_fieldAccessorTable; + private static akka.protobuf.Descriptors.Descriptor + internal_static_Throwable_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_Throwable_fieldAccessorTable; + private static akka.protobuf.Descriptors.Descriptor + internal_static_ActorInitializationException_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_ActorInitializationException_fieldAccessorTable; + private static akka.protobuf.Descriptors.Descriptor + internal_static_StackTraceElement_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_StackTraceElement_fieldAccessorTable; public static akka.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -4619,9 +8043,17 @@ public final class ContainerFormats { "path\030\001 \002(\t\"!\n\006Option\022\027\n\005value\030\001 \001(\0132\010.Pa", "yload\"Q\n\007Payload\022\027\n\017enclosedMessage\030\001 \002(" + "\014\022\024\n\014serializerId\030\002 \002(\005\022\027\n\017messageManife" + - "st\030\004 \001(\014*<\n\013PatternType\022\n\n\006PARENT\020\000\022\016\n\nC" + - "HILD_NAME\020\001\022\021\n\rCHILD_PATTERN\020\002B\017\n\013akka.r" + - "emoteH\001" + "st\030\004 \001(\014\"\'\n\030WatcherHeartbeatResponse\022\013\n\003" + + "uid\030\001 \002(\004\"p\n\tThrowable\022\021\n\tclassName\030\001 \002(" + + "\t\022\017\n\007message\030\002 \001(\t\022\027\n\005cause\030\003 \001(\0132\010.Payl" + + "oad\022&\n\nstackTrace\030\004 \003(\0132\022.StackTraceElem" + + "ent\"b\n\034ActorInitializationException\022\030\n\005a" + + "ctor\030\001 \001(\0132\t.ActorRef\022\017\n\007message\030\002 \002(\t\022\027" + + "\n\005cause\030\003 \002(\0132\010.Payload\"`\n\021StackTraceEle" + + "ment\022\021\n\tclassName\030\001 \002(\t\022\022\n\nmethodName\030\002 ", + "\002(\t\022\020\n\010fileName\030\003 \002(\t\022\022\n\nlineNumber\030\004 \002(" + + "\005*<\n\013PatternType\022\n\n\006PARENT\020\000\022\016\n\nCHILD_NA" + + "ME\020\001\022\021\n\rCHILD_PATTERN\020\002B\017\n\013akka.remoteH\001" }; akka.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new akka.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -4670,6 +8102,30 @@ public final class ContainerFormats { akka.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_Payload_descriptor, new java.lang.String[] { "EnclosedMessage", "SerializerId", "MessageManifest", }); + internal_static_WatcherHeartbeatResponse_descriptor = + getDescriptor().getMessageTypes().get(7); + internal_static_WatcherHeartbeatResponse_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_WatcherHeartbeatResponse_descriptor, + new java.lang.String[] { "Uid", }); + internal_static_Throwable_descriptor = + getDescriptor().getMessageTypes().get(8); + internal_static_Throwable_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_Throwable_descriptor, + new java.lang.String[] { "ClassName", "Message", "Cause", "StackTrace", }); + internal_static_ActorInitializationException_descriptor = + getDescriptor().getMessageTypes().get(9); + internal_static_ActorInitializationException_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_ActorInitializationException_descriptor, + new java.lang.String[] { "Actor", "Message", "Cause", }); + internal_static_StackTraceElement_descriptor = + getDescriptor().getMessageTypes().get(10); + internal_static_StackTraceElement_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_StackTraceElement_descriptor, + new java.lang.String[] { "ClassName", "MethodName", "FileName", "LineNumber", }); return null; } }; diff --git a/akka-remote/src/main/java/akka/remote/SystemMessageFormats.java b/akka-remote/src/main/java/akka/remote/SystemMessageFormats.java new file mode 100644 index 0000000000..122d30729a --- /dev/null +++ b/akka-remote/src/main/java/akka/remote/SystemMessageFormats.java @@ -0,0 +1,4354 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: SystemMessageFormats.proto + +package akka.remote; + +public final class SystemMessageFormats { + private SystemMessageFormats() {} + public static void registerAllExtensions( + akka.protobuf.ExtensionRegistry registry) { + } + public interface SystemMessageOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required .SystemMessage.Type type = 1; + /** + * required .SystemMessage.Type type = 1; + */ + boolean hasType(); + /** + * required .SystemMessage.Type type = 1; + */ + akka.remote.SystemMessageFormats.SystemMessage.Type getType(); + + // optional .WatchData watchData = 2; + /** + * optional .WatchData watchData = 2; + */ + boolean hasWatchData(); + /** + * optional .WatchData watchData = 2; + */ + akka.remote.SystemMessageFormats.WatchData getWatchData(); + /** + * optional .WatchData watchData = 2; + */ + akka.remote.SystemMessageFormats.WatchDataOrBuilder getWatchDataOrBuilder(); + + // optional .Payload causeData = 3; + /** + * optional .Payload causeData = 3; + */ + boolean hasCauseData(); + /** + * optional .Payload causeData = 3; + */ + akka.remote.ContainerFormats.Payload getCauseData(); + /** + * optional .Payload causeData = 3; + */ + akka.remote.ContainerFormats.PayloadOrBuilder getCauseDataOrBuilder(); + + // optional .SuperviseData superviseData = 5; + /** + * optional .SuperviseData superviseData = 5; + */ + boolean hasSuperviseData(); + /** + * optional .SuperviseData superviseData = 5; + */ + akka.remote.SystemMessageFormats.SuperviseData getSuperviseData(); + /** + * optional .SuperviseData superviseData = 5; + */ + akka.remote.SystemMessageFormats.SuperviseDataOrBuilder getSuperviseDataOrBuilder(); + + // optional .FailedData failedData = 6; + /** + * optional .FailedData failedData = 6; + */ + boolean hasFailedData(); + /** + * optional .FailedData failedData = 6; + */ + akka.remote.SystemMessageFormats.FailedData getFailedData(); + /** + * optional .FailedData failedData = 6; + */ + akka.remote.SystemMessageFormats.FailedDataOrBuilder getFailedDataOrBuilder(); + + // optional .DeathWatchNotificationData dwNotificationData = 7; + /** + * optional .DeathWatchNotificationData dwNotificationData = 7; + */ + boolean hasDwNotificationData(); + /** + * optional .DeathWatchNotificationData dwNotificationData = 7; + */ + akka.remote.SystemMessageFormats.DeathWatchNotificationData getDwNotificationData(); + /** + * optional .DeathWatchNotificationData dwNotificationData = 7; + */ + akka.remote.SystemMessageFormats.DeathWatchNotificationDataOrBuilder getDwNotificationDataOrBuilder(); + } + /** + * Protobuf type {@code SystemMessage} + */ + public static final class SystemMessage extends + akka.protobuf.GeneratedMessage + implements SystemMessageOrBuilder { + // Use SystemMessage.newBuilder() to construct. + private SystemMessage(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private SystemMessage(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final SystemMessage defaultInstance; + public static SystemMessage getDefaultInstance() { + return defaultInstance; + } + + public SystemMessage getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private SystemMessage( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + int rawValue = input.readEnum(); + akka.remote.SystemMessageFormats.SystemMessage.Type value = akka.remote.SystemMessageFormats.SystemMessage.Type.valueOf(rawValue); + if (value == null) { + unknownFields.mergeVarintField(1, rawValue); + } else { + bitField0_ |= 0x00000001; + type_ = value; + } + break; + } + case 18: { + akka.remote.SystemMessageFormats.WatchData.Builder subBuilder = null; + if (((bitField0_ & 0x00000002) == 0x00000002)) { + subBuilder = watchData_.toBuilder(); + } + watchData_ = input.readMessage(akka.remote.SystemMessageFormats.WatchData.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(watchData_); + watchData_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000002; + break; + } + case 26: { + akka.remote.ContainerFormats.Payload.Builder subBuilder = null; + if (((bitField0_ & 0x00000004) == 0x00000004)) { + subBuilder = causeData_.toBuilder(); + } + causeData_ = input.readMessage(akka.remote.ContainerFormats.Payload.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(causeData_); + causeData_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000004; + break; + } + case 42: { + akka.remote.SystemMessageFormats.SuperviseData.Builder subBuilder = null; + if (((bitField0_ & 0x00000008) == 0x00000008)) { + subBuilder = superviseData_.toBuilder(); + } + superviseData_ = input.readMessage(akka.remote.SystemMessageFormats.SuperviseData.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(superviseData_); + superviseData_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000008; + break; + } + case 50: { + akka.remote.SystemMessageFormats.FailedData.Builder subBuilder = null; + if (((bitField0_ & 0x00000010) == 0x00000010)) { + subBuilder = failedData_.toBuilder(); + } + failedData_ = input.readMessage(akka.remote.SystemMessageFormats.FailedData.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(failedData_); + failedData_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000010; + break; + } + case 58: { + akka.remote.SystemMessageFormats.DeathWatchNotificationData.Builder subBuilder = null; + if (((bitField0_ & 0x00000020) == 0x00000020)) { + subBuilder = dwNotificationData_.toBuilder(); + } + dwNotificationData_ = input.readMessage(akka.remote.SystemMessageFormats.DeathWatchNotificationData.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(dwNotificationData_); + dwNotificationData_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000020; + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.SystemMessageFormats.internal_static_SystemMessage_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.SystemMessageFormats.internal_static_SystemMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.SystemMessageFormats.SystemMessage.class, akka.remote.SystemMessageFormats.SystemMessage.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public SystemMessage parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new SystemMessage(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + /** + * Protobuf enum {@code SystemMessage.Type} + */ + public enum Type + implements akka.protobuf.ProtocolMessageEnum { + /** + * CREATE = 0; + */ + CREATE(0, 0), + /** + * RECREATE = 1; + */ + RECREATE(1, 1), + /** + * SUSPEND = 2; + */ + SUSPEND(2, 2), + /** + * RESUME = 3; + */ + RESUME(3, 3), + /** + * TERMINATE = 4; + */ + TERMINATE(4, 4), + /** + * SUPERVISE = 5; + */ + SUPERVISE(5, 5), + /** + * WATCH = 6; + */ + WATCH(6, 6), + /** + * UNWATCH = 7; + */ + UNWATCH(7, 7), + /** + * FAILED = 8; + */ + FAILED(8, 8), + /** + * DEATHWATCH_NOTIFICATION = 9; + */ + DEATHWATCH_NOTIFICATION(9, 9), + ; + + /** + * CREATE = 0; + */ + public static final int CREATE_VALUE = 0; + /** + * RECREATE = 1; + */ + public static final int RECREATE_VALUE = 1; + /** + * SUSPEND = 2; + */ + public static final int SUSPEND_VALUE = 2; + /** + * RESUME = 3; + */ + public static final int RESUME_VALUE = 3; + /** + * TERMINATE = 4; + */ + public static final int TERMINATE_VALUE = 4; + /** + * SUPERVISE = 5; + */ + public static final int SUPERVISE_VALUE = 5; + /** + * WATCH = 6; + */ + public static final int WATCH_VALUE = 6; + /** + * UNWATCH = 7; + */ + public static final int UNWATCH_VALUE = 7; + /** + * FAILED = 8; + */ + public static final int FAILED_VALUE = 8; + /** + * DEATHWATCH_NOTIFICATION = 9; + */ + public static final int DEATHWATCH_NOTIFICATION_VALUE = 9; + + + public final int getNumber() { return value; } + + public static Type valueOf(int value) { + switch (value) { + case 0: return CREATE; + case 1: return RECREATE; + case 2: return SUSPEND; + case 3: return RESUME; + case 4: return TERMINATE; + case 5: return SUPERVISE; + case 6: return WATCH; + case 7: return UNWATCH; + case 8: return FAILED; + case 9: return DEATHWATCH_NOTIFICATION; + default: return null; + } + } + + public static akka.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static akka.protobuf.Internal.EnumLiteMap + internalValueMap = + new akka.protobuf.Internal.EnumLiteMap() { + public Type findValueByNumber(int number) { + return Type.valueOf(number); + } + }; + + public final akka.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + return getDescriptor().getValues().get(index); + } + public final akka.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final akka.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return akka.remote.SystemMessageFormats.SystemMessage.getDescriptor().getEnumTypes().get(0); + } + + private static final Type[] VALUES = values(); + + public static Type valueOf( + akka.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 Type(int index, int value) { + this.index = index; + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:SystemMessage.Type) + } + + private int bitField0_; + // required .SystemMessage.Type type = 1; + public static final int TYPE_FIELD_NUMBER = 1; + private akka.remote.SystemMessageFormats.SystemMessage.Type type_; + /** + * required .SystemMessage.Type type = 1; + */ + public boolean hasType() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .SystemMessage.Type type = 1; + */ + public akka.remote.SystemMessageFormats.SystemMessage.Type getType() { + return type_; + } + + // optional .WatchData watchData = 2; + public static final int WATCHDATA_FIELD_NUMBER = 2; + private akka.remote.SystemMessageFormats.WatchData watchData_; + /** + * optional .WatchData watchData = 2; + */ + public boolean hasWatchData() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional .WatchData watchData = 2; + */ + public akka.remote.SystemMessageFormats.WatchData getWatchData() { + return watchData_; + } + /** + * optional .WatchData watchData = 2; + */ + public akka.remote.SystemMessageFormats.WatchDataOrBuilder getWatchDataOrBuilder() { + return watchData_; + } + + // optional .Payload causeData = 3; + public static final int CAUSEDATA_FIELD_NUMBER = 3; + private akka.remote.ContainerFormats.Payload causeData_; + /** + * optional .Payload causeData = 3; + */ + public boolean hasCauseData() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional .Payload causeData = 3; + */ + public akka.remote.ContainerFormats.Payload getCauseData() { + return causeData_; + } + /** + * optional .Payload causeData = 3; + */ + public akka.remote.ContainerFormats.PayloadOrBuilder getCauseDataOrBuilder() { + return causeData_; + } + + // optional .SuperviseData superviseData = 5; + public static final int SUPERVISEDATA_FIELD_NUMBER = 5; + private akka.remote.SystemMessageFormats.SuperviseData superviseData_; + /** + * optional .SuperviseData superviseData = 5; + */ + public boolean hasSuperviseData() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional .SuperviseData superviseData = 5; + */ + public akka.remote.SystemMessageFormats.SuperviseData getSuperviseData() { + return superviseData_; + } + /** + * optional .SuperviseData superviseData = 5; + */ + public akka.remote.SystemMessageFormats.SuperviseDataOrBuilder getSuperviseDataOrBuilder() { + return superviseData_; + } + + // optional .FailedData failedData = 6; + public static final int FAILEDDATA_FIELD_NUMBER = 6; + private akka.remote.SystemMessageFormats.FailedData failedData_; + /** + * optional .FailedData failedData = 6; + */ + public boolean hasFailedData() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + /** + * optional .FailedData failedData = 6; + */ + public akka.remote.SystemMessageFormats.FailedData getFailedData() { + return failedData_; + } + /** + * optional .FailedData failedData = 6; + */ + public akka.remote.SystemMessageFormats.FailedDataOrBuilder getFailedDataOrBuilder() { + return failedData_; + } + + // optional .DeathWatchNotificationData dwNotificationData = 7; + public static final int DWNOTIFICATIONDATA_FIELD_NUMBER = 7; + private akka.remote.SystemMessageFormats.DeathWatchNotificationData dwNotificationData_; + /** + * optional .DeathWatchNotificationData dwNotificationData = 7; + */ + public boolean hasDwNotificationData() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + /** + * optional .DeathWatchNotificationData dwNotificationData = 7; + */ + public akka.remote.SystemMessageFormats.DeathWatchNotificationData getDwNotificationData() { + return dwNotificationData_; + } + /** + * optional .DeathWatchNotificationData dwNotificationData = 7; + */ + public akka.remote.SystemMessageFormats.DeathWatchNotificationDataOrBuilder getDwNotificationDataOrBuilder() { + return dwNotificationData_; + } + + private void initFields() { + type_ = akka.remote.SystemMessageFormats.SystemMessage.Type.CREATE; + watchData_ = akka.remote.SystemMessageFormats.WatchData.getDefaultInstance(); + causeData_ = akka.remote.ContainerFormats.Payload.getDefaultInstance(); + superviseData_ = akka.remote.SystemMessageFormats.SuperviseData.getDefaultInstance(); + failedData_ = akka.remote.SystemMessageFormats.FailedData.getDefaultInstance(); + dwNotificationData_ = akka.remote.SystemMessageFormats.DeathWatchNotificationData.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasType()) { + memoizedIsInitialized = 0; + return false; + } + if (hasWatchData()) { + if (!getWatchData().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasCauseData()) { + if (!getCauseData().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasSuperviseData()) { + if (!getSuperviseData().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasFailedData()) { + if (!getFailedData().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasDwNotificationData()) { + if (!getDwNotificationData().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeEnum(1, type_.getNumber()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, watchData_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeMessage(3, causeData_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeMessage(5, superviseData_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeMessage(6, failedData_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeMessage(7, dwNotificationData_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeEnumSize(1, type_.getNumber()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(2, watchData_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(3, causeData_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(5, superviseData_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(6, failedData_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(7, dwNotificationData_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.SystemMessageFormats.SystemMessage parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.SystemMessageFormats.SystemMessage parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.SystemMessage parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.SystemMessageFormats.SystemMessage parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.SystemMessage parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.SystemMessageFormats.SystemMessage parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.SystemMessage parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.SystemMessageFormats.SystemMessage parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.SystemMessage parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.SystemMessageFormats.SystemMessage parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.SystemMessageFormats.SystemMessage prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code SystemMessage} + */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.SystemMessageFormats.SystemMessageOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.SystemMessageFormats.internal_static_SystemMessage_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.SystemMessageFormats.internal_static_SystemMessage_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.SystemMessageFormats.SystemMessage.class, akka.remote.SystemMessageFormats.SystemMessage.Builder.class); + } + + // Construct using akka.remote.SystemMessageFormats.SystemMessage.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getWatchDataFieldBuilder(); + getCauseDataFieldBuilder(); + getSuperviseDataFieldBuilder(); + getFailedDataFieldBuilder(); + getDwNotificationDataFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + type_ = akka.remote.SystemMessageFormats.SystemMessage.Type.CREATE; + bitField0_ = (bitField0_ & ~0x00000001); + if (watchDataBuilder_ == null) { + watchData_ = akka.remote.SystemMessageFormats.WatchData.getDefaultInstance(); + } else { + watchDataBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + if (causeDataBuilder_ == null) { + causeData_ = akka.remote.ContainerFormats.Payload.getDefaultInstance(); + } else { + causeDataBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + if (superviseDataBuilder_ == null) { + superviseData_ = akka.remote.SystemMessageFormats.SuperviseData.getDefaultInstance(); + } else { + superviseDataBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000008); + if (failedDataBuilder_ == null) { + failedData_ = akka.remote.SystemMessageFormats.FailedData.getDefaultInstance(); + } else { + failedDataBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000010); + if (dwNotificationDataBuilder_ == null) { + dwNotificationData_ = akka.remote.SystemMessageFormats.DeathWatchNotificationData.getDefaultInstance(); + } else { + dwNotificationDataBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000020); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.SystemMessageFormats.internal_static_SystemMessage_descriptor; + } + + public akka.remote.SystemMessageFormats.SystemMessage getDefaultInstanceForType() { + return akka.remote.SystemMessageFormats.SystemMessage.getDefaultInstance(); + } + + public akka.remote.SystemMessageFormats.SystemMessage build() { + akka.remote.SystemMessageFormats.SystemMessage result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.SystemMessageFormats.SystemMessage buildPartial() { + akka.remote.SystemMessageFormats.SystemMessage result = new akka.remote.SystemMessageFormats.SystemMessage(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.type_ = type_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (watchDataBuilder_ == null) { + result.watchData_ = watchData_; + } else { + result.watchData_ = watchDataBuilder_.build(); + } + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + if (causeDataBuilder_ == null) { + result.causeData_ = causeData_; + } else { + result.causeData_ = causeDataBuilder_.build(); + } + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + if (superviseDataBuilder_ == null) { + result.superviseData_ = superviseData_; + } else { + result.superviseData_ = superviseDataBuilder_.build(); + } + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + if (failedDataBuilder_ == null) { + result.failedData_ = failedData_; + } else { + result.failedData_ = failedDataBuilder_.build(); + } + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000020; + } + if (dwNotificationDataBuilder_ == null) { + result.dwNotificationData_ = dwNotificationData_; + } else { + result.dwNotificationData_ = dwNotificationDataBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.SystemMessageFormats.SystemMessage) { + return mergeFrom((akka.remote.SystemMessageFormats.SystemMessage)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.SystemMessageFormats.SystemMessage other) { + if (other == akka.remote.SystemMessageFormats.SystemMessage.getDefaultInstance()) return this; + if (other.hasType()) { + setType(other.getType()); + } + if (other.hasWatchData()) { + mergeWatchData(other.getWatchData()); + } + if (other.hasCauseData()) { + mergeCauseData(other.getCauseData()); + } + if (other.hasSuperviseData()) { + mergeSuperviseData(other.getSuperviseData()); + } + if (other.hasFailedData()) { + mergeFailedData(other.getFailedData()); + } + if (other.hasDwNotificationData()) { + mergeDwNotificationData(other.getDwNotificationData()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasType()) { + + return false; + } + if (hasWatchData()) { + if (!getWatchData().isInitialized()) { + + return false; + } + } + if (hasCauseData()) { + if (!getCauseData().isInitialized()) { + + return false; + } + } + if (hasSuperviseData()) { + if (!getSuperviseData().isInitialized()) { + + return false; + } + } + if (hasFailedData()) { + if (!getFailedData().isInitialized()) { + + return false; + } + } + if (hasDwNotificationData()) { + if (!getDwNotificationData().isInitialized()) { + + return false; + } + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.SystemMessageFormats.SystemMessage parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.SystemMessageFormats.SystemMessage) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required .SystemMessage.Type type = 1; + private akka.remote.SystemMessageFormats.SystemMessage.Type type_ = akka.remote.SystemMessageFormats.SystemMessage.Type.CREATE; + /** + * required .SystemMessage.Type type = 1; + */ + public boolean hasType() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .SystemMessage.Type type = 1; + */ + public akka.remote.SystemMessageFormats.SystemMessage.Type getType() { + return type_; + } + /** + * required .SystemMessage.Type type = 1; + */ + public Builder setType(akka.remote.SystemMessageFormats.SystemMessage.Type value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + type_ = value; + onChanged(); + return this; + } + /** + * required .SystemMessage.Type type = 1; + */ + public Builder clearType() { + bitField0_ = (bitField0_ & ~0x00000001); + type_ = akka.remote.SystemMessageFormats.SystemMessage.Type.CREATE; + onChanged(); + return this; + } + + // optional .WatchData watchData = 2; + private akka.remote.SystemMessageFormats.WatchData watchData_ = akka.remote.SystemMessageFormats.WatchData.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.SystemMessageFormats.WatchData, akka.remote.SystemMessageFormats.WatchData.Builder, akka.remote.SystemMessageFormats.WatchDataOrBuilder> watchDataBuilder_; + /** + * optional .WatchData watchData = 2; + */ + public boolean hasWatchData() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional .WatchData watchData = 2; + */ + public akka.remote.SystemMessageFormats.WatchData getWatchData() { + if (watchDataBuilder_ == null) { + return watchData_; + } else { + return watchDataBuilder_.getMessage(); + } + } + /** + * optional .WatchData watchData = 2; + */ + public Builder setWatchData(akka.remote.SystemMessageFormats.WatchData value) { + if (watchDataBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + watchData_ = value; + onChanged(); + } else { + watchDataBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * optional .WatchData watchData = 2; + */ + public Builder setWatchData( + akka.remote.SystemMessageFormats.WatchData.Builder builderForValue) { + if (watchDataBuilder_ == null) { + watchData_ = builderForValue.build(); + onChanged(); + } else { + watchDataBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * optional .WatchData watchData = 2; + */ + public Builder mergeWatchData(akka.remote.SystemMessageFormats.WatchData value) { + if (watchDataBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + watchData_ != akka.remote.SystemMessageFormats.WatchData.getDefaultInstance()) { + watchData_ = + akka.remote.SystemMessageFormats.WatchData.newBuilder(watchData_).mergeFrom(value).buildPartial(); + } else { + watchData_ = value; + } + onChanged(); + } else { + watchDataBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * optional .WatchData watchData = 2; + */ + public Builder clearWatchData() { + if (watchDataBuilder_ == null) { + watchData_ = akka.remote.SystemMessageFormats.WatchData.getDefaultInstance(); + onChanged(); + } else { + watchDataBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + /** + * optional .WatchData watchData = 2; + */ + public akka.remote.SystemMessageFormats.WatchData.Builder getWatchDataBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getWatchDataFieldBuilder().getBuilder(); + } + /** + * optional .WatchData watchData = 2; + */ + public akka.remote.SystemMessageFormats.WatchDataOrBuilder getWatchDataOrBuilder() { + if (watchDataBuilder_ != null) { + return watchDataBuilder_.getMessageOrBuilder(); + } else { + return watchData_; + } + } + /** + * optional .WatchData watchData = 2; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.SystemMessageFormats.WatchData, akka.remote.SystemMessageFormats.WatchData.Builder, akka.remote.SystemMessageFormats.WatchDataOrBuilder> + getWatchDataFieldBuilder() { + if (watchDataBuilder_ == null) { + watchDataBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.SystemMessageFormats.WatchData, akka.remote.SystemMessageFormats.WatchData.Builder, akka.remote.SystemMessageFormats.WatchDataOrBuilder>( + watchData_, + getParentForChildren(), + isClean()); + watchData_ = null; + } + return watchDataBuilder_; + } + + // optional .Payload causeData = 3; + private akka.remote.ContainerFormats.Payload causeData_ = akka.remote.ContainerFormats.Payload.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.Payload, akka.remote.ContainerFormats.Payload.Builder, akka.remote.ContainerFormats.PayloadOrBuilder> causeDataBuilder_; + /** + * optional .Payload causeData = 3; + */ + public boolean hasCauseData() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * optional .Payload causeData = 3; + */ + public akka.remote.ContainerFormats.Payload getCauseData() { + if (causeDataBuilder_ == null) { + return causeData_; + } else { + return causeDataBuilder_.getMessage(); + } + } + /** + * optional .Payload causeData = 3; + */ + public Builder setCauseData(akka.remote.ContainerFormats.Payload value) { + if (causeDataBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + causeData_ = value; + onChanged(); + } else { + causeDataBuilder_.setMessage(value); + } + bitField0_ |= 0x00000004; + return this; + } + /** + * optional .Payload causeData = 3; + */ + public Builder setCauseData( + akka.remote.ContainerFormats.Payload.Builder builderForValue) { + if (causeDataBuilder_ == null) { + causeData_ = builderForValue.build(); + onChanged(); + } else { + causeDataBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000004; + return this; + } + /** + * optional .Payload causeData = 3; + */ + public Builder mergeCauseData(akka.remote.ContainerFormats.Payload value) { + if (causeDataBuilder_ == null) { + if (((bitField0_ & 0x00000004) == 0x00000004) && + causeData_ != akka.remote.ContainerFormats.Payload.getDefaultInstance()) { + causeData_ = + akka.remote.ContainerFormats.Payload.newBuilder(causeData_).mergeFrom(value).buildPartial(); + } else { + causeData_ = value; + } + onChanged(); + } else { + causeDataBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000004; + return this; + } + /** + * optional .Payload causeData = 3; + */ + public Builder clearCauseData() { + if (causeDataBuilder_ == null) { + causeData_ = akka.remote.ContainerFormats.Payload.getDefaultInstance(); + onChanged(); + } else { + causeDataBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + /** + * optional .Payload causeData = 3; + */ + public akka.remote.ContainerFormats.Payload.Builder getCauseDataBuilder() { + bitField0_ |= 0x00000004; + onChanged(); + return getCauseDataFieldBuilder().getBuilder(); + } + /** + * optional .Payload causeData = 3; + */ + public akka.remote.ContainerFormats.PayloadOrBuilder getCauseDataOrBuilder() { + if (causeDataBuilder_ != null) { + return causeDataBuilder_.getMessageOrBuilder(); + } else { + return causeData_; + } + } + /** + * optional .Payload causeData = 3; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.Payload, akka.remote.ContainerFormats.Payload.Builder, akka.remote.ContainerFormats.PayloadOrBuilder> + getCauseDataFieldBuilder() { + if (causeDataBuilder_ == null) { + causeDataBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.Payload, akka.remote.ContainerFormats.Payload.Builder, akka.remote.ContainerFormats.PayloadOrBuilder>( + causeData_, + getParentForChildren(), + isClean()); + causeData_ = null; + } + return causeDataBuilder_; + } + + // optional .SuperviseData superviseData = 5; + private akka.remote.SystemMessageFormats.SuperviseData superviseData_ = akka.remote.SystemMessageFormats.SuperviseData.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.SystemMessageFormats.SuperviseData, akka.remote.SystemMessageFormats.SuperviseData.Builder, akka.remote.SystemMessageFormats.SuperviseDataOrBuilder> superviseDataBuilder_; + /** + * optional .SuperviseData superviseData = 5; + */ + public boolean hasSuperviseData() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional .SuperviseData superviseData = 5; + */ + public akka.remote.SystemMessageFormats.SuperviseData getSuperviseData() { + if (superviseDataBuilder_ == null) { + return superviseData_; + } else { + return superviseDataBuilder_.getMessage(); + } + } + /** + * optional .SuperviseData superviseData = 5; + */ + public Builder setSuperviseData(akka.remote.SystemMessageFormats.SuperviseData value) { + if (superviseDataBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + superviseData_ = value; + onChanged(); + } else { + superviseDataBuilder_.setMessage(value); + } + bitField0_ |= 0x00000008; + return this; + } + /** + * optional .SuperviseData superviseData = 5; + */ + public Builder setSuperviseData( + akka.remote.SystemMessageFormats.SuperviseData.Builder builderForValue) { + if (superviseDataBuilder_ == null) { + superviseData_ = builderForValue.build(); + onChanged(); + } else { + superviseDataBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000008; + return this; + } + /** + * optional .SuperviseData superviseData = 5; + */ + public Builder mergeSuperviseData(akka.remote.SystemMessageFormats.SuperviseData value) { + if (superviseDataBuilder_ == null) { + if (((bitField0_ & 0x00000008) == 0x00000008) && + superviseData_ != akka.remote.SystemMessageFormats.SuperviseData.getDefaultInstance()) { + superviseData_ = + akka.remote.SystemMessageFormats.SuperviseData.newBuilder(superviseData_).mergeFrom(value).buildPartial(); + } else { + superviseData_ = value; + } + onChanged(); + } else { + superviseDataBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000008; + return this; + } + /** + * optional .SuperviseData superviseData = 5; + */ + public Builder clearSuperviseData() { + if (superviseDataBuilder_ == null) { + superviseData_ = akka.remote.SystemMessageFormats.SuperviseData.getDefaultInstance(); + onChanged(); + } else { + superviseDataBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + /** + * optional .SuperviseData superviseData = 5; + */ + public akka.remote.SystemMessageFormats.SuperviseData.Builder getSuperviseDataBuilder() { + bitField0_ |= 0x00000008; + onChanged(); + return getSuperviseDataFieldBuilder().getBuilder(); + } + /** + * optional .SuperviseData superviseData = 5; + */ + public akka.remote.SystemMessageFormats.SuperviseDataOrBuilder getSuperviseDataOrBuilder() { + if (superviseDataBuilder_ != null) { + return superviseDataBuilder_.getMessageOrBuilder(); + } else { + return superviseData_; + } + } + /** + * optional .SuperviseData superviseData = 5; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.SystemMessageFormats.SuperviseData, akka.remote.SystemMessageFormats.SuperviseData.Builder, akka.remote.SystemMessageFormats.SuperviseDataOrBuilder> + getSuperviseDataFieldBuilder() { + if (superviseDataBuilder_ == null) { + superviseDataBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.SystemMessageFormats.SuperviseData, akka.remote.SystemMessageFormats.SuperviseData.Builder, akka.remote.SystemMessageFormats.SuperviseDataOrBuilder>( + superviseData_, + getParentForChildren(), + isClean()); + superviseData_ = null; + } + return superviseDataBuilder_; + } + + // optional .FailedData failedData = 6; + private akka.remote.SystemMessageFormats.FailedData failedData_ = akka.remote.SystemMessageFormats.FailedData.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.SystemMessageFormats.FailedData, akka.remote.SystemMessageFormats.FailedData.Builder, akka.remote.SystemMessageFormats.FailedDataOrBuilder> failedDataBuilder_; + /** + * optional .FailedData failedData = 6; + */ + public boolean hasFailedData() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + /** + * optional .FailedData failedData = 6; + */ + public akka.remote.SystemMessageFormats.FailedData getFailedData() { + if (failedDataBuilder_ == null) { + return failedData_; + } else { + return failedDataBuilder_.getMessage(); + } + } + /** + * optional .FailedData failedData = 6; + */ + public Builder setFailedData(akka.remote.SystemMessageFormats.FailedData value) { + if (failedDataBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + failedData_ = value; + onChanged(); + } else { + failedDataBuilder_.setMessage(value); + } + bitField0_ |= 0x00000010; + return this; + } + /** + * optional .FailedData failedData = 6; + */ + public Builder setFailedData( + akka.remote.SystemMessageFormats.FailedData.Builder builderForValue) { + if (failedDataBuilder_ == null) { + failedData_ = builderForValue.build(); + onChanged(); + } else { + failedDataBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000010; + return this; + } + /** + * optional .FailedData failedData = 6; + */ + public Builder mergeFailedData(akka.remote.SystemMessageFormats.FailedData value) { + if (failedDataBuilder_ == null) { + if (((bitField0_ & 0x00000010) == 0x00000010) && + failedData_ != akka.remote.SystemMessageFormats.FailedData.getDefaultInstance()) { + failedData_ = + akka.remote.SystemMessageFormats.FailedData.newBuilder(failedData_).mergeFrom(value).buildPartial(); + } else { + failedData_ = value; + } + onChanged(); + } else { + failedDataBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000010; + return this; + } + /** + * optional .FailedData failedData = 6; + */ + public Builder clearFailedData() { + if (failedDataBuilder_ == null) { + failedData_ = akka.remote.SystemMessageFormats.FailedData.getDefaultInstance(); + onChanged(); + } else { + failedDataBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + /** + * optional .FailedData failedData = 6; + */ + public akka.remote.SystemMessageFormats.FailedData.Builder getFailedDataBuilder() { + bitField0_ |= 0x00000010; + onChanged(); + return getFailedDataFieldBuilder().getBuilder(); + } + /** + * optional .FailedData failedData = 6; + */ + public akka.remote.SystemMessageFormats.FailedDataOrBuilder getFailedDataOrBuilder() { + if (failedDataBuilder_ != null) { + return failedDataBuilder_.getMessageOrBuilder(); + } else { + return failedData_; + } + } + /** + * optional .FailedData failedData = 6; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.SystemMessageFormats.FailedData, akka.remote.SystemMessageFormats.FailedData.Builder, akka.remote.SystemMessageFormats.FailedDataOrBuilder> + getFailedDataFieldBuilder() { + if (failedDataBuilder_ == null) { + failedDataBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.SystemMessageFormats.FailedData, akka.remote.SystemMessageFormats.FailedData.Builder, akka.remote.SystemMessageFormats.FailedDataOrBuilder>( + failedData_, + getParentForChildren(), + isClean()); + failedData_ = null; + } + return failedDataBuilder_; + } + + // optional .DeathWatchNotificationData dwNotificationData = 7; + private akka.remote.SystemMessageFormats.DeathWatchNotificationData dwNotificationData_ = akka.remote.SystemMessageFormats.DeathWatchNotificationData.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.SystemMessageFormats.DeathWatchNotificationData, akka.remote.SystemMessageFormats.DeathWatchNotificationData.Builder, akka.remote.SystemMessageFormats.DeathWatchNotificationDataOrBuilder> dwNotificationDataBuilder_; + /** + * optional .DeathWatchNotificationData dwNotificationData = 7; + */ + public boolean hasDwNotificationData() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + /** + * optional .DeathWatchNotificationData dwNotificationData = 7; + */ + public akka.remote.SystemMessageFormats.DeathWatchNotificationData getDwNotificationData() { + if (dwNotificationDataBuilder_ == null) { + return dwNotificationData_; + } else { + return dwNotificationDataBuilder_.getMessage(); + } + } + /** + * optional .DeathWatchNotificationData dwNotificationData = 7; + */ + public Builder setDwNotificationData(akka.remote.SystemMessageFormats.DeathWatchNotificationData value) { + if (dwNotificationDataBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + dwNotificationData_ = value; + onChanged(); + } else { + dwNotificationDataBuilder_.setMessage(value); + } + bitField0_ |= 0x00000020; + return this; + } + /** + * optional .DeathWatchNotificationData dwNotificationData = 7; + */ + public Builder setDwNotificationData( + akka.remote.SystemMessageFormats.DeathWatchNotificationData.Builder builderForValue) { + if (dwNotificationDataBuilder_ == null) { + dwNotificationData_ = builderForValue.build(); + onChanged(); + } else { + dwNotificationDataBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000020; + return this; + } + /** + * optional .DeathWatchNotificationData dwNotificationData = 7; + */ + public Builder mergeDwNotificationData(akka.remote.SystemMessageFormats.DeathWatchNotificationData value) { + if (dwNotificationDataBuilder_ == null) { + if (((bitField0_ & 0x00000020) == 0x00000020) && + dwNotificationData_ != akka.remote.SystemMessageFormats.DeathWatchNotificationData.getDefaultInstance()) { + dwNotificationData_ = + akka.remote.SystemMessageFormats.DeathWatchNotificationData.newBuilder(dwNotificationData_).mergeFrom(value).buildPartial(); + } else { + dwNotificationData_ = value; + } + onChanged(); + } else { + dwNotificationDataBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000020; + return this; + } + /** + * optional .DeathWatchNotificationData dwNotificationData = 7; + */ + public Builder clearDwNotificationData() { + if (dwNotificationDataBuilder_ == null) { + dwNotificationData_ = akka.remote.SystemMessageFormats.DeathWatchNotificationData.getDefaultInstance(); + onChanged(); + } else { + dwNotificationDataBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000020); + return this; + } + /** + * optional .DeathWatchNotificationData dwNotificationData = 7; + */ + public akka.remote.SystemMessageFormats.DeathWatchNotificationData.Builder getDwNotificationDataBuilder() { + bitField0_ |= 0x00000020; + onChanged(); + return getDwNotificationDataFieldBuilder().getBuilder(); + } + /** + * optional .DeathWatchNotificationData dwNotificationData = 7; + */ + public akka.remote.SystemMessageFormats.DeathWatchNotificationDataOrBuilder getDwNotificationDataOrBuilder() { + if (dwNotificationDataBuilder_ != null) { + return dwNotificationDataBuilder_.getMessageOrBuilder(); + } else { + return dwNotificationData_; + } + } + /** + * optional .DeathWatchNotificationData dwNotificationData = 7; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.SystemMessageFormats.DeathWatchNotificationData, akka.remote.SystemMessageFormats.DeathWatchNotificationData.Builder, akka.remote.SystemMessageFormats.DeathWatchNotificationDataOrBuilder> + getDwNotificationDataFieldBuilder() { + if (dwNotificationDataBuilder_ == null) { + dwNotificationDataBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.SystemMessageFormats.DeathWatchNotificationData, akka.remote.SystemMessageFormats.DeathWatchNotificationData.Builder, akka.remote.SystemMessageFormats.DeathWatchNotificationDataOrBuilder>( + dwNotificationData_, + getParentForChildren(), + isClean()); + dwNotificationData_ = null; + } + return dwNotificationDataBuilder_; + } + + // @@protoc_insertion_point(builder_scope:SystemMessage) + } + + static { + defaultInstance = new SystemMessage(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:SystemMessage) + } + + public interface WatchDataOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required .ActorRef watchee = 1; + /** + * required .ActorRef watchee = 1; + */ + boolean hasWatchee(); + /** + * required .ActorRef watchee = 1; + */ + akka.remote.ContainerFormats.ActorRef getWatchee(); + /** + * required .ActorRef watchee = 1; + */ + akka.remote.ContainerFormats.ActorRefOrBuilder getWatcheeOrBuilder(); + + // required .ActorRef watcher = 2; + /** + * required .ActorRef watcher = 2; + */ + boolean hasWatcher(); + /** + * required .ActorRef watcher = 2; + */ + akka.remote.ContainerFormats.ActorRef getWatcher(); + /** + * required .ActorRef watcher = 2; + */ + akka.remote.ContainerFormats.ActorRefOrBuilder getWatcherOrBuilder(); + } + /** + * Protobuf type {@code WatchData} + */ + public static final class WatchData extends + akka.protobuf.GeneratedMessage + implements WatchDataOrBuilder { + // Use WatchData.newBuilder() to construct. + private WatchData(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private WatchData(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final WatchData defaultInstance; + public static WatchData getDefaultInstance() { + return defaultInstance; + } + + public WatchData getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private WatchData( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + akka.remote.ContainerFormats.ActorRef.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = watchee_.toBuilder(); + } + watchee_ = input.readMessage(akka.remote.ContainerFormats.ActorRef.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(watchee_); + watchee_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + case 18: { + akka.remote.ContainerFormats.ActorRef.Builder subBuilder = null; + if (((bitField0_ & 0x00000002) == 0x00000002)) { + subBuilder = watcher_.toBuilder(); + } + watcher_ = input.readMessage(akka.remote.ContainerFormats.ActorRef.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(watcher_); + watcher_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000002; + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.SystemMessageFormats.internal_static_WatchData_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.SystemMessageFormats.internal_static_WatchData_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.SystemMessageFormats.WatchData.class, akka.remote.SystemMessageFormats.WatchData.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public WatchData parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new WatchData(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required .ActorRef watchee = 1; + public static final int WATCHEE_FIELD_NUMBER = 1; + private akka.remote.ContainerFormats.ActorRef watchee_; + /** + * required .ActorRef watchee = 1; + */ + public boolean hasWatchee() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .ActorRef watchee = 1; + */ + public akka.remote.ContainerFormats.ActorRef getWatchee() { + return watchee_; + } + /** + * required .ActorRef watchee = 1; + */ + public akka.remote.ContainerFormats.ActorRefOrBuilder getWatcheeOrBuilder() { + return watchee_; + } + + // required .ActorRef watcher = 2; + public static final int WATCHER_FIELD_NUMBER = 2; + private akka.remote.ContainerFormats.ActorRef watcher_; + /** + * required .ActorRef watcher = 2; + */ + public boolean hasWatcher() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required .ActorRef watcher = 2; + */ + public akka.remote.ContainerFormats.ActorRef getWatcher() { + return watcher_; + } + /** + * required .ActorRef watcher = 2; + */ + public akka.remote.ContainerFormats.ActorRefOrBuilder getWatcherOrBuilder() { + return watcher_; + } + + private void initFields() { + watchee_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + watcher_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasWatchee()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasWatcher()) { + memoizedIsInitialized = 0; + return false; + } + if (!getWatchee().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + if (!getWatcher().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, watchee_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, watcher_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(1, watchee_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(2, watcher_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.SystemMessageFormats.WatchData parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.SystemMessageFormats.WatchData parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.WatchData parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.SystemMessageFormats.WatchData parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.WatchData parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.SystemMessageFormats.WatchData parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.WatchData parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.SystemMessageFormats.WatchData parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.WatchData parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.SystemMessageFormats.WatchData parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.SystemMessageFormats.WatchData prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code WatchData} + */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.SystemMessageFormats.WatchDataOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.SystemMessageFormats.internal_static_WatchData_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.SystemMessageFormats.internal_static_WatchData_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.SystemMessageFormats.WatchData.class, akka.remote.SystemMessageFormats.WatchData.Builder.class); + } + + // Construct using akka.remote.SystemMessageFormats.WatchData.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getWatcheeFieldBuilder(); + getWatcherFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (watcheeBuilder_ == null) { + watchee_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + } else { + watcheeBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + if (watcherBuilder_ == null) { + watcher_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + } else { + watcherBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.SystemMessageFormats.internal_static_WatchData_descriptor; + } + + public akka.remote.SystemMessageFormats.WatchData getDefaultInstanceForType() { + return akka.remote.SystemMessageFormats.WatchData.getDefaultInstance(); + } + + public akka.remote.SystemMessageFormats.WatchData build() { + akka.remote.SystemMessageFormats.WatchData result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.SystemMessageFormats.WatchData buildPartial() { + akka.remote.SystemMessageFormats.WatchData result = new akka.remote.SystemMessageFormats.WatchData(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (watcheeBuilder_ == null) { + result.watchee_ = watchee_; + } else { + result.watchee_ = watcheeBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (watcherBuilder_ == null) { + result.watcher_ = watcher_; + } else { + result.watcher_ = watcherBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.SystemMessageFormats.WatchData) { + return mergeFrom((akka.remote.SystemMessageFormats.WatchData)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.SystemMessageFormats.WatchData other) { + if (other == akka.remote.SystemMessageFormats.WatchData.getDefaultInstance()) return this; + if (other.hasWatchee()) { + mergeWatchee(other.getWatchee()); + } + if (other.hasWatcher()) { + mergeWatcher(other.getWatcher()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasWatchee()) { + + return false; + } + if (!hasWatcher()) { + + return false; + } + if (!getWatchee().isInitialized()) { + + return false; + } + if (!getWatcher().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.SystemMessageFormats.WatchData parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.SystemMessageFormats.WatchData) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required .ActorRef watchee = 1; + private akka.remote.ContainerFormats.ActorRef watchee_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.ActorRef, akka.remote.ContainerFormats.ActorRef.Builder, akka.remote.ContainerFormats.ActorRefOrBuilder> watcheeBuilder_; + /** + * required .ActorRef watchee = 1; + */ + public boolean hasWatchee() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .ActorRef watchee = 1; + */ + public akka.remote.ContainerFormats.ActorRef getWatchee() { + if (watcheeBuilder_ == null) { + return watchee_; + } else { + return watcheeBuilder_.getMessage(); + } + } + /** + * required .ActorRef watchee = 1; + */ + public Builder setWatchee(akka.remote.ContainerFormats.ActorRef value) { + if (watcheeBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + watchee_ = value; + onChanged(); + } else { + watcheeBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .ActorRef watchee = 1; + */ + public Builder setWatchee( + akka.remote.ContainerFormats.ActorRef.Builder builderForValue) { + if (watcheeBuilder_ == null) { + watchee_ = builderForValue.build(); + onChanged(); + } else { + watcheeBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .ActorRef watchee = 1; + */ + public Builder mergeWatchee(akka.remote.ContainerFormats.ActorRef value) { + if (watcheeBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + watchee_ != akka.remote.ContainerFormats.ActorRef.getDefaultInstance()) { + watchee_ = + akka.remote.ContainerFormats.ActorRef.newBuilder(watchee_).mergeFrom(value).buildPartial(); + } else { + watchee_ = value; + } + onChanged(); + } else { + watcheeBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .ActorRef watchee = 1; + */ + public Builder clearWatchee() { + if (watcheeBuilder_ == null) { + watchee_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + onChanged(); + } else { + watcheeBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * required .ActorRef watchee = 1; + */ + public akka.remote.ContainerFormats.ActorRef.Builder getWatcheeBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getWatcheeFieldBuilder().getBuilder(); + } + /** + * required .ActorRef watchee = 1; + */ + public akka.remote.ContainerFormats.ActorRefOrBuilder getWatcheeOrBuilder() { + if (watcheeBuilder_ != null) { + return watcheeBuilder_.getMessageOrBuilder(); + } else { + return watchee_; + } + } + /** + * required .ActorRef watchee = 1; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.ActorRef, akka.remote.ContainerFormats.ActorRef.Builder, akka.remote.ContainerFormats.ActorRefOrBuilder> + getWatcheeFieldBuilder() { + if (watcheeBuilder_ == null) { + watcheeBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.ActorRef, akka.remote.ContainerFormats.ActorRef.Builder, akka.remote.ContainerFormats.ActorRefOrBuilder>( + watchee_, + getParentForChildren(), + isClean()); + watchee_ = null; + } + return watcheeBuilder_; + } + + // required .ActorRef watcher = 2; + private akka.remote.ContainerFormats.ActorRef watcher_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.ActorRef, akka.remote.ContainerFormats.ActorRef.Builder, akka.remote.ContainerFormats.ActorRefOrBuilder> watcherBuilder_; + /** + * required .ActorRef watcher = 2; + */ + public boolean hasWatcher() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required .ActorRef watcher = 2; + */ + public akka.remote.ContainerFormats.ActorRef getWatcher() { + if (watcherBuilder_ == null) { + return watcher_; + } else { + return watcherBuilder_.getMessage(); + } + } + /** + * required .ActorRef watcher = 2; + */ + public Builder setWatcher(akka.remote.ContainerFormats.ActorRef value) { + if (watcherBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + watcher_ = value; + onChanged(); + } else { + watcherBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .ActorRef watcher = 2; + */ + public Builder setWatcher( + akka.remote.ContainerFormats.ActorRef.Builder builderForValue) { + if (watcherBuilder_ == null) { + watcher_ = builderForValue.build(); + onChanged(); + } else { + watcherBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .ActorRef watcher = 2; + */ + public Builder mergeWatcher(akka.remote.ContainerFormats.ActorRef value) { + if (watcherBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + watcher_ != akka.remote.ContainerFormats.ActorRef.getDefaultInstance()) { + watcher_ = + akka.remote.ContainerFormats.ActorRef.newBuilder(watcher_).mergeFrom(value).buildPartial(); + } else { + watcher_ = value; + } + onChanged(); + } else { + watcherBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** + * required .ActorRef watcher = 2; + */ + public Builder clearWatcher() { + if (watcherBuilder_ == null) { + watcher_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + onChanged(); + } else { + watcherBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + /** + * required .ActorRef watcher = 2; + */ + public akka.remote.ContainerFormats.ActorRef.Builder getWatcherBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getWatcherFieldBuilder().getBuilder(); + } + /** + * required .ActorRef watcher = 2; + */ + public akka.remote.ContainerFormats.ActorRefOrBuilder getWatcherOrBuilder() { + if (watcherBuilder_ != null) { + return watcherBuilder_.getMessageOrBuilder(); + } else { + return watcher_; + } + } + /** + * required .ActorRef watcher = 2; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.ActorRef, akka.remote.ContainerFormats.ActorRef.Builder, akka.remote.ContainerFormats.ActorRefOrBuilder> + getWatcherFieldBuilder() { + if (watcherBuilder_ == null) { + watcherBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.ActorRef, akka.remote.ContainerFormats.ActorRef.Builder, akka.remote.ContainerFormats.ActorRefOrBuilder>( + watcher_, + getParentForChildren(), + isClean()); + watcher_ = null; + } + return watcherBuilder_; + } + + // @@protoc_insertion_point(builder_scope:WatchData) + } + + static { + defaultInstance = new WatchData(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:WatchData) + } + + public interface SuperviseDataOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required .ActorRef child = 1; + /** + * required .ActorRef child = 1; + */ + boolean hasChild(); + /** + * required .ActorRef child = 1; + */ + akka.remote.ContainerFormats.ActorRef getChild(); + /** + * required .ActorRef child = 1; + */ + akka.remote.ContainerFormats.ActorRefOrBuilder getChildOrBuilder(); + + // required bool async = 2; + /** + * required bool async = 2; + */ + boolean hasAsync(); + /** + * required bool async = 2; + */ + boolean getAsync(); + } + /** + * Protobuf type {@code SuperviseData} + */ + public static final class SuperviseData extends + akka.protobuf.GeneratedMessage + implements SuperviseDataOrBuilder { + // Use SuperviseData.newBuilder() to construct. + private SuperviseData(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private SuperviseData(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final SuperviseData defaultInstance; + public static SuperviseData getDefaultInstance() { + return defaultInstance; + } + + public SuperviseData getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private SuperviseData( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + akka.remote.ContainerFormats.ActorRef.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = child_.toBuilder(); + } + child_ = input.readMessage(akka.remote.ContainerFormats.ActorRef.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(child_); + child_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + case 16: { + bitField0_ |= 0x00000002; + async_ = input.readBool(); + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.SystemMessageFormats.internal_static_SuperviseData_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.SystemMessageFormats.internal_static_SuperviseData_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.SystemMessageFormats.SuperviseData.class, akka.remote.SystemMessageFormats.SuperviseData.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public SuperviseData parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new SuperviseData(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required .ActorRef child = 1; + public static final int CHILD_FIELD_NUMBER = 1; + private akka.remote.ContainerFormats.ActorRef child_; + /** + * required .ActorRef child = 1; + */ + public boolean hasChild() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .ActorRef child = 1; + */ + public akka.remote.ContainerFormats.ActorRef getChild() { + return child_; + } + /** + * required .ActorRef child = 1; + */ + public akka.remote.ContainerFormats.ActorRefOrBuilder getChildOrBuilder() { + return child_; + } + + // required bool async = 2; + public static final int ASYNC_FIELD_NUMBER = 2; + private boolean async_; + /** + * required bool async = 2; + */ + public boolean hasAsync() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required bool async = 2; + */ + public boolean getAsync() { + return async_; + } + + private void initFields() { + child_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + async_ = false; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasChild()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasAsync()) { + memoizedIsInitialized = 0; + return false; + } + if (!getChild().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, child_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBool(2, async_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(1, child_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += akka.protobuf.CodedOutputStream + .computeBoolSize(2, async_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.SystemMessageFormats.SuperviseData parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.SystemMessageFormats.SuperviseData parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.SuperviseData parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.SystemMessageFormats.SuperviseData parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.SuperviseData parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.SystemMessageFormats.SuperviseData parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.SuperviseData parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.SystemMessageFormats.SuperviseData parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.SuperviseData parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.SystemMessageFormats.SuperviseData parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.SystemMessageFormats.SuperviseData prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code SuperviseData} + */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.SystemMessageFormats.SuperviseDataOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.SystemMessageFormats.internal_static_SuperviseData_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.SystemMessageFormats.internal_static_SuperviseData_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.SystemMessageFormats.SuperviseData.class, akka.remote.SystemMessageFormats.SuperviseData.Builder.class); + } + + // Construct using akka.remote.SystemMessageFormats.SuperviseData.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getChildFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (childBuilder_ == null) { + child_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + } else { + childBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + async_ = false; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.SystemMessageFormats.internal_static_SuperviseData_descriptor; + } + + public akka.remote.SystemMessageFormats.SuperviseData getDefaultInstanceForType() { + return akka.remote.SystemMessageFormats.SuperviseData.getDefaultInstance(); + } + + public akka.remote.SystemMessageFormats.SuperviseData build() { + akka.remote.SystemMessageFormats.SuperviseData result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.SystemMessageFormats.SuperviseData buildPartial() { + akka.remote.SystemMessageFormats.SuperviseData result = new akka.remote.SystemMessageFormats.SuperviseData(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (childBuilder_ == null) { + result.child_ = child_; + } else { + result.child_ = childBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.async_ = async_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.SystemMessageFormats.SuperviseData) { + return mergeFrom((akka.remote.SystemMessageFormats.SuperviseData)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.SystemMessageFormats.SuperviseData other) { + if (other == akka.remote.SystemMessageFormats.SuperviseData.getDefaultInstance()) return this; + if (other.hasChild()) { + mergeChild(other.getChild()); + } + if (other.hasAsync()) { + setAsync(other.getAsync()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasChild()) { + + return false; + } + if (!hasAsync()) { + + return false; + } + if (!getChild().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.SystemMessageFormats.SuperviseData parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.SystemMessageFormats.SuperviseData) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required .ActorRef child = 1; + private akka.remote.ContainerFormats.ActorRef child_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.ActorRef, akka.remote.ContainerFormats.ActorRef.Builder, akka.remote.ContainerFormats.ActorRefOrBuilder> childBuilder_; + /** + * required .ActorRef child = 1; + */ + public boolean hasChild() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .ActorRef child = 1; + */ + public akka.remote.ContainerFormats.ActorRef getChild() { + if (childBuilder_ == null) { + return child_; + } else { + return childBuilder_.getMessage(); + } + } + /** + * required .ActorRef child = 1; + */ + public Builder setChild(akka.remote.ContainerFormats.ActorRef value) { + if (childBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + child_ = value; + onChanged(); + } else { + childBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .ActorRef child = 1; + */ + public Builder setChild( + akka.remote.ContainerFormats.ActorRef.Builder builderForValue) { + if (childBuilder_ == null) { + child_ = builderForValue.build(); + onChanged(); + } else { + childBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .ActorRef child = 1; + */ + public Builder mergeChild(akka.remote.ContainerFormats.ActorRef value) { + if (childBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + child_ != akka.remote.ContainerFormats.ActorRef.getDefaultInstance()) { + child_ = + akka.remote.ContainerFormats.ActorRef.newBuilder(child_).mergeFrom(value).buildPartial(); + } else { + child_ = value; + } + onChanged(); + } else { + childBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .ActorRef child = 1; + */ + public Builder clearChild() { + if (childBuilder_ == null) { + child_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + onChanged(); + } else { + childBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * required .ActorRef child = 1; + */ + public akka.remote.ContainerFormats.ActorRef.Builder getChildBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getChildFieldBuilder().getBuilder(); + } + /** + * required .ActorRef child = 1; + */ + public akka.remote.ContainerFormats.ActorRefOrBuilder getChildOrBuilder() { + if (childBuilder_ != null) { + return childBuilder_.getMessageOrBuilder(); + } else { + return child_; + } + } + /** + * required .ActorRef child = 1; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.ActorRef, akka.remote.ContainerFormats.ActorRef.Builder, akka.remote.ContainerFormats.ActorRefOrBuilder> + getChildFieldBuilder() { + if (childBuilder_ == null) { + childBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.ActorRef, akka.remote.ContainerFormats.ActorRef.Builder, akka.remote.ContainerFormats.ActorRefOrBuilder>( + child_, + getParentForChildren(), + isClean()); + child_ = null; + } + return childBuilder_; + } + + // required bool async = 2; + private boolean async_ ; + /** + * required bool async = 2; + */ + public boolean hasAsync() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required bool async = 2; + */ + public boolean getAsync() { + return async_; + } + /** + * required bool async = 2; + */ + public Builder setAsync(boolean value) { + bitField0_ |= 0x00000002; + async_ = value; + onChanged(); + return this; + } + /** + * required bool async = 2; + */ + public Builder clearAsync() { + bitField0_ = (bitField0_ & ~0x00000002); + async_ = false; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:SuperviseData) + } + + static { + defaultInstance = new SuperviseData(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:SuperviseData) + } + + public interface FailedDataOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required .ActorRef child = 1; + /** + * required .ActorRef child = 1; + */ + boolean hasChild(); + /** + * required .ActorRef child = 1; + */ + akka.remote.ContainerFormats.ActorRef getChild(); + /** + * required .ActorRef child = 1; + */ + akka.remote.ContainerFormats.ActorRefOrBuilder getChildOrBuilder(); + + // required uint64 uid = 2; + /** + * required uint64 uid = 2; + */ + boolean hasUid(); + /** + * required uint64 uid = 2; + */ + long getUid(); + } + /** + * Protobuf type {@code FailedData} + */ + public static final class FailedData extends + akka.protobuf.GeneratedMessage + implements FailedDataOrBuilder { + // Use FailedData.newBuilder() to construct. + private FailedData(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private FailedData(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final FailedData defaultInstance; + public static FailedData getDefaultInstance() { + return defaultInstance; + } + + public FailedData getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private FailedData( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + akka.remote.ContainerFormats.ActorRef.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = child_.toBuilder(); + } + child_ = input.readMessage(akka.remote.ContainerFormats.ActorRef.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(child_); + child_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + case 16: { + bitField0_ |= 0x00000002; + uid_ = input.readUInt64(); + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.SystemMessageFormats.internal_static_FailedData_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.SystemMessageFormats.internal_static_FailedData_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.SystemMessageFormats.FailedData.class, akka.remote.SystemMessageFormats.FailedData.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public FailedData parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new FailedData(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required .ActorRef child = 1; + public static final int CHILD_FIELD_NUMBER = 1; + private akka.remote.ContainerFormats.ActorRef child_; + /** + * required .ActorRef child = 1; + */ + public boolean hasChild() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .ActorRef child = 1; + */ + public akka.remote.ContainerFormats.ActorRef getChild() { + return child_; + } + /** + * required .ActorRef child = 1; + */ + public akka.remote.ContainerFormats.ActorRefOrBuilder getChildOrBuilder() { + return child_; + } + + // required uint64 uid = 2; + public static final int UID_FIELD_NUMBER = 2; + private long uid_; + /** + * required uint64 uid = 2; + */ + public boolean hasUid() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required uint64 uid = 2; + */ + public long getUid() { + return uid_; + } + + private void initFields() { + child_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + uid_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasChild()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasUid()) { + memoizedIsInitialized = 0; + return false; + } + if (!getChild().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, child_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, uid_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(1, child_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += akka.protobuf.CodedOutputStream + .computeUInt64Size(2, uid_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.SystemMessageFormats.FailedData parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.SystemMessageFormats.FailedData parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.FailedData parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.SystemMessageFormats.FailedData parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.FailedData parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.SystemMessageFormats.FailedData parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.FailedData parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.SystemMessageFormats.FailedData parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.FailedData parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.SystemMessageFormats.FailedData parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.SystemMessageFormats.FailedData prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code FailedData} + */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.SystemMessageFormats.FailedDataOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.SystemMessageFormats.internal_static_FailedData_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.SystemMessageFormats.internal_static_FailedData_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.SystemMessageFormats.FailedData.class, akka.remote.SystemMessageFormats.FailedData.Builder.class); + } + + // Construct using akka.remote.SystemMessageFormats.FailedData.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getChildFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (childBuilder_ == null) { + child_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + } else { + childBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + uid_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.SystemMessageFormats.internal_static_FailedData_descriptor; + } + + public akka.remote.SystemMessageFormats.FailedData getDefaultInstanceForType() { + return akka.remote.SystemMessageFormats.FailedData.getDefaultInstance(); + } + + public akka.remote.SystemMessageFormats.FailedData build() { + akka.remote.SystemMessageFormats.FailedData result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.SystemMessageFormats.FailedData buildPartial() { + akka.remote.SystemMessageFormats.FailedData result = new akka.remote.SystemMessageFormats.FailedData(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (childBuilder_ == null) { + result.child_ = child_; + } else { + result.child_ = childBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.uid_ = uid_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.SystemMessageFormats.FailedData) { + return mergeFrom((akka.remote.SystemMessageFormats.FailedData)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.SystemMessageFormats.FailedData other) { + if (other == akka.remote.SystemMessageFormats.FailedData.getDefaultInstance()) return this; + if (other.hasChild()) { + mergeChild(other.getChild()); + } + if (other.hasUid()) { + setUid(other.getUid()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasChild()) { + + return false; + } + if (!hasUid()) { + + return false; + } + if (!getChild().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.SystemMessageFormats.FailedData parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.SystemMessageFormats.FailedData) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required .ActorRef child = 1; + private akka.remote.ContainerFormats.ActorRef child_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.ActorRef, akka.remote.ContainerFormats.ActorRef.Builder, akka.remote.ContainerFormats.ActorRefOrBuilder> childBuilder_; + /** + * required .ActorRef child = 1; + */ + public boolean hasChild() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .ActorRef child = 1; + */ + public akka.remote.ContainerFormats.ActorRef getChild() { + if (childBuilder_ == null) { + return child_; + } else { + return childBuilder_.getMessage(); + } + } + /** + * required .ActorRef child = 1; + */ + public Builder setChild(akka.remote.ContainerFormats.ActorRef value) { + if (childBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + child_ = value; + onChanged(); + } else { + childBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .ActorRef child = 1; + */ + public Builder setChild( + akka.remote.ContainerFormats.ActorRef.Builder builderForValue) { + if (childBuilder_ == null) { + child_ = builderForValue.build(); + onChanged(); + } else { + childBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .ActorRef child = 1; + */ + public Builder mergeChild(akka.remote.ContainerFormats.ActorRef value) { + if (childBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + child_ != akka.remote.ContainerFormats.ActorRef.getDefaultInstance()) { + child_ = + akka.remote.ContainerFormats.ActorRef.newBuilder(child_).mergeFrom(value).buildPartial(); + } else { + child_ = value; + } + onChanged(); + } else { + childBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .ActorRef child = 1; + */ + public Builder clearChild() { + if (childBuilder_ == null) { + child_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + onChanged(); + } else { + childBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * required .ActorRef child = 1; + */ + public akka.remote.ContainerFormats.ActorRef.Builder getChildBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getChildFieldBuilder().getBuilder(); + } + /** + * required .ActorRef child = 1; + */ + public akka.remote.ContainerFormats.ActorRefOrBuilder getChildOrBuilder() { + if (childBuilder_ != null) { + return childBuilder_.getMessageOrBuilder(); + } else { + return child_; + } + } + /** + * required .ActorRef child = 1; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.ActorRef, akka.remote.ContainerFormats.ActorRef.Builder, akka.remote.ContainerFormats.ActorRefOrBuilder> + getChildFieldBuilder() { + if (childBuilder_ == null) { + childBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.ActorRef, akka.remote.ContainerFormats.ActorRef.Builder, akka.remote.ContainerFormats.ActorRefOrBuilder>( + child_, + getParentForChildren(), + isClean()); + child_ = null; + } + return childBuilder_; + } + + // required uint64 uid = 2; + private long uid_ ; + /** + * required uint64 uid = 2; + */ + public boolean hasUid() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required uint64 uid = 2; + */ + public long getUid() { + return uid_; + } + /** + * required uint64 uid = 2; + */ + public Builder setUid(long value) { + bitField0_ |= 0x00000002; + uid_ = value; + onChanged(); + return this; + } + /** + * required uint64 uid = 2; + */ + public Builder clearUid() { + bitField0_ = (bitField0_ & ~0x00000002); + uid_ = 0L; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:FailedData) + } + + static { + defaultInstance = new FailedData(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:FailedData) + } + + public interface DeathWatchNotificationDataOrBuilder + extends akka.protobuf.MessageOrBuilder { + + // required .ActorRef actor = 1; + /** + * required .ActorRef actor = 1; + */ + boolean hasActor(); + /** + * required .ActorRef actor = 1; + */ + akka.remote.ContainerFormats.ActorRef getActor(); + /** + * required .ActorRef actor = 1; + */ + akka.remote.ContainerFormats.ActorRefOrBuilder getActorOrBuilder(); + + // required bool existenceConfirmed = 2; + /** + * required bool existenceConfirmed = 2; + */ + boolean hasExistenceConfirmed(); + /** + * required bool existenceConfirmed = 2; + */ + boolean getExistenceConfirmed(); + + // required bool addressTerminated = 3; + /** + * required bool addressTerminated = 3; + */ + boolean hasAddressTerminated(); + /** + * required bool addressTerminated = 3; + */ + boolean getAddressTerminated(); + } + /** + * Protobuf type {@code DeathWatchNotificationData} + */ + public static final class DeathWatchNotificationData extends + akka.protobuf.GeneratedMessage + implements DeathWatchNotificationDataOrBuilder { + // Use DeathWatchNotificationData.newBuilder() to construct. + private DeathWatchNotificationData(akka.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private DeathWatchNotificationData(boolean noInit) { this.unknownFields = akka.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final DeathWatchNotificationData defaultInstance; + public static DeathWatchNotificationData getDefaultInstance() { + return defaultInstance; + } + + public DeathWatchNotificationData getDefaultInstanceForType() { + return defaultInstance; + } + + private final akka.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final akka.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private DeathWatchNotificationData( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + akka.protobuf.UnknownFieldSet.Builder unknownFields = + akka.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + akka.remote.ContainerFormats.ActorRef.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = actor_.toBuilder(); + } + actor_ = input.readMessage(akka.remote.ContainerFormats.ActorRef.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(actor_); + actor_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + case 16: { + bitField0_ |= 0x00000002; + existenceConfirmed_ = input.readBool(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + addressTerminated_ = input.readBool(); + break; + } + } + } + } catch (akka.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.SystemMessageFormats.internal_static_DeathWatchNotificationData_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.SystemMessageFormats.internal_static_DeathWatchNotificationData_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.SystemMessageFormats.DeathWatchNotificationData.class, akka.remote.SystemMessageFormats.DeathWatchNotificationData.Builder.class); + } + + public static akka.protobuf.Parser PARSER = + new akka.protobuf.AbstractParser() { + public DeathWatchNotificationData parsePartialFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return new DeathWatchNotificationData(input, extensionRegistry); + } + }; + + @java.lang.Override + public akka.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required .ActorRef actor = 1; + public static final int ACTOR_FIELD_NUMBER = 1; + private akka.remote.ContainerFormats.ActorRef actor_; + /** + * required .ActorRef actor = 1; + */ + public boolean hasActor() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .ActorRef actor = 1; + */ + public akka.remote.ContainerFormats.ActorRef getActor() { + return actor_; + } + /** + * required .ActorRef actor = 1; + */ + public akka.remote.ContainerFormats.ActorRefOrBuilder getActorOrBuilder() { + return actor_; + } + + // required bool existenceConfirmed = 2; + public static final int EXISTENCECONFIRMED_FIELD_NUMBER = 2; + private boolean existenceConfirmed_; + /** + * required bool existenceConfirmed = 2; + */ + public boolean hasExistenceConfirmed() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required bool existenceConfirmed = 2; + */ + public boolean getExistenceConfirmed() { + return existenceConfirmed_; + } + + // required bool addressTerminated = 3; + public static final int ADDRESSTERMINATED_FIELD_NUMBER = 3; + private boolean addressTerminated_; + /** + * required bool addressTerminated = 3; + */ + public boolean hasAddressTerminated() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required bool addressTerminated = 3; + */ + public boolean getAddressTerminated() { + return addressTerminated_; + } + + private void initFields() { + actor_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + existenceConfirmed_ = false; + addressTerminated_ = false; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasActor()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasExistenceConfirmed()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasAddressTerminated()) { + memoizedIsInitialized = 0; + return false; + } + if (!getActor().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(akka.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, actor_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBool(2, existenceConfirmed_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBool(3, addressTerminated_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += akka.protobuf.CodedOutputStream + .computeMessageSize(1, actor_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += akka.protobuf.CodedOutputStream + .computeBoolSize(2, existenceConfirmed_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += akka.protobuf.CodedOutputStream + .computeBoolSize(3, addressTerminated_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.SystemMessageFormats.DeathWatchNotificationData parseFrom( + akka.protobuf.ByteString data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.SystemMessageFormats.DeathWatchNotificationData parseFrom( + akka.protobuf.ByteString data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.DeathWatchNotificationData parseFrom(byte[] data) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static akka.remote.SystemMessageFormats.DeathWatchNotificationData parseFrom( + byte[] data, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws akka.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.DeathWatchNotificationData parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.SystemMessageFormats.DeathWatchNotificationData parseFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.DeathWatchNotificationData parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static akka.remote.SystemMessageFormats.DeathWatchNotificationData parseDelimitedFrom( + java.io.InputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static akka.remote.SystemMessageFormats.DeathWatchNotificationData parseFrom( + akka.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static akka.remote.SystemMessageFormats.DeathWatchNotificationData parseFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.SystemMessageFormats.DeathWatchNotificationData prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code DeathWatchNotificationData} + */ + public static final class Builder extends + akka.protobuf.GeneratedMessage.Builder + implements akka.remote.SystemMessageFormats.DeathWatchNotificationDataOrBuilder { + public static final akka.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.SystemMessageFormats.internal_static_DeathWatchNotificationData_descriptor; + } + + protected akka.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.SystemMessageFormats.internal_static_DeathWatchNotificationData_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.remote.SystemMessageFormats.DeathWatchNotificationData.class, akka.remote.SystemMessageFormats.DeathWatchNotificationData.Builder.class); + } + + // Construct using akka.remote.SystemMessageFormats.DeathWatchNotificationData.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + akka.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (akka.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getActorFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (actorBuilder_ == null) { + actor_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + } else { + actorBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + existenceConfirmed_ = false; + bitField0_ = (bitField0_ & ~0x00000002); + addressTerminated_ = false; + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public akka.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.SystemMessageFormats.internal_static_DeathWatchNotificationData_descriptor; + } + + public akka.remote.SystemMessageFormats.DeathWatchNotificationData getDefaultInstanceForType() { + return akka.remote.SystemMessageFormats.DeathWatchNotificationData.getDefaultInstance(); + } + + public akka.remote.SystemMessageFormats.DeathWatchNotificationData build() { + akka.remote.SystemMessageFormats.DeathWatchNotificationData result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public akka.remote.SystemMessageFormats.DeathWatchNotificationData buildPartial() { + akka.remote.SystemMessageFormats.DeathWatchNotificationData result = new akka.remote.SystemMessageFormats.DeathWatchNotificationData(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (actorBuilder_ == null) { + result.actor_ = actor_; + } else { + result.actor_ = actorBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.existenceConfirmed_ = existenceConfirmed_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.addressTerminated_ = addressTerminated_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(akka.protobuf.Message other) { + if (other instanceof akka.remote.SystemMessageFormats.DeathWatchNotificationData) { + return mergeFrom((akka.remote.SystemMessageFormats.DeathWatchNotificationData)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.SystemMessageFormats.DeathWatchNotificationData other) { + if (other == akka.remote.SystemMessageFormats.DeathWatchNotificationData.getDefaultInstance()) return this; + if (other.hasActor()) { + mergeActor(other.getActor()); + } + if (other.hasExistenceConfirmed()) { + setExistenceConfirmed(other.getExistenceConfirmed()); + } + if (other.hasAddressTerminated()) { + setAddressTerminated(other.getAddressTerminated()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasActor()) { + + return false; + } + if (!hasExistenceConfirmed()) { + + return false; + } + if (!hasAddressTerminated()) { + + return false; + } + if (!getActor().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + akka.protobuf.CodedInputStream input, + akka.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.remote.SystemMessageFormats.DeathWatchNotificationData parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (akka.remote.SystemMessageFormats.DeathWatchNotificationData) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required .ActorRef actor = 1; + private akka.remote.ContainerFormats.ActorRef actor_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + private akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.ActorRef, akka.remote.ContainerFormats.ActorRef.Builder, akka.remote.ContainerFormats.ActorRefOrBuilder> actorBuilder_; + /** + * required .ActorRef actor = 1; + */ + public boolean hasActor() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required .ActorRef actor = 1; + */ + public akka.remote.ContainerFormats.ActorRef getActor() { + if (actorBuilder_ == null) { + return actor_; + } else { + return actorBuilder_.getMessage(); + } + } + /** + * required .ActorRef actor = 1; + */ + public Builder setActor(akka.remote.ContainerFormats.ActorRef value) { + if (actorBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + actor_ = value; + onChanged(); + } else { + actorBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .ActorRef actor = 1; + */ + public Builder setActor( + akka.remote.ContainerFormats.ActorRef.Builder builderForValue) { + if (actorBuilder_ == null) { + actor_ = builderForValue.build(); + onChanged(); + } else { + actorBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .ActorRef actor = 1; + */ + public Builder mergeActor(akka.remote.ContainerFormats.ActorRef value) { + if (actorBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + actor_ != akka.remote.ContainerFormats.ActorRef.getDefaultInstance()) { + actor_ = + akka.remote.ContainerFormats.ActorRef.newBuilder(actor_).mergeFrom(value).buildPartial(); + } else { + actor_ = value; + } + onChanged(); + } else { + actorBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * required .ActorRef actor = 1; + */ + public Builder clearActor() { + if (actorBuilder_ == null) { + actor_ = akka.remote.ContainerFormats.ActorRef.getDefaultInstance(); + onChanged(); + } else { + actorBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * required .ActorRef actor = 1; + */ + public akka.remote.ContainerFormats.ActorRef.Builder getActorBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getActorFieldBuilder().getBuilder(); + } + /** + * required .ActorRef actor = 1; + */ + public akka.remote.ContainerFormats.ActorRefOrBuilder getActorOrBuilder() { + if (actorBuilder_ != null) { + return actorBuilder_.getMessageOrBuilder(); + } else { + return actor_; + } + } + /** + * required .ActorRef actor = 1; + */ + private akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.ActorRef, akka.remote.ContainerFormats.ActorRef.Builder, akka.remote.ContainerFormats.ActorRefOrBuilder> + getActorFieldBuilder() { + if (actorBuilder_ == null) { + actorBuilder_ = new akka.protobuf.SingleFieldBuilder< + akka.remote.ContainerFormats.ActorRef, akka.remote.ContainerFormats.ActorRef.Builder, akka.remote.ContainerFormats.ActorRefOrBuilder>( + actor_, + getParentForChildren(), + isClean()); + actor_ = null; + } + return actorBuilder_; + } + + // required bool existenceConfirmed = 2; + private boolean existenceConfirmed_ ; + /** + * required bool existenceConfirmed = 2; + */ + public boolean hasExistenceConfirmed() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required bool existenceConfirmed = 2; + */ + public boolean getExistenceConfirmed() { + return existenceConfirmed_; + } + /** + * required bool existenceConfirmed = 2; + */ + public Builder setExistenceConfirmed(boolean value) { + bitField0_ |= 0x00000002; + existenceConfirmed_ = value; + onChanged(); + return this; + } + /** + * required bool existenceConfirmed = 2; + */ + public Builder clearExistenceConfirmed() { + bitField0_ = (bitField0_ & ~0x00000002); + existenceConfirmed_ = false; + onChanged(); + return this; + } + + // required bool addressTerminated = 3; + private boolean addressTerminated_ ; + /** + * required bool addressTerminated = 3; + */ + public boolean hasAddressTerminated() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + /** + * required bool addressTerminated = 3; + */ + public boolean getAddressTerminated() { + return addressTerminated_; + } + /** + * required bool addressTerminated = 3; + */ + public Builder setAddressTerminated(boolean value) { + bitField0_ |= 0x00000004; + addressTerminated_ = value; + onChanged(); + return this; + } + /** + * required bool addressTerminated = 3; + */ + public Builder clearAddressTerminated() { + bitField0_ = (bitField0_ & ~0x00000004); + addressTerminated_ = false; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:DeathWatchNotificationData) + } + + static { + defaultInstance = new DeathWatchNotificationData(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:DeathWatchNotificationData) + } + + private static akka.protobuf.Descriptors.Descriptor + internal_static_SystemMessage_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_SystemMessage_fieldAccessorTable; + private static akka.protobuf.Descriptors.Descriptor + internal_static_WatchData_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_WatchData_fieldAccessorTable; + private static akka.protobuf.Descriptors.Descriptor + internal_static_SuperviseData_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_SuperviseData_fieldAccessorTable; + private static akka.protobuf.Descriptors.Descriptor + internal_static_FailedData_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_FailedData_fieldAccessorTable; + private static akka.protobuf.Descriptors.Descriptor + internal_static_DeathWatchNotificationData_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_DeathWatchNotificationData_fieldAccessorTable; + + public static akka.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static akka.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\032SystemMessageFormats.proto\032\026ContainerF" + + "ormats.proto\"\212\003\n\rSystemMessage\022!\n\004type\030\001" + + " \002(\0162\023.SystemMessage.Type\022\035\n\twatchData\030\002" + + " \001(\0132\n.WatchData\022\033\n\tcauseData\030\003 \001(\0132\010.Pa" + + "yload\022%\n\rsuperviseData\030\005 \001(\0132\016.Supervise" + + "Data\022\037\n\nfailedData\030\006 \001(\0132\013.FailedData\0227\n" + + "\022dwNotificationData\030\007 \001(\0132\033.DeathWatchNo" + + "tificationData\"\230\001\n\004Type\022\n\n\006CREATE\020\000\022\014\n\010R" + + "ECREATE\020\001\022\013\n\007SUSPEND\020\002\022\n\n\006RESUME\020\003\022\r\n\tTE" + + "RMINATE\020\004\022\r\n\tSUPERVISE\020\005\022\t\n\005WATCH\020\006\022\013\n\007U", + "NWATCH\020\007\022\n\n\006FAILED\020\010\022\033\n\027DEATHWATCH_NOTIF" + + "ICATION\020\t\"C\n\tWatchData\022\032\n\007watchee\030\001 \002(\0132" + + "\t.ActorRef\022\032\n\007watcher\030\002 \002(\0132\t.ActorRef\"8" + + "\n\rSuperviseData\022\030\n\005child\030\001 \002(\0132\t.ActorRe" + + "f\022\r\n\005async\030\002 \002(\010\"3\n\nFailedData\022\030\n\005child\030" + + "\001 \002(\0132\t.ActorRef\022\013\n\003uid\030\002 \002(\004\"m\n\032DeathWa" + + "tchNotificationData\022\030\n\005actor\030\001 \002(\0132\t.Act" + + "orRef\022\032\n\022existenceConfirmed\030\002 \002(\010\022\031\n\021add" + + "ressTerminated\030\003 \002(\010B\017\n\013akka.remoteH\001" + }; + akka.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = + new akka.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { + public akka.protobuf.ExtensionRegistry assignDescriptors( + akka.protobuf.Descriptors.FileDescriptor root) { + descriptor = root; + internal_static_SystemMessage_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_SystemMessage_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_SystemMessage_descriptor, + new java.lang.String[] { "Type", "WatchData", "CauseData", "SuperviseData", "FailedData", "DwNotificationData", }); + internal_static_WatchData_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_WatchData_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_WatchData_descriptor, + new java.lang.String[] { "Watchee", "Watcher", }); + internal_static_SuperviseData_descriptor = + getDescriptor().getMessageTypes().get(2); + internal_static_SuperviseData_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_SuperviseData_descriptor, + new java.lang.String[] { "Child", "Async", }); + internal_static_FailedData_descriptor = + getDescriptor().getMessageTypes().get(3); + internal_static_FailedData_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_FailedData_descriptor, + new java.lang.String[] { "Child", "Uid", }); + internal_static_DeathWatchNotificationData_descriptor = + getDescriptor().getMessageTypes().get(4); + internal_static_DeathWatchNotificationData_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_DeathWatchNotificationData_descriptor, + new java.lang.String[] { "Actor", "ExistenceConfirmed", "AddressTerminated", }); + return null; + } + }; + akka.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new akka.protobuf.Descriptors.FileDescriptor[] { + akka.remote.ContainerFormats.getDescriptor(), + }, assigner); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/akka-remote/src/main/java/akka/remote/artery/AbstractAssociation.java b/akka-remote/src/main/java/akka/remote/artery/AbstractAssociation.java new file mode 100644 index 0000000000..03407235e2 --- /dev/null +++ b/akka-remote/src/main/java/akka/remote/artery/AbstractAssociation.java @@ -0,0 +1,18 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery; + +import akka.util.Unsafe; + +class AbstractAssociation { + protected final static long sharedStateOffset; + + static { + try { + sharedStateOffset = Unsafe.instance.objectFieldOffset(Association.class.getDeclaredField("_sharedStateDoNotCallMeDirectly")); + } catch(Throwable t){ + throw new ExceptionInInitializerError(t); + } + } +} diff --git a/akka-remote/src/main/java/akka/remote/artery/AeronErrorLog.java b/akka-remote/src/main/java/akka/remote/artery/AeronErrorLog.java new file mode 100644 index 0000000000..87f0389594 --- /dev/null +++ b/akka-remote/src/main/java/akka/remote/artery/AeronErrorLog.java @@ -0,0 +1,85 @@ +/* + * Copyright 2014 - 2016 Real Logic Ltd. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package akka.remote.artery; + +import io.aeron.CncFileDescriptor; +import org.agrona.DirectBuffer; +import org.agrona.IoUtil; +import org.agrona.concurrent.AtomicBuffer; +import org.agrona.concurrent.errors.ErrorLogReader; + +import akka.event.LoggingAdapter; + +import java.io.File; +import java.nio.MappedByteBuffer; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Application to print out errors recorded in the command-and-control (cnc) file is maintained by media driver in shared + * memory. This application reads the the cnc file and prints the distinct errors. Layout of the cnc file is described in + * {@link CncFileDescriptor}. + */ +public class AeronErrorLog +{ + private final File cncFile; + final MappedByteBuffer cncByteBuffer; + final DirectBuffer cncMetaDataBuffer; + final int cncVersion; + final AtomicBuffer buffer; + final SimpleDateFormat dateFormat; + + public AeronErrorLog(File cncFile) + { + this.cncFile = cncFile; + cncByteBuffer = IoUtil.mapExistingFile(cncFile, "cnc"); + cncMetaDataBuffer = CncFileDescriptor.createMetaDataBuffer(cncByteBuffer); + cncVersion = cncMetaDataBuffer.getInt(CncFileDescriptor.cncVersionOffset(0)); + buffer = CncFileDescriptor.createErrorLogBuffer(cncByteBuffer, cncMetaDataBuffer); + dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSSZ"); + + + if (CncFileDescriptor.CNC_VERSION != cncVersion) + { + IoUtil.unmap(cncByteBuffer); + throw new IllegalStateException("CNC version not supported: file version=" + cncVersion); + } + } + + public long logErrors(LoggingAdapter log, long sinceTimestamp) + { + // using AtomicLong because access from lambda, not because of currency + final AtomicLong lastTimestamp = new AtomicLong(sinceTimestamp); + + ErrorLogReader.read( + buffer, + (observationCount, firstObservationTimestamp, lastObservationTimestamp, encodedException) -> { + log.error(String.format( + "Aeron error: %d observations from %s to %s for:%n %s", + observationCount, + dateFormat.format(new Date(firstObservationTimestamp)), + dateFormat.format(new Date(lastObservationTimestamp)), + encodedException)); + lastTimestamp.set(Math.max(lastTimestamp.get(), lastObservationTimestamp)); + }, sinceTimestamp); + return lastTimestamp.get(); + } + + public void close() { + IoUtil.unmap(cncByteBuffer); + } +} diff --git a/akka-remote/src/main/java/akka/remote/artery/compress/CountMinSketch.java b/akka-remote/src/main/java/akka/remote/artery/compress/CountMinSketch.java new file mode 100644 index 0000000000..7cd729da60 --- /dev/null +++ b/akka-remote/src/main/java/akka/remote/artery/compress/CountMinSketch.java @@ -0,0 +1,253 @@ +/* + * Copyright (C) 2009-2016 Lightbend Inc. + */ + +package akka.remote.artery.compress; + +import akka.actor.ActorRef; + +/** + * INTERNAL API: Count-Min Sketch datastructure. + * + * Not thread-safe. + * + * An Improved Data Stream Summary: The Count-Min Sketch and its Applications + * https://web.archive.org/web/20060907232042/http://www.eecs.harvard.edu/~michaelm/CS222/countmin.pdf + * This implementation is mostly taken and adjusted from the Apache V2 licensed project `stream-lib`, located here: + * https://github.com/clearspring/stream-lib/blob/master/src/main/java/com/clearspring/analytics/stream/frequency/CountMinSketch.java + */ +public class CountMinSketch { + + private int depth; + private int width; + private long[][] table; + private long size; + private double eps; + private double confidence; + + private int[] recyclableCMSHashBuckets; + + + public CountMinSketch(int depth, int width, int seed) { + if((width & (width-1)) != 0){ + throw new IllegalArgumentException("width must be a power of 2, was: " + width ); + } + this.depth = depth; + this.width = width; + this.eps = 2.0 / width; + this.confidence = 1 - 1 / Math.pow(2, depth); + recyclableCMSHashBuckets = preallocateHashBucketsArray(depth); + initTablesWith(depth, width, seed); + } + + + private void initTablesWith(int depth, int width, int seed) { + this.table = new long[depth][width]; + } + + /** + * Referred to as {@code epsilon} in the whitepaper + */ + public double relativeError() { + return eps; + } + + public double confidence() { + return confidence; + } + + /** + * Similar to {@code add}, however we reuse the fact that the hask buckets have to be calculated for {@code add} + * already, and a separate {@code estimateCount} operation would have to calculate them again, so we do it all in one go. + */ + public long addObjectAndEstimateCount(Object item, long count) { + if (count < 0) { + // Actually for negative increments we'll need to use the median + // instead of minimum, and accuracy will suffer somewhat. + // Probably makes sense to add an "allow negative increments" + // parameter to constructor. + throw new IllegalArgumentException("Negative increments not implemented"); + } + Murmur3.hashBuckets(item, recyclableCMSHashBuckets, width); + for (int i = 0; i < depth; ++i) { + table[i][recyclableCMSHashBuckets[i]] += count; + } + size += count; + return estimateCount(recyclableCMSHashBuckets); + } + + public long size() { + return size; + } + + /** + * The estimate is correct within {@code 'epsilon' * (total item count)}, + * with probability {@code confidence}. + */ + public long estimateCount(Object item) { + Murmur3.hashBuckets(item, recyclableCMSHashBuckets, width); + return estimateCount(recyclableCMSHashBuckets); + } + + /** + * The estimate is correct within {@code 'epsilon' * (total item count)}, + * with probability {@code confidence}. + * + * @param buckets the "indexes" of buckets from which we want to calculate the count + */ + private long estimateCount(int[] buckets) { + long res = Long.MAX_VALUE; + for (int i = 0; i < depth; ++i) { + res = Math.min(res, table[i][buckets[i]]); + } + return res; + } + + + /** + * Local implementation of murmur3 hash optimized to used in count min sketch + * + * Inspired by scala (scala.util.hashing.MurmurHash3) and C port of MurmurHash3 + * + * scala.util.hashing => https://github.com/scala/scala/blob/2.12.x/src/library/scala/util/hashing/MurmurHash3.scala + * C port of MurmurHash3 => https://github.com/PeterScott/murmur3/blob/master/murmur3.c + */ + private static class Murmur3 { + + /** + * Force all bits of the hash to avalanche. Used for finalizing the hash. + */ + private static int avalanche(int hash) { + int h = hash; + + h ^= h >>> 16; + h *= 0x85ebca6b; + h ^= h >>> 13; + h *= 0xc2b2ae35; + h ^= h >>> 16; + + return h; + } + + private static int mixLast(int hash, int data) { + int k = data; + + k *= 0xcc9e2d51; //c1 + k = Integer.rotateLeft(k, 15); + k *= 0x1b873593; //c2 + + return hash ^ k; + } + + + private static int mix(int hash, int data) { + int h = mixLast(hash, data); + h = Integer.rotateLeft(h, 13); + return h * 5 + 0xe6546b64; + } + + public static int hash(Object o) { + if (o == null) { + return 0; + } + if (o instanceof ActorRef) { // TODO possibly scary optimisation + // ActorRef hashcode is the ActorPath#uid, which is a random number assigned at its creation, + // thus no hashing happens here - the value is already cached. + // TODO it should be thought over if this preciseness (just a random number, and not hashing) is good enough here? + // this is not cryptographic one, anything which is stable and random is good enough + return o.hashCode(); + } + if (o instanceof String) { + return hash(((String) o).getBytes()); + } + if (o instanceof Long) { + return hashLong((Long) o, 0); + } + if (o instanceof Integer) { + return hashLong((Integer) o, 0); + } + if (o instanceof Double) { + return hashLong(Double.doubleToRawLongBits((Double) o), 0); + } + if (o instanceof Float) { + return hashLong(Float.floatToRawIntBits((Float) o), 0); + } + if (o instanceof byte[]) { + return bytesHash((byte[]) o, 0); + } + return hash(o.toString()); + } + + static int hashLong(long value, int seed) { + int h = seed; + h = mix(h, (int) (value)); + h = mixLast(h, (int) (value >>> 32)); + return avalanche(h ^ 2); + } + + static int bytesHash(final byte[] data, int seed) { + int len = data.length; + int h = seed; + + // Body + int i = 0; + while (len >= 4) { + int k = data[i] & 0xFF; + k |= (data[i + 1] & 0xFF) << 8; + k |= (data[i + 2] & 0xFF) << 16; + k |= (data[i + 3] & 0xFF) << 24; + + h = mix(h, k); + + i += 4; + len -= 4; + } + + // Tail + int k = 0; + if (len == 3) k ^= (data[i + 2] & 0xFF) << 16; + if (len >= 2) k ^= (data[i + 1] & 0xFF) << 8; + if (len >= 1) { + k ^= (data[i] & 0xFF); + h = mixLast(h, k); + } + + // Finalization + return avalanche(h ^ data.length); + } + + /** + * Hash item using pair independent hash functions. + * + * Implemetation based on "Less Hashing, Same Performance: Building a + * Better Bloom Filter" http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf + * @param item what should be hashed + * @param hashBuckets where hashes should be placed + * @param limit value to shrink result + */ + static void hashBuckets(Object item, final int[] hashBuckets, int limit) { + final int hash1 = hash(item); // specialized hash for ActorRef and Strings + final int hash2 = hashLong(hash1, hash1); + final int depth = hashBuckets.length; + final int mask = limit - 1; + for (int i = 0; i < depth; i++) { + hashBuckets[i] = Math.abs((hash1 + i * hash2) & mask); //shrink done by AND instead MOD. Assume limit is power of 2 + } + } + + } + + private int[] preallocateHashBucketsArray(int depth) { + return new int[depth]; + } + + @Override + public String toString() { + return "CountMinSketch{" + + "confidence=" + confidence + + ", size=" + size + + ", depth=" + depth + + ", width=" + width + + '}'; + } +} diff --git a/akka-remote/src/main/protobuf/ArteryControlFormats.proto b/akka-remote/src/main/protobuf/ArteryControlFormats.proto new file mode 100644 index 0000000000..8970092b9f --- /dev/null +++ b/akka-remote/src/main/protobuf/ArteryControlFormats.proto @@ -0,0 +1,87 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ + +option java_package = "akka.remote"; +option optimize_for = SPEED; + +message Quarantined { + required UniqueAddress from = 1; + required UniqueAddress to = 2; +} + +// Generic message declaration that is used for all types of message that (currently) have a single +// address field. A message that needs to changed later can be cloned from this one and then adapted. +// ActorSystemTerminating +// ActorSystemTerminating.Ack +// OutboundHandshake.HandshakeRsp +message MessageWithAddress { + required UniqueAddress address = 1; +} + +message HandshakeReq { + required UniqueAddress from = 1; + required Address to = 2; +} + +// CompressionProtocol.ActorRefCompressionAdvertisement +// CompressionProtocol.ClassManifestCompressionAdvertisement +message CompressionTableAdvertisement { + required UniqueAddress from = 1; + required uint64 originUid = 2; + required uint32 tableVersion = 3; + + // actual Map is represented by separate sequences of keys and values, + // relies on both sequences using the same order so that corresponding entries can be + // associated again when deserializing + repeated string keys = 4; + repeated uint32 values = 5; +} + +// CompressionProtocol.ActorRefCompressionAdvertisementAck +// CompressionProtocol.ClassManifestCompressionAdvertisementAck +message CompressionTableAdvertisementAck { + required UniqueAddress from = 1; + required uint32 version = 2; +} + +// SystemMessageDelivery.SystemMessageEnvelope +message SystemMessageEnvelope { + required bytes message = 1; + required int32 serializerId = 2; + optional bytes messageManifest = 3; + required uint64 seqNo = 4; + required UniqueAddress ackReplyTo = 5; +} + +// SystemMessageDelivery.Ack +// SystemMessageDelivery.Nack +message SystemMessageDeliveryAck { + required uint64 seqNo = 1; + required UniqueAddress from = 2; +} + +/** + * Defines a remote address. + */ +message Address { + required string protocol = 1; + required string system = 2; + required string hostname = 3; + required uint32 port = 4; +} + +/** + * Defines a remote address with uid. + */ +message UniqueAddress { + required Address address = 1; + required uint64 uid = 2; +} + + +// RemoteWatcher.ArteryHeartbeat is empty array +// RemoteWatcher.ArteryHeartbeatRsp +message ArteryHeartbeatRsp { + required uint64 uid = 1; +} \ No newline at end of file diff --git a/akka-remote/src/main/protobuf/ContainerFormats.proto b/akka-remote/src/main/protobuf/ContainerFormats.proto index 84785740c4..a760413f8d 100644 --- a/akka-remote/src/main/protobuf/ContainerFormats.proto +++ b/akka-remote/src/main/protobuf/ContainerFormats.proto @@ -50,3 +50,27 @@ message Payload { required int32 serializerId = 2; optional bytes messageManifest = 4; } + +message WatcherHeartbeatResponse { + required uint64 uid = 1; +} + +message Throwable { + required string className = 1; + optional string message = 2; + optional Payload cause = 3; + repeated StackTraceElement stackTrace = 4; +} + +message ActorInitializationException { + optional ActorRef actor = 1; + required string message = 2; + required Payload cause = 3; +} + +message StackTraceElement { + required string className = 1; + required string methodName = 2; + required string fileName = 3; + required int32 lineNumber = 4; +} diff --git a/akka-remote/src/main/protobuf/SystemMessageFormats.proto b/akka-remote/src/main/protobuf/SystemMessageFormats.proto new file mode 100644 index 0000000000..1c6a0d51fa --- /dev/null +++ b/akka-remote/src/main/protobuf/SystemMessageFormats.proto @@ -0,0 +1,62 @@ +/** + * Copyright (C) 2009-2015 Lightbend Inc. + */ + +option java_package = "akka.remote"; +option optimize_for = SPEED; + +import "ContainerFormats.proto"; + +/****************************************** + * System message formats + ******************************************/ + +message SystemMessage { + enum Type { + CREATE = 0; + RECREATE = 1; + SUSPEND = 2; + RESUME = 3; + TERMINATE = 4; + SUPERVISE = 5; + WATCH = 6; + UNWATCH = 7; + FAILED = 8; + DEATHWATCH_NOTIFICATION = 9; + } + + required Type type = 1; + + optional WatchData watchData = 2; + optional Payload causeData = 3; + optional SuperviseData superviseData = 5; + optional FailedData failedData = 6; + optional DeathWatchNotificationData dwNotificationData = 7; + +} + +message WatchData { + required ActorRef watchee = 1; + required ActorRef watcher = 2; +} + +message SuperviseData { + required ActorRef child = 1; + required bool async = 2; +} + +message FailedData { + required ActorRef child = 1; + required uint64 uid = 2; +} + +message DeathWatchNotificationData { + required ActorRef actor = 1; + required bool existenceConfirmed = 2; + required bool addressTerminated = 3; +} + + + + + diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 9eff6d0f37..d1c630e2e4 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -15,36 +15,77 @@ akka { serializers { akka-containers = "akka.remote.serialization.MessageContainerSerializer" akka-misc = "akka.remote.serialization.MiscMessageSerializer" + artery = "akka.remote.serialization.ArteryMessageSerializer" proto = "akka.remote.serialization.ProtobufSerializer" daemon-create = "akka.remote.serialization.DaemonMsgCreateSerializer" + primitive-long = "akka.remote.serialization.LongSerializer" + primitive-int = "akka.remote.serialization.IntSerializer" + primitive-string = "akka.remote.serialization.StringSerializer" + primitive-bytestring = "akka.remote.serialization.ByteStringSerializer" + akka-system-msg = "akka.remote.serialization.SystemMessageSerializer" } serialization-bindings { "akka.actor.ActorSelectionMessage" = akka-containers - # The classes akka.actor.Identify and akka.actor.ActorIdentity serialization/deserialization are required by - # the cluster client to work. - # For the purpose of preserving protocol backward compatibility, akka.actor.Identify and akka.actor.ActorIdentity - # are stil using java serialization by default. - # Should java serialization be disabled, uncomment the following lines - # "akka.actor.Identify" = akka-misc - # "akka.actor.ActorIdentity" = akka-misc - # Should java serialization be disabled, uncomment the following lines - # "scala.Some" = akka-misc - # "scala.None$" = akka-misc + "akka.remote.DaemonMsgCreate" = daemon-create - + + "akka.remote.artery.ArteryMessage" = artery + # Since akka.protobuf.Message does not extend Serializable but # GeneratedMessage does, need to use the more specific one here in order # to avoid ambiguity. "akka.protobuf.GeneratedMessage" = proto - + # Since com.google.protobuf.Message does not extend Serializable but # GeneratedMessage does, need to use the more specific one here in order # to avoid ambiguity. # This com.google.protobuf serialization binding is only used if the class can be loaded, # i.e. com.google.protobuf dependency has been added in the application project. "com.google.protobuf.GeneratedMessage" = proto - + } + + # For the purpose of preserving protocol backward compatibility these bindings are not + # included by default. They can be enabled with enable-additional-serialization-bindings=on. + # They are enabled by default if akka.remote.artery.enabled=on. + additional-serialization-bindings { + "akka.actor.Identify" = akka-misc + "akka.actor.ActorIdentity" = akka-misc + "scala.Some" = akka-misc + "scala.None$" = akka-misc + "akka.actor.Status$Success" = akka-misc + "akka.actor.Status$Failure" = akka-misc + "akka.actor.ActorRef" = akka-misc + "akka.actor.PoisonPill$" = akka-misc + "akka.actor.Kill$" = akka-misc + "akka.remote.RemoteWatcher$Heartbeat$" = akka-misc + "akka.remote.RemoteWatcher$HeartbeatRsp" = akka-misc + "akka.actor.ActorInitializationException" = akka-misc + + "akka.dispatch.sysmsg.SystemMessage" = akka-system-msg + + "java.lang.String" = primitive-string + "akka.util.ByteString$ByteString1C" = primitive-bytestring + "akka.util.ByteString$ByteString1" = primitive-bytestring + "akka.util.ByteString$ByteStrings" = primitive-bytestring + "java.lang.Long" = primitive-long + "scala.Long" = primitive-long + "java.lang.Integer" = primitive-int + "scala.Int" = primitive-int + + # Java Serializer is by default used for exceptions. + # It's recommended that you implement custom serializer for exceptions that are + # sent remotely, e.g. in akka.actor.Status.Failure for ask replies. You can add + # binding to akka-misc (MiscMessageSerializerSpec) for the exceptions that have + # a constructor with single message String or constructor with message String as + # first parameter and cause Throwable as second parameter. Note that it's not + # safe to add this binding for general exceptions such as IllegalArgumentException + # because it may have a subclass without required constructor. + "java.lang.Throwable" = java + "akka.actor.IllegalActorStateException" = akka-misc + "akka.actor.ActorKilledException" = akka-misc + "akka.actor.InvalidActorNameException" = akka-misc + "akka.actor.InvalidMessageException" = akka-misc } serialization-identifiers { @@ -52,6 +93,12 @@ akka { "akka.remote.serialization.DaemonMsgCreateSerializer" = 3 "akka.remote.serialization.MessageContainerSerializer" = 6 "akka.remote.serialization.MiscMessageSerializer" = 16 + "akka.remote.serialization.ArteryMessageSerializer" = 17 + "akka.remote.serialization.LongSerializer" = 18 + "akka.remote.serialization.IntSerializer" = 19 + "akka.remote.serialization.StringSerializer" = 20 + "akka.remote.serialization.ByteStringSerializer" = 21 + "akka.remote.serialization.SystemMessageSerializer" = 22 } deployment { @@ -83,6 +130,306 @@ akka { remote { + artery { + + # Enable the new remoting with this flag + enabled = off + + # Canonical address is the address other clients should connect to. + # Artery transport will expect messages to this address. + canonical { + + # The default remote server port clients should connect to. + # Default is 25520, use 0 if you want a random available port + # This port needs to be unique for each actor system on the same machine. + port = 25520 + + # Hostname clients should connect to. Can be set to an ip, hostname + # or one of the following special values: + # "" InetAddress.getLocalHost.getHostAddress + # "" InetAddress.getLocalHost.getHostName + # + hostname = "" + } + + # Use these settings to bind a network interface to a different address + # than artery expects messages at. This may be used when running Akka + # nodes in a separated networks (under NATs or in containers). If canonical + # and bind addresses are different, then network configuration that relays + # communications from canonical to bind addresses is expected. + bind { + + # Port to bind a network interface to. Can be set to a port number + # of one of the following special values: + # 0 random available port + # "" akka.remote.artery.canonical.port + # + port = "" + + # Hostname to bind a network interface to. Can be set to an ip, hostname + # or one of the following special values: + # "0.0.0.0" all interfaces + # "" akka.remote.artery.canonical.hostname + # "" InetAddress.getLocalHost.getHostAddress + # "" InetAddress.getLocalHost.getHostName + # + hostname = "" + } + + # Actor paths to use the large message stream for when a message + # is sent to them over remoting. The large message stream dedicated + # is separate from "normal" and system messages so that sending a + # large message does not interfere with them. + # Entries should be the full path to the actor. Wildcards in the form of "*" + # can be supplied at any place and matches any name at that segment - + # "/user/supervisor/actor/*" will match any direct child to actor, + # while "/supervisor/*/child" will match any grandchild to "supervisor" that + # has the name "child" + # Messages sent to ActorSelections will not be passed through the large message + # stream, to pass such messages through the large message stream the selections + # but must be resolved to ActorRefs first. + large-message-destinations = [] + + # Enable untrusted mode, which discards inbound system messages, PossiblyHarmful and + # ActorSelection messages. E.g. remote watch and remote deployment will not work. + # ActorSelection messages can be enabled for specific paths with the trusted-selection-paths + untrusted-mode = off + + # When 'untrusted-mode=on' inbound actor selections are by default discarded. + # Actors with paths defined in this white list are granted permission to receive actor + # selections messages. + # E.g. trusted-selection-paths = ["/user/receptionist", "/user/namingService"] + trusted-selection-paths = [] + + # If this is "on", all inbound remote messages will be logged at DEBUG level, + # if off then they are not logged + log-received-messages = off + + # If this is "on", all outbound remote messages will be logged at DEBUG level, + # if off then they are not logged + log-sent-messages = off + + advanced { + + # Maximum serialized message size, including header data. + maximum-frame-size = 256 KiB + + # Direct byte buffers are reused in a pool with this maximum size. + # Each buffer has the size of 'maximum-frame-size'. + # This is not a hard upper limit on number of created buffers. Additional + # buffers will be created if needed, e.g. when using many outbound + # associations at the same time. Such additional buffers will be garbage + # collected, which is not as efficient as reusing buffers in the pool. + buffer-pool-size = 128 + + # Maximum serialized message size for the large messages, including header data. + # See 'large-message-destinations'. + maximum-large-frame-size = 2 MiB + + # Direct byte buffers for the large messages are reused in a pool with this maximum size. + # Each buffer has the size of 'maximum-large-frame-size'. + # See 'large-message-destinations'. + # This is not a hard upper limit on number of created buffers. Additional + # buffers will be created if needed, e.g. when using many outbound + # associations at the same time. Such additional buffers will be garbage + # collected, which is not as efficient as reusing buffers in the pool. + large-buffer-pool-size = 32 + + # For enabling testing features, such as blackhole in akka-remote-testkit. + test-mode = off + + # Settings for the materializer that is used for the remote streams. + materializer = ${akka.stream.materializer} + + # If set to a nonempty string artery will use the given dispatcher for + # the ordinary and large message streams, otherwise the default dispatcher is used. + use-dispatcher = "akka.remote.default-remote-dispatcher" + + # If set to a nonempty string remoting will use the given dispatcher for + # the control stream, otherwise the default dispatcher is used. + # It can be good to not use the same dispatcher for the control stream as + # the dispatcher for the ordinary message stream so that heartbeat messages + # are not disturbed. + use-control-stream-dispatcher = "" + + # Controls whether to start the Aeron media driver in the same JVM or use external + # process. Set to 'off' when using external media driver, and then also set the + # 'aeron-dir'. + embedded-media-driver = on + + # Directory used by the Aeron media driver. It's mandatory to define the 'aeron-dir' + # if using external media driver, i.e. when 'embedded-media-driver = off'. + # Embedded media driver will use a this directory, or a temporary directory if this + # property is not defined (empty). + aeron-dir = "" + + # Whether to delete aeron embeded driver directory upon driver stop. + delete-aeron-dir = yes + + # Level of CPU time used, on a scale between 1 and 10, during backoff/idle. + # The tradeoff is that to have low latency more CPU time must be used to be + # able to react quickly on incoming messages or send as fast as possible after + # backoff backpressure. + # Level 1 strongly prefer low CPU consumption over low latency. + # Level 10 strongly prefer low latency over low CPU consumption. + idle-cpu-level = 5 + + # WARNING: This feature is not supported yet. Don't use other value than 1. + # It requires more hardening and performance optimizations. + # Number of outbound lanes for each outbound association. A value greater than 1 + # means that serialization can be performed in parallel for different destination + # actors. The selection of lane is based on consistent hashing of the recipient + # ActorRef to preserve message ordering per receiver. + outbound-lanes = 1 + + # WARNING: This feature is not supported yet. Don't use other value than 1. + # It requires more hardening and performance optimizations. + # Total number of inbound lanes, shared among all inbound associations. A value + # greater than 1 means that deserialization can be performed in parallel for + # different destination actors. The selection of lane is based on consistent + # hashing of the recipient ActorRef to preserve message ordering per receiver. + inbound-lanes = 1 + + # Size of the send queue for outgoing messages. Messages will be dropped if + # the queue becomes full. This may happen if you send a burst of many messages + # without end-to-end flow control. Note that there is one such queue per + # outbound association. The trade-off of using a larger queue size is that + # it consumes more memory, since the queue is based on preallocated array with + # fixed size. + outbound-message-queue-size = 3072 + + # Size of the send queue for outgoing control messages, such as system messages. + # If this limit is reached the remote system is declared to be dead and its UID + # marked as quarantined. + # The trade-off of using a larger queue size is that it consumes more memory, + # since the queue is based on preallocated array with fixed size. + outbound-control-queue-size = 3072 + + # Size of the send queue for outgoing large messages. Messages will be dropped if + # the queue becomes full. This may happen if you send a burst of many messages + # without end-to-end flow control. Note that there is one such queue per + # outbound association. The trade-off of using a larger queue size is that + # it consumes more memory, since the queue is based on preallocated array with + # fixed size. + outbound-large-message-queue-size = 256 + + # This setting defines the maximum number of unacknowledged system messages + # allowed for a remote system. If this limit is reached the remote system is + # declared to be dead and its UID marked as quarantined. + system-message-buffer-size = 20000 + + # unacknowledged system messages are re-delivered with this interval + system-message-resend-interval = 1 second + + # The timeout for outbound associations to perform the handshake. + # This timeout must be greater than the 'image-liveness-timeout'. + handshake-timeout = 20 s + + # incomplete handshake attempt is retried with this interval + handshake-retry-interval = 1 second + + # handshake requests are performed periodically with this interval, + # also after the handshake has been completed to be able to establish + # a new session with a restarted destination system + inject-handshake-interval = 1 second + + # messages that are not accepted by Aeron are dropped after retrying for this period + give-up-message-after = 60 seconds + + # System messages that are not acknowledged after re-sending for this period are + # dropped and will trigger quarantine. The value should be longer than the length + # of a network partition that you need to survive. + give-up-system-message-after = 6 hours + + # during ActorSystem termination the remoting will wait this long for + # an acknowledgment by the destination system that flushing of outstanding + # remote messages has been completed + shutdown-flush-timeout = 1 second + + # See 'inbound-max-restarts' + inbound-restart-timeout = 5 seconds + + # Max number of restarts within 'inbound-restart-timeout' for the inbound streams. + # If more restarts occurs the ActorSystem will be terminated. + inbound-max-restarts = 5 + + # See 'outbound-max-restarts' + outbound-restart-timeout = 5 seconds + + # Max number of restarts within 'outbound-restart-timeout' for the outbound streams. + # If more restarts occurs the ActorSystem will be terminated. + outbound-max-restarts = 5 + + # Stop outbound stream of a quarantined association after this idle timeout, i.e. + # when not used any more. + stop-quarantined-after-idle = 3 seconds + + # Timeout after which aeron driver has not had keepalive messages + # from a client before it considers the client dead. + client-liveness-timeout = 20 seconds + + # Timeout for each the INACTIVE and LINGER stages an aeron image + # will be retained for when it is no longer referenced. + # This timeout must be less than the 'handshake-timeout'. + image-liveness-timeout = 10 seconds + + # Timeout after which the aeron driver is considered dead + # if it does not update its C'n'C timestamp. + driver-timeout = 20 seconds + + flight-recorder { + // FIXME it should be enabled by default, but there is some concurrency issue that crashes the JVM + enabled = off + # Controls where the flight recorder file will be written. There are three options: + # 1. Empty: a file will be generated in the temporary directory of the OS + # 2. A relative or absolute path ending with ".afr": this file will be used + # 3. A relative or absolute path: this directory will be used, the file will get a random file name + destination="" + } + + # compression of common strings in remoting messages, like actor destinations, serializers etc + compression { + + actor-refs { + # Max number of compressed actor-refs + # Note that compression tables are "rolling" (i.e. a new table replaces the old + # compression table once in a while), and this setting is only about the total number + # of compressions within a single such table. + # Must be a positive natural number. + max = 256 + + # interval between new table compression advertisements. + # this means the time during which we collect heavy-hitter data and then turn it into a compression table. + advertisement-interval = 1 minute # TODO find good number as default, for benchmarks trigger immediately + } + manifests { + # Max number of compressed manifests + # Note that compression tables are "rolling" (i.e. a new table replaces the old + # compression table once in a while), and this setting is only about the total number + # of compressions within a single such table. + # Must be a positive natural number. + max = 256 + + # interval between new table compression advertisements. + # this means the time during which we collect heavy-hitter data and then turn it into a compression table. + advertisement-interval = 1 minute # TODO find good number as default, for benchmarks trigger immediately + } + } + + # List of fully qualified class names of remote instruments which should + # be initialized and used for monitoring of remote messages. + # The class must extend akka.remote.artery.RemoteInstrument and + # have a public constructor with empty parameters or one ExtendedActorSystem + # parameter. + # A new instance of RemoteInstrument will be created for each encoder and decoder. + # It's only called from the stage, so if it dosn't delegate to any shared instance + # it doesn't have to be thread-safe. + # Refer to `akka.remote.artery.RemoteInstrument` for more information. + instruments = ${?akka.remote.artery.advanced.instruments} [] + } + + } + ### General settings # Timeout after which the startup of the remoting subsystem is considered @@ -110,7 +457,7 @@ akka { # Acknowledgment timeout of management commands sent to the transport stack. command-ack-timeout = 30 s - + # The timeout for outbound associations to perform the handshake. # If the transport is akka.remote.netty.tcp or akka.remote.netty.ssl # the configured connection-timeout for the transport will be used instead. @@ -122,18 +469,18 @@ akka { # "enabled-transport" and "adapters" entries) it is not guaranteed that # every module will respect this setting. use-dispatcher = "akka.remote.default-remote-dispatcher" - + ### Security settings # Enable untrusted mode for full security of server managed actors, prevents # system messages to be send by clients, e.g. messages like 'Create', # 'Suspend', 'Resume', 'Terminate', 'Supervise', 'Link' etc. untrusted-mode = off - + # When 'untrusted-mode=on' inbound actor selections are by default discarded. # Actors with paths defined in this white list are granted permission to receive actor - # selections messages. - # E.g. trusted-selection-paths = ["/user/receptionist", "/user/namingService"] + # selections messages. + # E.g. trusted-selection-paths = ["/user/receptionist", "/user/namingService"] trusted-selection-paths = [] # Should the remote server require that its peers share the same @@ -171,7 +518,7 @@ akka { # a value in bytes, such as 1000b. Note that for all messages larger than this # limit there will be extra performance and scalability cost. log-frame-size-exceeding = off - + # Log warning if the number of messages in the backoff buffer in the endpoint # writer exceeds this limit. It can be disabled by setting the value to off. log-buffer-size-exceeding = 50000 @@ -180,7 +527,7 @@ akka { # Settings for the failure detector to monitor connections. # For TCP it is not important to have fast failure detection, since - # most connection failures are captured by TCP itself. + # most connection failures are captured by TCP itself. # The default DeadlineFailureDetector will trigger if there are no heartbeats within # the duration heartbeat-interval + acceptable-heartbeat-pause, i.e. 20 seconds # with the default settings. @@ -308,7 +655,7 @@ akka { # Messages that were negatively acknowledged are always immediately # resent. resend-interval = 2 s - + # Maximum number of unacknowledged system messages that will be resent # each 'resend-interval'. If you watch many (> 1000) remote actors you can # increase this value to for example 600, but a too large limit (e.g. 10000) @@ -548,7 +895,7 @@ akka { # "AES256CounterSecureRNG" # # The following are deprecated in Akka 2.4. They use one of 3 possible - # seed sources, depending on availability: /dev/random, random.org and + # seed sources, depending on availability: /dev/random, random.org and # SecureRandom (provided by Java) # "AES128CounterInetRNG" # "AES256CounterInetRNG" (Install JCE Unlimited Strength Jurisdiction @@ -572,12 +919,13 @@ akka { type = Dispatcher executor = "fork-join-executor" fork-join-executor { - # Min number of threads to cap factor-based parallelism number to parallelism-min = 2 - parallelism-max = 2 + parallelism-factor = 0.5 + parallelism-max = 16 } + throughput = 10 } - + backoff-remote-dispatcher { type = Dispatcher executor = "fork-join-executor" @@ -587,8 +935,5 @@ akka { parallelism-max = 2 } } - - } - } diff --git a/akka-remote/src/main/scala/akka/remote/AddressUidExtension.scala b/akka-remote/src/main/scala/akka/remote/AddressUidExtension.scala index 027f703a24..ddec6fa52a 100644 --- a/akka-remote/src/main/scala/akka/remote/AddressUidExtension.scala +++ b/akka-remote/src/main/scala/akka/remote/AddressUidExtension.scala @@ -11,7 +11,9 @@ import akka.actor.ExtensionId import akka.actor.ExtensionIdProvider /** - * Extension that holds a uid that is assigned as a random `Int`. + * Extension that holds a uid that is assigned as a random `Long` or `Int` depending + * on which version of remoting that is used. + * * The uid is intended to be used together with an [[akka.actor.Address]] * to be able to distinguish restarted actor system using the same host * and port. @@ -22,8 +24,26 @@ object AddressUidExtension extends ExtensionId[AddressUidExtension] with Extensi override def lookup = AddressUidExtension override def createExtension(system: ExtendedActorSystem): AddressUidExtension = new AddressUidExtension(system) + } class AddressUidExtension(val system: ExtendedActorSystem) extends Extension { - val addressUid: Int = ThreadLocalRandom.current.nextInt() -} \ No newline at end of file + + private def arteryEnabled = system.provider.asInstanceOf[RemoteActorRefProvider].remoteSettings.Artery.Enabled + + val longAddressUid: Long = { + val tlr = ThreadLocalRandom.current + if (arteryEnabled) tlr.nextLong() + // with the old remoting we need to make toInt.toLong return the same number + // to keep wire compatibility + else tlr.nextInt().toLong + } + + // used by old remoting and part of public api + @deprecated("Use longAddressUid instead", "2.4.x") + lazy val addressUid: Int = { + if (arteryEnabled) { + throw new IllegalStateException("Int UID must never be used with Artery") + } else longAddressUid.toInt + } +} diff --git a/akka-remote/src/main/scala/akka/remote/BoundAddressesExtension.scala b/akka-remote/src/main/scala/akka/remote/BoundAddressesExtension.scala index 07d1df14b8..9e931fde62 100644 --- a/akka-remote/src/main/scala/akka/remote/BoundAddressesExtension.scala +++ b/akka-remote/src/main/scala/akka/remote/BoundAddressesExtension.scala @@ -9,6 +9,7 @@ import akka.actor.ExtendedActorSystem import akka.actor.Extension import akka.actor.ExtensionId import akka.actor.ExtensionIdProvider +import akka.remote.artery.ArteryTransport /** * Extension provides access to bound addresses. @@ -27,6 +28,8 @@ class BoundAddressesExtension(val system: ExtendedActorSystem) extends Extension * Returns a mapping from a protocol to a set of bound addresses. */ def boundAddresses: Map[String, Set[Address]] = system.provider - .asInstanceOf[RemoteActorRefProvider].transport - .asInstanceOf[Remoting].boundAddresses + .asInstanceOf[RemoteActorRefProvider].transport match { + case artery: ArteryTransport ⇒ Map((ArteryTransport.ProtocolName → Set(artery.bindAddress.address))) + case remoting: Remoting ⇒ remoting.boundAddresses + } } diff --git a/akka-remote/src/main/scala/akka/remote/Endpoint.scala b/akka-remote/src/main/scala/akka/remote/Endpoint.scala index 18624b2417..497f179141 100644 --- a/akka-remote/src/main/scala/akka/remote/Endpoint.scala +++ b/akka-remote/src/main/scala/akka/remote/Endpoint.scala @@ -27,6 +27,8 @@ import scala.concurrent.duration.{ Deadline } import scala.util.control.NonFatal import java.util.concurrent.locks.LockSupport import scala.concurrent.Future +import akka.util.OptionVal +import akka.util.OptionVal /** * INTERNAL API @@ -36,7 +38,7 @@ private[remote] trait InboundMessageDispatcher { recipient: InternalActorRef, recipientAddress: Address, serializedMessage: SerializedMessage, - senderOption: Option[ActorRef]): Unit + senderOption: OptionVal[ActorRef]): Unit } /** @@ -53,7 +55,7 @@ private[remote] class DefaultMessageDispatcher( recipient: InternalActorRef, recipientAddress: Address, serializedMessage: SerializedMessage, - senderOption: Option[ActorRef]): Unit = { + senderOption: OptionVal[ActorRef]): Unit = { import provider.remoteSettings._ diff --git a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala index 8306ff533a..f1422c7a1a 100644 --- a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala @@ -7,6 +7,9 @@ package akka.remote import akka.remote.WireFormats._ import akka.protobuf.ByteString import akka.actor.ExtendedActorSystem +import akka.remote.artery.{ EnvelopeBuffer, HeaderBuilder } +import akka.serialization.Serialization +import akka.serialization.ByteBufferSerializer import akka.serialization.SerializationExtension import akka.serialization.SerializerWithStringManifest import scala.util.control.NonFatal @@ -59,4 +62,31 @@ private[akka] object MessageSerializer { s"using serializer [${serializer.getClass}].", e) } } + + def serializeForArtery(serialization: Serialization, message: AnyRef, headerBuilder: HeaderBuilder, envelope: EnvelopeBuffer): Unit = { + val serializer = serialization.findSerializerFor(message) + + headerBuilder setSerializer serializer.identifier + + def manifest: String = serializer match { + case ser: SerializerWithStringManifest ⇒ ser.manifest(message) + case _ ⇒ if (serializer.includeManifest) message.getClass.getName else "" + } + + serializer match { + case ser: ByteBufferSerializer ⇒ + headerBuilder setManifest manifest + envelope.writeHeader(headerBuilder) + ser.toBinary(message, envelope.byteBuffer) + case _ ⇒ + headerBuilder setManifest manifest + envelope.writeHeader(headerBuilder) + envelope.byteBuffer.put(serializer.toBinary(message)) + } + } + + def deserializeForArtery(system: ExtendedActorSystem, originUid: Long, serialization: Serialization, + serializer: Int, classManifest: String, envelope: EnvelopeBuffer): AnyRef = { + serialization.deserializeByteBuffer(envelope.byteBuffer, serializer, classManifest) + } } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 4ad398f696..7b8b871941 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -7,16 +7,22 @@ package akka.remote import akka.Done import akka.actor._ import akka.dispatch.sysmsg._ -import akka.event.{ Logging, LoggingAdapter, EventStream } +import akka.event.{ EventStream, Logging, LoggingAdapter } import akka.event.Logging.Error import akka.serialization.{ Serialization, SerializationExtension } import akka.pattern.pipe import scala.util.control.NonFatal -import akka.actor.SystemGuardian.{ TerminationHookDone, TerminationHook, RegisterTerminationHook } +import akka.actor.SystemGuardian.{ RegisterTerminationHook, TerminationHook, TerminationHookDone } import scala.util.control.Exception.Catcher import scala.concurrent.Future import akka.ConfigurationException import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics } +import akka.remote.artery.ArteryTransport +import akka.util.OptionVal +import akka.remote.artery.OutboundEnvelope +import akka.remote.artery.SystemMessageDelivery.SystemMessageEnvelope +import akka.remote.serialization.ActorRefResolveCache +import akka.remote.serialization.ActorRefResolveThreadLocalCache /** * INTERNAL API @@ -86,17 +92,28 @@ private[akka] object RemoteActorRefProvider { import EndpointManager.Send override def !(message: Any)(implicit sender: ActorRef): Unit = message match { - case Send(m, senderOption, _, seqOpt) ⇒ + case Send(m, senderOption, recipient, seqOpt) ⇒ // else ignore: it is a reliably delivered message that might be retried later, and it has not yet deserved // the dead letter status - if (seqOpt.isEmpty) super.!(m)(senderOption.orNull) + if (seqOpt.isEmpty) super.!(DeadLetter(m, senderOption.getOrElse(_provider.deadLetters), recipient)) case DeadLetter(Send(m, senderOption, recipient, seqOpt), _, _) ⇒ // else ignore: it is a reliably delivered message that might be retried later, and it has not yet deserved // the dead letter status - if (seqOpt.isEmpty) super.!(m)(senderOption.orNull) + if (seqOpt.isEmpty) super.!(DeadLetter(m, senderOption.getOrElse(_provider.deadLetters), recipient)) + case env: OutboundEnvelope ⇒ + super.!(DeadLetter(unwrapSystemMessageEnvelope(env.message), env.sender.getOrElse(_provider.deadLetters), + env.recipient.getOrElse(_provider.deadLetters))) + case DeadLetter(env: OutboundEnvelope, _, _) ⇒ + super.!(DeadLetter(unwrapSystemMessageEnvelope(env.message), env.sender.getOrElse(_provider.deadLetters), + env.recipient.getOrElse(_provider.deadLetters))) case _ ⇒ super.!(message)(sender) } + private def unwrapSystemMessageEnvelope(msg: AnyRef): AnyRef = msg match { + case SystemMessageEnvelope(m, _, _) ⇒ m + case _ ⇒ msg + } + @throws(classOf[java.io.ObjectStreamException]) override protected def writeReplace(): AnyRef = DeadLetterActorRef.serialized } @@ -160,9 +177,13 @@ private[akka] class RemoteActorRefProvider( @volatile private var remoteDeploymentWatcher: ActorRef = _ + @volatile private var actorRefResolveThreadLocalCache: ActorRefResolveThreadLocalCache = _ + def init(system: ActorSystemImpl): Unit = { local.init(system) + actorRefResolveThreadLocalCache = ActorRefResolveThreadLocalCache(system) + remotingTerminator = system.systemActorOf( remoteSettings.configureDispatcher(Props(classOf[RemotingTerminator], local.systemGuardian)), "remoting-terminator") @@ -180,7 +201,7 @@ private[akka] class RemoteActorRefProvider( d }, serialization = SerializationExtension(system), - transport = new Remoting(system, this)) + transport = if (remoteSettings.Artery.Enabled) new ArteryTransport(system, this) else new Remoting(system, this)) _internals = internals remotingTerminator ! internals @@ -215,7 +236,7 @@ private[akka] class RemoteActorRefProvider( } protected def createRemoteDeploymentWatcher(system: ActorSystemImpl): ActorRef = - system.systemActorOf(remoteSettings.configureDispatcher(Props[RemoteDeploymentWatcher]), "remote-deployment-watcher") + system.systemActorOf(remoteSettings.configureDispatcher(Props[RemoteDeploymentWatcher]()), "remote-deployment-watcher") def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath, systemService: Boolean, deploy: Option[Deploy], lookupDeploy: Boolean, async: Boolean): InternalActorRef = @@ -340,10 +361,17 @@ private[akka] class RemoteActorRefProvider( override private[akka] def actorFor(ref: InternalActorRef, path: Iterable[String]): InternalActorRef = local.actorFor(ref, path) - def rootGuardianAt(address: Address): ActorRef = + def rootGuardianAt(address: Address): ActorRef = { if (hasAddress(address)) rootGuardian - else new RemoteActorRef(transport, transport.localAddressForRemote(address), - RootActorPath(address), Nobody, props = None, deploy = None) + else try { + new RemoteActorRef(transport, transport.localAddressForRemote(address), + RootActorPath(address), Nobody, props = None, deploy = None) + } catch { + case NonFatal(e) ⇒ + log.error(e, "No root guardian at [{}]", address) + new EmptyLocalActorRef(this, RootActorPath(address), eventStream) + } + } /** * INTERNAL API @@ -352,17 +380,35 @@ private[akka] class RemoteActorRefProvider( private[akka] def resolveActorRefWithLocalAddress(path: String, localAddress: Address): InternalActorRef = { path match { case ActorPathExtractor(address, elems) ⇒ - if (hasAddress(address)) local.resolveActorRef(rootGuardian, elems) - else - new RemoteActorRef(transport, localAddress, RootActorPath(address) / elems, - Nobody, props = None, deploy = None) + if (hasAddress(address)) + local.resolveActorRef(rootGuardian, elems) + else try { + new RemoteActorRef(transport, localAddress, RootActorPath(address) / elems, Nobody, props = None, deploy = None) + } catch { + case NonFatal(e) ⇒ + log.warning("Error while resolving ActorRef [{}] due to [{}]", path, e.getMessage) + new EmptyLocalActorRef(this, RootActorPath(address) / elems, eventStream) + } case _ ⇒ log.debug("resolve of unknown path [{}] failed", path) deadLetters } } - def resolveActorRef(path: String): ActorRef = path match { + def resolveActorRef(path: String): ActorRef = { + // using thread local LRU cache, which will call internalRresolveActorRef + // if the value is not cached + actorRefResolveThreadLocalCache match { + case null ⇒ internalResolveActorRef(path) // not initalized yet + case c ⇒ c.threadLocalCache(this).getOrCompute(path) + } + } + + /** + * INTERNAL API: This is used by the `ActorRefResolveCache` via the + * public `resolveActorRef(path: String)`. + */ + private[akka] def internalResolveActorRef(path: String): ActorRef = path match { case ActorPathExtractor(address, elems) ⇒ if (hasAddress(address)) local.resolveActorRef(rootGuardian, elems) else { @@ -372,7 +418,7 @@ private[akka] class RemoteActorRefProvider( rootPath, Nobody, props = None, deploy = None) } catch { case NonFatal(e) ⇒ - log.warning("Error while resolving address [{}] due to [{}]", rootPath.address, e.getMessage) + log.warning("Error while resolving ActorRef [{}] due to [{}]", path, e.getMessage) new EmptyLocalActorRef(this, rootPath, eventStream) } } @@ -388,7 +434,7 @@ private[akka] class RemoteActorRefProvider( path, Nobody, props = None, deploy = None) } catch { case NonFatal(e) ⇒ - log.error(e, "Error while resolving address [{}]", path.address) + log.warning("Error while resolving ActorRef [{}] due to [{}]", path, e.getMessage) new EmptyLocalActorRef(this, path, eventStream) } } @@ -423,11 +469,13 @@ private[akka] class RemoteActorRefProvider( /** * Marks a remote system as out of sync and prevents reconnects until the quarantine timeout elapses. + * * @param address Address of the remote system to be quarantined * @param uid UID of the remote system, if the uid is not defined it will not be a strong quarantine but * the current endpoint writer will be stopped (dropping system messages) and the address will be gated */ - def quarantine(address: Address, uid: Option[Int]): Unit = transport.quarantine(address, uid) + def quarantine(address: Address, uid: Option[Long], reason: String): Unit = + transport.quarantine(address, uid, reason) } @@ -449,6 +497,19 @@ private[akka] class RemoteActorRef private[akka] ( deploy: Option[Deploy]) extends InternalActorRef with RemoteRef { + remote match { + case t: ArteryTransport ⇒ + // detect mistakes such as using "akka.tcp" with Artery + if (path.address.protocol != t.localAddress.address.protocol) + throw new IllegalArgumentException( + s"Wrong protocol of [${path}], expected [${t.localAddress.address.protocol}]") + case _ ⇒ + } + @volatile private[remote] var cachedAssociation: artery.Association = null + + // used by artery to direct messages to separate specialized streams + @volatile private[remote] var cachedSendQueueIndex: Int = -1 + def getChild(name: Iterator[String]): InternalActorRef = { val s = name.toStream s.headOption match { @@ -461,12 +522,14 @@ private[akka] class RemoteActorRef private[akka] ( @deprecated("Use context.watch(actor) and receive Terminated(actor)", "2.2") override private[akka] def isTerminated: Boolean = false - private def handleException: Catcher[Unit] = { + private def handleException(message: Any, sender: ActorRef): Catcher[Unit] = { case e: InterruptedException ⇒ remote.system.eventStream.publish(Error(e, path.toString, getClass, "interrupted during message send")) + remote.system.deadLetters.tell(message, sender) Thread.currentThread.interrupt() case NonFatal(e) ⇒ remote.system.eventStream.publish(Error(e, path.toString, getClass, "swallowing exception during message send")) + remote.system.deadLetters.tell(message, sender) } /** @@ -491,13 +554,13 @@ private[akka] class RemoteActorRef private[akka] ( //Unwatch has a different signature, need to pattern match arguments against InternalActorRef case Unwatch(watchee: InternalActorRef, watcher: InternalActorRef) if isWatchIntercepted(watchee, watcher) ⇒ provider.remoteWatcher ! RemoteWatcher.UnwatchRemote(watchee, watcher) - case _ ⇒ remote.send(message, None, this) + case _ ⇒ remote.send(message, OptionVal.None, this) } - } catch handleException + } catch handleException(message, Actor.noSender) override def !(message: Any)(implicit sender: ActorRef = Actor.noSender): Unit = { if (message == null) throw new InvalidMessageException("Message is null") - try remote.send(message, Option(sender), this) catch handleException + try remote.send(message, OptionVal(sender), this) catch handleException(message, sender) } override def provider: RemoteActorRefProvider = remote.provider diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala index 618b71aa3e..847821dbcc 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala @@ -4,6 +4,7 @@ package akka.remote +import scala.concurrent.duration._ import scala.annotation.tailrec import scala.util.control.NonFatal import akka.actor.{ VirtualPathContainer, Deploy, Props, Nobody, InternalActorRef, ActorSystemImpl, ActorRef, ActorPathExtractor, ActorPath, Actor, AddressTerminated } @@ -22,6 +23,7 @@ import akka.actor.EmptyLocalActorRef import akka.event.AddressTerminatedTopic import java.util.concurrent.ConcurrentHashMap import akka.dispatch.sysmsg.Unwatch +import akka.NotUsed /** * INTERNAL API @@ -57,6 +59,7 @@ private[akka] class RemoteSystemDaemon( AddressTerminatedTopic(system).subscribe(this) private val parent2children = new ConcurrentHashMap[ActorRef, Set[ActorRef]] + private val dedupDaemonMsgCreateMessages = new ConcurrentHashMap[String, NotUsed] @tailrec private def addChildParentNeedsWatch(parent: ActorRef, child: ActorRef): Boolean = parent2children.get(parent) match { @@ -138,32 +141,41 @@ private[akka] class RemoteSystemDaemon( case message: DaemonMsg ⇒ log.debug("Received command [{}] to RemoteSystemDaemon on [{}]", message, path.address) message match { - case DaemonMsgCreate(_, _, path, _) if untrustedMode ⇒ log.debug("does not accept deployments (untrusted) for [{}]", path) + case DaemonMsgCreate(_, _, path, _) if untrustedMode ⇒ + log.debug("does not accept deployments (untrusted) for [{}]", path) case DaemonMsgCreate(props, deploy, path, supervisor) ⇒ - path match { - case ActorPathExtractor(address, elems) if elems.nonEmpty && elems.head == "remote" ⇒ - // TODO RK currently the extracted “address” is just ignored, is that okay? - // TODO RK canonicalize path so as not to duplicate it always #1446 - val subpath = elems.drop(1) - val p = this.path / subpath - val childName = { - val s = subpath.mkString("/") - val i = s.indexOf('#') - if (i < 0) s - else s.substring(0, i) - } - val isTerminating = !terminating.whileOff { - val parent = supervisor.asInstanceOf[InternalActorRef] - val actor = system.provider.actorOf(system, props, parent, - p, systemService = false, Some(deploy), lookupDeploy = true, async = false) - addChild(childName, actor) - actor.sendSystemMessage(Watch(actor, this)) - actor.start() - if (addChildParentNeedsWatch(parent, actor)) parent.sendSystemMessage(Watch(parent, this)) - } - if (isTerminating) log.error("Skipping [{}] to RemoteSystemDaemon on [{}] while terminating", message, p.address) - case _ ⇒ - log.debug("remote path does not match path from message [{}]", message) + // Artery sends multiple DaemonMsgCreate over several streams to preserve ordering assumptions, + // DaemonMsgCreate for this unique path is already handled and therefore deduplicated + if (dedupDaemonMsgCreateMessages.putIfAbsent(path, NotUsed) == null) { + // we only need to keep the dedup info for a short period + // this is not a real actor, so no point in scheduling message + system.scheduler.scheduleOnce(5.seconds)(dedupDaemonMsgCreateMessages.remove(path))(system.dispatcher) + + path match { + case ActorPathExtractor(address, elems) if elems.nonEmpty && elems.head == "remote" ⇒ + // TODO RK currently the extracted “address” is just ignored, is that okay? + // TODO RK canonicalize path so as not to duplicate it always #1446 + val subpath = elems.drop(1) + val p = this.path / subpath + val childName = { + val s = subpath.mkString("/") + val i = s.indexOf('#') + if (i < 0) s + else s.substring(0, i) + } + val isTerminating = !terminating.whileOff { + val parent = supervisor.asInstanceOf[InternalActorRef] + val actor = system.provider.actorOf(system, props, parent, + p, systemService = false, Some(deploy), lookupDeploy = true, async = false) + addChild(childName, actor) + actor.sendSystemMessage(Watch(actor, this)) + actor.start() + if (addChildParentNeedsWatch(parent, actor)) parent.sendSystemMessage(Watch(parent, this)) + } + if (isTerminating) log.error("Skipping [{}] to RemoteSystemDaemon on [{}] while terminating", message, p.address) + case _ ⇒ + log.debug("remote path does not match path from message [{}]", message) + } } } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index 33a5a01d34..1ffd06ecca 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -13,11 +13,14 @@ import akka.actor.Props import akka.event.Logging import akka.event.Logging.LogLevel import akka.ConfigurationException +import akka.remote.artery.ArterySettings final class RemoteSettings(val config: Config) { import config._ import scala.collection.JavaConverters._ + val Artery = ArterySettings(getConfig("akka.remote.artery")) + val LogReceive: Boolean = getBoolean("akka.remote.log-received-messages") val LogSend: Boolean = getBoolean("akka.remote.log-sent-messages") diff --git a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala index 55ce049ffe..c3cc61cf5d 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala @@ -11,6 +11,7 @@ import akka.event.{ LoggingAdapter } import scala.collection.immutable import scala.concurrent.Future import scala.util.control.NoStackTrace +import akka.util.OptionVal /** * RemoteTransportException represents a general failure within a RemoteTransport, @@ -68,7 +69,7 @@ private[akka] abstract class RemoteTransport(val system: ExtendedActorSystem, va /** * Sends the given message to the recipient supplying the sender() if any */ - def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit + def send(message: Any, senderOption: OptionVal[ActorRef], recipient: RemoteActorRef): Unit /** * Sends a management command to the underlying transport stack. The call returns with a Future that indicates @@ -89,11 +90,6 @@ private[akka] abstract class RemoteTransport(val system: ExtendedActorSystem, va * @param uid UID of the remote system, if the uid is not defined it will not be a strong quarantine but * the current endpoint writer will be stopped (dropping system messages) and the address will be gated */ - def quarantine(address: Address, uid: Option[Int]): Unit - - /** - * When this method returns true, some functionality will be turned off for security purposes. - */ - protected def useUntrustedMode: Boolean + def quarantine(address: Address, uid: Option[Long], reason: String): Unit } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteWatcher.scala b/akka-remote/src/main/scala/akka/remote/RemoteWatcher.scala index 1da2c8c797..6df02b24f1 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteWatcher.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteWatcher.scala @@ -8,6 +8,7 @@ import akka.actor._ import akka.dispatch.sysmsg.{ DeathWatchNotification, Watch } import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics } import akka.event.AddressTerminatedTopic +import akka.remote.artery.ArteryMessage import scala.collection.mutable import scala.concurrent.duration._ @@ -31,8 +32,12 @@ private[akka] object RemoteWatcher { final case class WatchRemote(watchee: InternalActorRef, watcher: InternalActorRef) final case class UnwatchRemote(watchee: InternalActorRef, watcher: InternalActorRef) - @SerialVersionUID(1L) case object Heartbeat extends PriorityMessage - @SerialVersionUID(1L) final case class HeartbeatRsp(addressUid: Int) extends PriorityMessage + @SerialVersionUID(1L) case object Heartbeat extends HeartbeatMessage + @SerialVersionUID(1L) final case class HeartbeatRsp(addressUid: Int) extends HeartbeatMessage + + // specific pair of messages for artery to allow for protobuf serialization and long uid + case object ArteryHeartbeat extends HeartbeatMessage with ArteryMessage + final case class ArteryHeartbeatRsp(uid: Long) extends HeartbeatMessage with ArteryMessage // sent to self only case object HeartbeatTick @@ -89,13 +94,12 @@ private[akka] class RemoteWatcher( import context.dispatcher def scheduler = context.system.scheduler - val remoteProvider: RemoteActorRefProvider = context.system.asInstanceOf[ExtendedActorSystem].provider match { - case rarp: RemoteActorRefProvider ⇒ rarp - case other ⇒ throw new ConfigurationException( - s"ActorSystem [${context.system}] needs to have a 'RemoteActorRefProvider' enabled in the configuration, currently uses [${other.getClass.getName}]") - } + val remoteProvider: RemoteActorRefProvider = RARP(context.system).provider + val artery = remoteProvider.remoteSettings.Artery.Enabled - val selfHeartbeatRspMsg = HeartbeatRsp(AddressUidExtension(context.system).addressUid) + val (heartBeatMsg, selfHeartbeatRspMsg) = + if (artery) (ArteryHeartbeat, ArteryHeartbeatRsp(AddressUidExtension(context.system).longAddressUid)) + else (Heartbeat, HeartbeatRsp(AddressUidExtension(context.system).addressUid)) // actors that this node is watching, map of watchee -> Set(watchers) val watching = new mutable.HashMap[InternalActorRef, mutable.Set[InternalActorRef]]() with mutable.MultiMap[InternalActorRef, InternalActorRef] @@ -105,7 +109,7 @@ private[akka] class RemoteWatcher( def watchingNodes = watcheeByNodes.keySet var unreachable: Set[Address] = Set.empty - var addressUids: Map[Address, Int] = Map.empty + var addressUids: Map[Address, Long] = Map.empty val heartbeatTask = scheduler.schedule(heartbeatInterval, heartbeatInterval, self, HeartbeatTick) val failureDetectorReaperTask = scheduler.schedule(unreachableReaperInterval, unreachableReaperInterval, @@ -119,8 +123,9 @@ private[akka] class RemoteWatcher( def receive = { case HeartbeatTick ⇒ sendHeartbeat() - case Heartbeat ⇒ receiveHeartbeat() - case HeartbeatRsp(uid) ⇒ receiveHeartbeatRsp(uid) + case Heartbeat | ArteryHeartbeat ⇒ receiveHeartbeat() + case HeartbeatRsp(uid) ⇒ receiveHeartbeatRsp(uid.toLong) + case ArteryHeartbeatRsp(uid) ⇒ receiveHeartbeatRsp(uid) case ReapUnreachableTick ⇒ reapUnreachable() case ExpectedFirstHeartbeat(from) ⇒ triggerFirstHeartbeat(from) case WatchRemote(watchee, watcher) ⇒ addWatch(watchee, watcher) @@ -138,7 +143,7 @@ private[akka] class RemoteWatcher( def receiveHeartbeat(): Unit = sender() ! selfHeartbeatRspMsg - def receiveHeartbeatRsp(uid: Int): Unit = { + def receiveHeartbeatRsp(uid: Long): Unit = { val from = sender().path.address if (failureDetector.isMonitoring(from)) @@ -158,7 +163,7 @@ private[akka] class RemoteWatcher( watchingNodes foreach { a ⇒ if (!unreachable(a) && !failureDetector.isAvailable(a)) { log.warning("Detected unreachable: [{}]", a) - quarantine(a, addressUids.get(a)) + quarantine(a, addressUids.get(a), "Deemed unreachable by remote failure detector") publishAddressTerminated(a) unreachable += a } @@ -167,8 +172,8 @@ private[akka] class RemoteWatcher( def publishAddressTerminated(address: Address): Unit = AddressTerminatedTopic(context.system).publish(AddressTerminated(address)) - def quarantine(address: Address, uid: Option[Int]): Unit = - remoteProvider.quarantine(address, uid) + def quarantine(address: Address, uid: Option[Long], reason: String): Unit = + remoteProvider.quarantine(address, uid, reason) def addWatch(watchee: InternalActorRef, watcher: InternalActorRef): Unit = { assert(watcher != self) @@ -256,7 +261,7 @@ private[akka] class RemoteWatcher( // other side a chance to reply, and also trigger some resends if needed scheduler.scheduleOnce(heartbeatExpectedResponseAfter, self, ExpectedFirstHeartbeat(a)) } - context.actorSelection(RootActorPath(a) / self.path.elements) ! Heartbeat + context.actorSelection(RootActorPath(a) / self.path.elements) ! heartBeatMsg } } @@ -282,4 +287,4 @@ private[akka] class RemoteWatcher( log.debug("Re-watch [{} -> {}]", watcher.path, watchee.path) watchee.sendSystemMessage(Watch(watchee, watcher)) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ } -} \ No newline at end of file +} diff --git a/akka-remote/src/main/scala/akka/remote/Remoting.scala b/akka-remote/src/main/scala/akka/remote/Remoting.scala index 5b54b33148..127c04ac9b 100644 --- a/akka-remote/src/main/scala/akka/remote/Remoting.scala +++ b/akka-remote/src/main/scala/akka/remote/Remoting.scala @@ -24,6 +24,7 @@ import akka.remote.transport.AkkaPduCodec.Message import java.util.concurrent.ConcurrentHashMap import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics } import akka.util.ByteString.UTF_8 +import akka.util.OptionVal /** * INTERNAL API @@ -35,13 +36,13 @@ private[remote] object AddressUrlEncoder { /** * INTERNAL API */ -private[remote] final case class RARP(provider: RemoteActorRefProvider) extends Extension { +private[akka] final case class RARP(provider: RemoteActorRefProvider) extends Extension { def configureDispatcher(props: Props): Props = provider.remoteSettings.configureDispatcher(props) } /** * INTERNAL API */ -private[remote] object RARP extends ExtensionId[RARP] with ExtensionIdProvider { +private[akka] object RARP extends ExtensionId[RARP] with ExtensionIdProvider { override def lookup() = RARP @@ -53,9 +54,16 @@ private[remote] object RARP extends ExtensionId[RARP] with ExtensionIdProvider { * Messages marked with this trait will be sent before other messages when buffering is active. * This means that these messages don't obey normal message ordering. * It is used for failure detector heartbeat messages. + * + * In Artery this is not used, and instead a preconfigured set of destinations select the priority lane. */ private[akka] trait PriorityMessage +/** + * Failure detector heartbeat messages are marked with this trait. + */ +private[akka] trait HeartbeatMessage extends PriorityMessage + /** * INTERNAL API */ @@ -209,7 +217,7 @@ private[remote] class Remoting(_system: ExtendedActorSystem, _provider: RemoteAc } } - override def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = endpointManager match { + override def send(message: Any, senderOption: OptionVal[ActorRef], recipient: RemoteActorRef): Unit = endpointManager match { case Some(manager) ⇒ manager.tell(Send(message, senderOption, recipient), sender = senderOption getOrElse Actor.noSender) case None ⇒ throw new RemoteTransportExceptionNoStackTrace("Attempted to send remote message but Remoting is not running.", null) } @@ -222,15 +230,13 @@ private[remote] class Remoting(_system: ExtendedActorSystem, _provider: RemoteAc case None ⇒ throw new RemoteTransportExceptionNoStackTrace("Attempted to send management command but Remoting is not running.", null) } - override def quarantine(remoteAddress: Address, uid: Option[Int]): Unit = endpointManager match { - case Some(manager) ⇒ manager ! Quarantine(remoteAddress, uid) + override def quarantine(remoteAddress: Address, uid: Option[Long], reason: String): Unit = endpointManager match { + case Some(manager) ⇒ + manager ! Quarantine(remoteAddress, uid.map(_.toInt)) case _ ⇒ throw new RemoteTransportExceptionNoStackTrace( - s"Attempted to quarantine address [$remoteAddress] with uid [$uid] but Remoting is not running", null) + s"Attempted to quarantine address [$remoteAddress] with UID [$uid] but Remoting is not running", null) } - // Not used anywhere only to keep compatibility with RemoteTransport interface - protected def useUntrustedMode: Boolean = provider.remoteSettings.UntrustedMode - private[akka] def boundAddresses: Map[String, Set[Address]] = { transportMapping.map { case (scheme, transports) ⇒ @@ -252,7 +258,7 @@ private[remote] object EndpointManager { final case class Listen(addressesPromise: Promise[Seq[(AkkaProtocolTransport, Address)]]) extends RemotingCommand case object StartupFinished extends RemotingCommand case object ShutdownAndFlush extends RemotingCommand - final case class Send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef, seqOpt: Option[SeqNo] = None) + final case class Send(message: Any, senderOption: OptionVal[ActorRef], recipient: RemoteActorRef, seqOpt: Option[SeqNo] = None) extends RemotingCommand with HasSequenceNumber { override def toString = s"Remote message $senderOption -> $recipient" diff --git a/akka-remote/src/main/scala/akka/remote/RemotingLifecycleEvent.scala b/akka-remote/src/main/scala/akka/remote/RemotingLifecycleEvent.scala index 1f8350640d..c57a576078 100644 --- a/akka-remote/src/main/scala/akka/remote/RemotingLifecycleEvent.scala +++ b/akka-remote/src/main/scala/akka/remote/RemotingLifecycleEvent.scala @@ -4,9 +4,11 @@ package akka.remote import akka.event.Logging.LogLevel -import akka.event.{ LoggingAdapter, Logging } +import akka.event.{ Logging, LoggingAdapter } import akka.actor.{ ActorSystem, Address } +import scala.runtime.AbstractFunction2 + @SerialVersionUID(1L) sealed trait RemotingLifecycleEvent extends Serializable { def logLevel: Logging.LogLevel @@ -79,13 +81,32 @@ final case class RemotingErrorEvent(cause: Throwable) extends RemotingLifecycleE override def toString: String = s"Remoting error: [${cause.getMessage}] [${Logging.stackTraceFor(cause)}]" } +// For binary compatibility +object QuarantinedEvent extends AbstractFunction2[Address, Int, QuarantinedEvent] { + + @deprecated("Use long uid apply") + def apply(address: Address, uid: Int) = new QuarantinedEvent(address, uid) +} + @SerialVersionUID(1L) -final case class QuarantinedEvent(address: Address, uid: Int) extends RemotingLifecycleEvent { +final case class QuarantinedEvent(address: Address, longUid: Long) extends RemotingLifecycleEvent { + override def logLevel: Logging.LogLevel = Logging.WarningLevel override val toString: String = - s"Association to [$address] having UID [$uid] is irrecoverably failed. UID is now quarantined and all " + + s"Association to [$address] having UID [$longUid] is irrecoverably failed. UID is now quarantined and all " + "messages to this UID will be delivered to dead letters. Remote actorsystem must be restarted to recover " + "from this situation." + + // For binary compatibility + + @deprecated("Use long uid constructor") + def this(address: Address, uid: Int) = this(address, uid.toLong) + + @deprecated("Use long uid") + def uid: Int = longUid.toInt + + @deprecated("Use long uid copy method") + def copy(address: Address = address, uid: Int = uid) = new QuarantinedEvent(address, uid) } @SerialVersionUID(1L) diff --git a/akka-remote/src/main/scala/akka/remote/UniqueAddress.scala b/akka-remote/src/main/scala/akka/remote/UniqueAddress.scala new file mode 100644 index 0000000000..d8ababfd03 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/UniqueAddress.scala @@ -0,0 +1,20 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote + +import akka.actor.Address + +@SerialVersionUID(1L) +final case class UniqueAddress(address: Address, uid: Long) extends Ordered[UniqueAddress] { + override def hashCode = java.lang.Long.hashCode(uid) + + def compare(that: UniqueAddress): Int = { + val result = Address.addressOrdering.compare(this.address, that.address) + if (result == 0) if (this.uid < that.uid) -1 else if (this.uid == that.uid) 0 else 1 + else result + } + + override def toString(): String = + address + "#" + uid +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala new file mode 100644 index 0000000000..cf6d56107c --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala @@ -0,0 +1,234 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.util.PrettyDuration.PrettyPrintableDuration +import java.nio.ByteBuffer +import java.util.concurrent.atomic.AtomicInteger +import scala.annotation.tailrec +import scala.concurrent.Future +import scala.concurrent.Promise +import scala.concurrent.duration._ +import scala.util.Failure +import scala.util.Success +import scala.util.Try +import scala.util.control.NoStackTrace +import akka.Done +import akka.stream.Attributes +import akka.stream.Inlet +import akka.stream.SinkShape +import akka.stream.stage.AsyncCallback +import akka.stream.stage.GraphStageLogic +import akka.stream.stage.GraphStageWithMaterializedValue +import akka.stream.stage.InHandler +import io.aeron.Aeron +import io.aeron.Publication +import org.agrona.concurrent.UnsafeBuffer +import org.agrona.hints.ThreadHints + +/** + * INTERNAL API + */ +private[remote] object AeronSink { + + final class GaveUpMessageException(msg: String) extends RuntimeException(msg) with NoStackTrace + + final class PublicationClosedException(msg: String) extends RuntimeException(msg) with NoStackTrace + + private val TimerCheckPeriod = 1 << 13 // 8192 + private val TimerCheckMask = TimerCheckPeriod - 1 + + private final class OfferTask(pub: Publication, var buffer: UnsafeBuffer, var msgSize: Int, onOfferSuccess: AsyncCallback[Unit], + giveUpAfter: Duration, onGiveUp: AsyncCallback[Unit], onPublicationClosed: AsyncCallback[Unit]) + extends (() ⇒ Boolean) { + val giveUpAfterNanos = giveUpAfter match { + case f: FiniteDuration ⇒ f.toNanos + case _ ⇒ -1L + } + var n = 0L + var startTime = 0L + + override def apply(): Boolean = { + if (n == 0L) { + // first invocation for this message + startTime = if (giveUpAfterNanos >= 0) System.nanoTime() else 0L + } + n += 1 + val result = pub.offer(buffer, 0, msgSize) + if (result >= 0) { + n = 0L + onOfferSuccess.invoke(()) + true + } else if (result == Publication.CLOSED) { + onPublicationClosed.invoke(()) + true + } else if (giveUpAfterNanos >= 0 && (n & TimerCheckMask) == 0 && (System.nanoTime() - startTime) > giveUpAfterNanos) { + // the task is invoked by the spinning thread, only check nanoTime each 8192th invocation + n = 0L + onGiveUp.invoke(()) + true + } else { + false + } + } + } +} + +/** + * INTERNAL API + * @param channel eg. "aeron:udp?endpoint=localhost:40123" + */ +private[remote] class AeronSink( + channel: String, + streamId: Int, + aeron: Aeron, + taskRunner: TaskRunner, + pool: EnvelopeBufferPool, + giveUpAfter: Duration, + flightRecorder: EventSink) + extends GraphStageWithMaterializedValue[SinkShape[EnvelopeBuffer], Future[Done]] { + import AeronSink._ + import TaskRunner._ + import FlightRecorderEvents._ + + val in: Inlet[EnvelopeBuffer] = Inlet("AeronSink") + override val shape: SinkShape[EnvelopeBuffer] = SinkShape(in) + + override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, Future[Done]) = { + val completed = Promise[Done]() + val logic = new GraphStageLogic(shape) with InHandler { + + private var envelopeInFlight: EnvelopeBuffer = null + private val pub = aeron.addPublication(channel, streamId) + + private var completedValue: Try[Done] = Success(Done) + + // spin between 2 to 20 depending on idleCpuLevel + private val spinning = 2 * taskRunner.idleCpuLevel + private var backoffCount = spinning + private var lastMsgSize = 0 + private val offerTask = new OfferTask(pub, null, lastMsgSize, getAsyncCallback(_ ⇒ taskOnOfferSuccess()), + giveUpAfter, getAsyncCallback(_ ⇒ onGiveUp()), getAsyncCallback(_ ⇒ onPublicationClosed())) + private val addOfferTask: Add = Add(offerTask) + + private var offerTaskInProgress = false + private var delegateTaskStartTime = 0L + private var countBeforeDelegate = 0L + + private val channelMetadata = channel.getBytes("US-ASCII") + + override def preStart(): Unit = { + setKeepGoing(true) + pull(in) + // TODO: Identify different sinks! + flightRecorder.loFreq(AeronSink_Started, channelMetadata) + } + + override def postStop(): Unit = { + try { + taskRunner.command(Remove(addOfferTask.task)) + flightRecorder.loFreq(AeronSink_TaskRunnerRemoved, channelMetadata) + pub.close() + flightRecorder.loFreq(AeronSink_PublicationClosed, channelMetadata) + } finally { + flightRecorder.loFreq(AeronSink_Stopped, channelMetadata) + completed.complete(completedValue) + } + } + + // InHandler + override def onPush(): Unit = { + envelopeInFlight = grab(in) + backoffCount = spinning + lastMsgSize = envelopeInFlight.byteBuffer.limit + flightRecorder.hiFreq(AeronSink_EnvelopeGrabbed, lastMsgSize) + publish() + } + + @tailrec private def publish(): Unit = { + val result = pub.offer(envelopeInFlight.aeronBuffer, 0, lastMsgSize) + if (result < 0) { + if (result == Publication.CLOSED) + onPublicationClosed() + else if (result == Publication.NOT_CONNECTED) + delegateBackoff() + else { + backoffCount -= 1 + if (backoffCount > 0) { + ThreadHints.onSpinWait() + publish() // recursive + } else + delegateBackoff() + } + } else { + countBeforeDelegate += 1 + onOfferSuccess() + } + } + + private def delegateBackoff(): Unit = { + // delegate backoff to shared TaskRunner + offerTaskInProgress = true + // visibility of these assignments are ensured by adding the task to the command queue + offerTask.buffer = envelopeInFlight.aeronBuffer + offerTask.msgSize = lastMsgSize + delegateTaskStartTime = System.nanoTime() + taskRunner.command(addOfferTask) + flightRecorder.hiFreq(AeronSink_DelegateToTaskRunner, countBeforeDelegate) + } + + private def taskOnOfferSuccess(): Unit = { + countBeforeDelegate = 0 + flightRecorder.hiFreq(AeronSink_ReturnFromTaskRunner, System.nanoTime() - delegateTaskStartTime) + onOfferSuccess() + } + + private def onOfferSuccess(): Unit = { + flightRecorder.hiFreq(AeronSink_EnvelopeOffered, lastMsgSize) + offerTaskInProgress = false + pool.release(envelopeInFlight) + offerTask.buffer = null + envelopeInFlight = null + + if (isClosed(in)) + completeStage() + else + pull(in) + } + + private def onGiveUp(): Unit = { + offerTaskInProgress = false + val cause = new GaveUpMessageException(s"Gave up sending message to $channel after ${giveUpAfter.pretty}.") + flightRecorder.alert(AeronSink_GaveUpEnvelope, cause.getMessage.getBytes("US-ASCII")) + completedValue = Failure(cause) + failStage(cause) + } + + private def onPublicationClosed(): Unit = { + offerTaskInProgress = false + val cause = new PublicationClosedException(s"Aeron Publication to [${channel}] was closed.") + // this is not exepected, since we didn't close the publication ourselves + flightRecorder.alert(AeronSink_PublicationClosed, channelMetadata) + completedValue = Failure(cause) + failStage(cause) + } + + override def onUpstreamFinish(): Unit = { + // flush outstanding offer before completing stage + if (!offerTaskInProgress) + super.onUpstreamFinish() + } + + override def onUpstreamFailure(cause: Throwable): Unit = { + completedValue = Failure(cause) + super.onUpstreamFailure(cause) + } + + setHandler(in, this) + } + + (logic, completed.future) + } + +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala new file mode 100644 index 0000000000..931998a8f1 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala @@ -0,0 +1,195 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.concurrent.TimeUnit +import scala.annotation.tailrec +import scala.concurrent.duration._ +import akka.stream.Attributes +import akka.stream.Outlet +import akka.stream.SourceShape +import akka.stream.stage.AsyncCallback +import akka.stream.stage.GraphStage +import akka.stream.stage.GraphStageLogic +import akka.stream.stage.OutHandler +import io.aeron.Aeron +import io.aeron.FragmentAssembler +import io.aeron.Subscription +import io.aeron.logbuffer.FragmentHandler +import io.aeron.logbuffer.Header +import org.agrona.DirectBuffer +import org.agrona.concurrent.BackoffIdleStrategy +import org.agrona.hints.ThreadHints +import akka.stream.stage.GraphStageWithMaterializedValue +import scala.util.control.NonFatal + +/** + * INTERNAL API + */ +private[remote] object AeronSource { + + private def pollTask(sub: Subscription, handler: MessageHandler, onMessage: AsyncCallback[EnvelopeBuffer]): () ⇒ Boolean = { + () ⇒ + { + handler.reset + val fragmentsRead = sub.poll(handler.fragmentsHandler, 1) + val msg = handler.messageReceived + handler.reset() // for GC + if (msg ne null) { + onMessage.invoke(msg) + true + } else + false + } + } + + class MessageHandler(pool: EnvelopeBufferPool) { + def reset(): Unit = messageReceived = null + + private[remote] var messageReceived: EnvelopeBuffer = null // private to avoid scalac warning about exposing EnvelopeBuffer + + val fragmentsHandler = new Fragments(data ⇒ messageReceived = data, pool) + } + + class Fragments(onMessage: EnvelopeBuffer ⇒ Unit, pool: EnvelopeBufferPool) extends FragmentAssembler(new FragmentHandler { + override def onFragment(aeronBuffer: DirectBuffer, offset: Int, length: Int, header: Header): Unit = { + val envelope = pool.acquire() + aeronBuffer.getBytes(offset, envelope.byteBuffer, length) + envelope.byteBuffer.flip() + onMessage(envelope) + } + }) + + trait ResourceLifecycle { + def onUnavailableImage(sessionId: Int): Unit + } +} + +/** + * INTERNAL API + * @param channel eg. "aeron:udp?endpoint=localhost:40123" + */ +private[remote] class AeronSource( + channel: String, + streamId: Int, + aeron: Aeron, + taskRunner: TaskRunner, + pool: EnvelopeBufferPool, + flightRecorder: EventSink) + extends GraphStageWithMaterializedValue[SourceShape[EnvelopeBuffer], AeronSource.ResourceLifecycle] { + import AeronSource._ + import TaskRunner._ + import FlightRecorderEvents._ + + val out: Outlet[EnvelopeBuffer] = Outlet("AeronSource") + override val shape: SourceShape[EnvelopeBuffer] = SourceShape(out) + + override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = { + val logic = new GraphStageLogic(shape) with OutHandler with ResourceLifecycle { + + private val sub = aeron.addSubscription(channel, streamId) + // spin between 100 to 10000 depending on idleCpuLevel + private val spinning = 1100 * taskRunner.idleCpuLevel - 1000 + private var backoffCount = spinning + private var delegateTaskStartTime = 0L + private var countBeforeDelegate = 0L + + // the fragmentHandler is called from `poll` in same thread, i.e. no async callback is needed + private val messageHandler = new MessageHandler(pool) + private val addPollTask: Add = Add(pollTask(sub, messageHandler, getAsyncCallback(taskOnMessage))) + + private val channelMetadata = channel.getBytes("US-ASCII") + + private var delegatingToTaskRunner = false + + private var pendingUnavailableImages: List[Int] = Nil + private val onUnavailableImageCb = getAsyncCallback[Int] { sessionId ⇒ + pendingUnavailableImages = sessionId :: pendingUnavailableImages + freeSessionBuffers() + } + + override def preStart(): Unit = { + flightRecorder.loFreq(AeronSource_Started, channelMetadata) + } + + override def postStop(): Unit = { + sub.close() + taskRunner.command(Remove(addPollTask.task)) + flightRecorder.loFreq(AeronSource_Stopped, channelMetadata) + } + + // OutHandler + override def onPull(): Unit = { + backoffCount = spinning + subscriberLoop() + } + + @tailrec private def subscriberLoop(): Unit = { + messageHandler.reset() + val fragmentsRead = sub.poll(messageHandler.fragmentsHandler, 1) + val msg = messageHandler.messageReceived + messageHandler.reset() // for GC + if (fragmentsRead > 0) { + countBeforeDelegate += 1 + if (msg ne null) + onMessage(msg) + else + subscriberLoop() // recursive, read more fragments + } else { + backoffCount -= 1 + if (backoffCount > 0) { + ThreadHints.onSpinWait() + subscriberLoop() // recursive + } else { + // delegate backoff to shared TaskRunner + flightRecorder.hiFreq(AeronSource_DelegateToTaskRunner, countBeforeDelegate) + delegatingToTaskRunner = true + delegateTaskStartTime = System.nanoTime() + taskRunner.command(addPollTask) + } + } + } + + private def taskOnMessage(data: EnvelopeBuffer): Unit = { + countBeforeDelegate = 0 + delegatingToTaskRunner = false + flightRecorder.hiFreq(AeronSource_ReturnFromTaskRunner, System.nanoTime() - delegateTaskStartTime) + freeSessionBuffers() + onMessage(data) + } + + private def onMessage(data: EnvelopeBuffer): Unit = { + flightRecorder.hiFreq(AeronSource_Received, data.byteBuffer.limit) + push(out, data) + } + + private def freeSessionBuffers(): Unit = + if (!delegatingToTaskRunner) { + def loop(remaining: List[Int]): Unit = { + remaining match { + case Nil ⇒ + case sessionId :: tail ⇒ + messageHandler.fragmentsHandler.freeSessionBuffer(sessionId) + loop(tail) + } + } + + loop(pendingUnavailableImages) + pendingUnavailableImages = Nil + } + + // External callback from ResourceLifecycle + def onUnavailableImage(sessionId: Int): Unit = + try { + onUnavailableImageCb.invoke(sessionId) + } catch { + case NonFatal(_) ⇒ // just in case it's called before stage is initialized, ignore + } + + setHandler(out, this) + } + + (logic, logic) + } +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala new file mode 100644 index 0000000000..23fee6d16c --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala @@ -0,0 +1,181 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.japi.Util.immutableSeq +import akka.ConfigurationException +import akka.event.Logging +import akka.event.Logging.LogLevel +import akka.stream.ActorMaterializerSettings +import akka.util.Helpers.{ ConfigOps, Requiring, toRootLowerCase } +import akka.util.WildcardIndex +import akka.NotUsed +import com.typesafe.config.Config + +import scala.collection.JavaConverters._ +import scala.concurrent.duration._ +import java.net.InetAddress +import java.nio.file.Path +import java.util.concurrent.TimeUnit + +/** INTERNAL API */ +private[akka] final class ArterySettings private (config: Config) { + import config._ + import ArterySettings._ + + val Enabled: Boolean = getBoolean("enabled") + + object Canonical { + val config = getConfig("canonical") + import config._ + + val Port: Int = getInt("port").requiring(port ⇒ + 0 to 65535 contains port, "canonical.port must be 0 through 65535") + val Hostname: String = getHostname("hostname", config) + } + + object Bind { + val config = getConfig("bind") + import config._ + + val Port: Int = getString("port") match { + case "" ⇒ Canonical.Port + case other ⇒ getInt("port").requiring(port ⇒ 0 to 65535 contains port, "bind.port must be 0 through 65535") + } + val Hostname: String = getHostname("hostname", config) match { + case "" ⇒ Canonical.Hostname + case other ⇒ other + } + } + + val LargeMessageDestinations = + config.getStringList("large-message-destinations").asScala.foldLeft(WildcardIndex[NotUsed]()) { (tree, entry) ⇒ + val segments = entry.split('/').tail + tree.insert(segments, NotUsed) + } + + val UntrustedMode: Boolean = getBoolean("untrusted-mode") + val TrustedSelectionPaths: Set[String] = immutableSeq(getStringList("trusted-selection-paths")).toSet + + val LogReceive: Boolean = getBoolean("log-received-messages") + val LogSend: Boolean = getBoolean("log-sent-messages") + + object Advanced { + val config = getConfig("advanced") + import config._ + + val TestMode: Boolean = getBoolean("test-mode") + + val Dispatcher = getString("use-dispatcher") + val ControlStreamDispatcher = getString("use-control-stream-dispatcher") + val MaterializerSettings = { + val settings = ActorMaterializerSettings(config.getConfig("materializer")) + if (Dispatcher.isEmpty) settings + else settings.withDispatcher(Dispatcher) + } + val ControlStreamMaterializerSettings = { + val settings = ActorMaterializerSettings(config.getConfig("materializer")) + if (ControlStreamDispatcher.isEmpty) settings + else settings.withDispatcher(ControlStreamDispatcher) + } + + val EmbeddedMediaDriver = getBoolean("embedded-media-driver") + val AeronDirectoryName = getString("aeron-dir") requiring (dir ⇒ + EmbeddedMediaDriver || dir.nonEmpty, "aeron-dir must be defined when using external media driver") + val DeleteAeronDirectory = getBoolean("delete-aeron-dir") + val IdleCpuLevel: Int = getInt("idle-cpu-level").requiring(level ⇒ + 1 <= level && level <= 10, "idle-cpu-level must be between 1 and 10") + val OutboundLanes = getInt("outbound-lanes").requiring(n ⇒ + n > 0, "outbound-lanes must be greater than zero") + val InboundLanes = getInt("inbound-lanes").requiring(n ⇒ + n > 0, "inbound-lanes must be greater than zero") + val SysMsgBufferSize: Int = getInt("system-message-buffer-size").requiring( + _ > 0, "system-message-buffer-size must be more than zero") + val OutboundMessageQueueSize: Int = getInt("outbound-message-queue-size").requiring( + _ > 0, "outbound-message-queue-size must be more than zero") + val OutboundControlQueueSize: Int = getInt("outbound-control-queue-size").requiring( + _ > 0, "outbound-control-queue-size must be more than zero") + val OutboundLargeMessageQueueSize: Int = getInt("outbound-large-message-queue-size").requiring( + _ > 0, "outbound-large-message-queue-size must be more than zero") + val SystemMessageResendInterval = config.getMillisDuration("system-message-resend-interval").requiring(interval ⇒ + interval > Duration.Zero, "system-message-resend-interval must be more than zero") + val HandshakeTimeout = config.getMillisDuration("handshake-timeout").requiring(interval ⇒ + interval > Duration.Zero, "handshake-timeout must be more than zero") + val HandshakeRetryInterval = config.getMillisDuration("handshake-retry-interval").requiring(interval ⇒ + interval > Duration.Zero, "handshake-retry-interval must be more than zero") + val InjectHandshakeInterval = config.getMillisDuration("inject-handshake-interval").requiring(interval ⇒ + interval > Duration.Zero, "inject-handshake-interval must be more than zero") + val GiveUpMessageAfter = config.getMillisDuration("give-up-message-after").requiring(interval ⇒ + interval > Duration.Zero, "give-up-message-after must be more than zero") + val GiveUpSystemMessageAfter = config.getMillisDuration("give-up-system-message-after").requiring(interval ⇒ + interval > Duration.Zero, "give-up-system-message-after must be more than zero") + val ShutdownFlushTimeout = config.getMillisDuration("shutdown-flush-timeout").requiring(interval ⇒ + interval > Duration.Zero, "shutdown-flush-timeout must be more than zero") + val InboundRestartTimeout = config.getMillisDuration("inbound-restart-timeout").requiring(interval ⇒ + interval > Duration.Zero, "inbound-restart-timeout must be more than zero") + val InboundMaxRestarts = getInt("inbound-max-restarts") + val OutboundRestartTimeout = config.getMillisDuration("outbound-restart-timeout").requiring(interval ⇒ + interval > Duration.Zero, "outbound-restart-timeout must be more than zero") + val OutboundMaxRestarts = getInt("outbound-max-restarts") + val StopQuarantinedAfterIdle = config.getMillisDuration("stop-quarantined-after-idle").requiring(interval ⇒ + interval > Duration.Zero, "stop-quarantined-after-idle must be more than zero") + val ClientLivenessTimeout = config.getMillisDuration("client-liveness-timeout").requiring(interval ⇒ + interval > Duration.Zero, "client-liveness-timeout must be more than zero") + val ImageLivenessTimeout = config.getMillisDuration("image-liveness-timeout").requiring(interval ⇒ + interval > Duration.Zero, "image-liveness-timeout must be more than zero") + require(ImageLivenessTimeout < HandshakeTimeout, "image-liveness-timeout must be less than handshake-timeout") + val DriverTimeout = config.getMillisDuration("driver-timeout").requiring(interval ⇒ + interval > Duration.Zero, "driver-timeout must be more than zero") + val FlightRecorderEnabled: Boolean = getBoolean("flight-recorder.enabled") + val FlightRecorderDestination: String = getString("flight-recorder.destination") + val Compression = new Compression(getConfig("compression")) + + final val MaximumFrameSize: Int = math.min(getBytes("maximum-frame-size"), Int.MaxValue).toInt + .requiring(_ >= 32 * 1024, "maximum-frame-size must be greater than or equal to 32 KiB") + final val BufferPoolSize: Int = getInt("buffer-pool-size") + .requiring(_ > 0, "buffer-pool-size must be greater than 0") + final val InboundBroadcastHubBufferSize = BufferPoolSize / 2 + final val MaximumLargeFrameSize: Int = math.min(getBytes("maximum-large-frame-size"), Int.MaxValue).toInt + .requiring(_ >= 32 * 1024, "maximum-large-frame-size must be greater than or equal to 32 KiB") + final val LargeBufferPoolSize: Int = getInt("large-buffer-pool-size") + .requiring(_ > 0, "large-buffer-pool-size must be greater than 0") + } +} + +/** INTERNAL API */ +private[akka] object ArterySettings { + def apply(config: Config) = new ArterySettings(config) + + /** INTERNAL API */ + private[remote] final class Compression private[ArterySettings] (config: Config) { + import config._ + + final val Enabled = true + + object ActorRefs { + val config = getConfig("actor-refs") + import config._ + + val AdvertisementInterval = config.getMillisDuration("advertisement-interval") + val Max = getInt("max") + } + object Manifests { + val config = getConfig("manifests") + import config._ + + val AdvertisementInterval = config.getMillisDuration("advertisement-interval") + val Max = getInt("max") + } + } + object Compression { + // Compile time constants + final val Debug = false // unlocks additional very verbose debug logging of compression events (to stdout) + } + + def getHostname(key: String, config: Config) = config.getString(key) match { + case "" ⇒ InetAddress.getLocalHost.getHostAddress + case "" ⇒ InetAddress.getLocalHost.getHostName + case other ⇒ other + } +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala new file mode 100644 index 0000000000..c92684c727 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -0,0 +1,1111 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.io.File +import java.net.InetSocketAddress +import java.nio.channels.{ DatagramChannel, FileChannel } +import java.nio.file.Path +import java.util.UUID +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.{ AtomicLong, AtomicReference } +import java.util.concurrent.atomic.AtomicBoolean + +import scala.annotation.tailrec +import scala.concurrent.{ Await, Future, Promise } +import scala.concurrent.duration._ +import scala.util.Failure +import scala.util.Success +import scala.util.control.NoStackTrace +import scala.util.control.NonFatal +import akka.Done +import akka.NotUsed +import akka.actor._ +import akka.actor.Actor +import akka.actor.Cancellable +import akka.actor.Props +import akka.event.Logging +import akka.event.LoggingAdapter +import akka.remote.AddressUidExtension +import akka.remote.RemoteActorRef +import akka.remote.RemoteActorRefProvider +import akka.remote.RemoteTransport +import akka.remote.ThisActorSystemQuarantinedEvent +import akka.remote.UniqueAddress +import akka.remote.artery.ArteryTransport.ShuttingDown +import akka.remote.artery.Encoder.ChangeOutboundCompression +import akka.remote.artery.InboundControlJunction.ControlMessageObserver +import akka.remote.artery.InboundControlJunction.ControlMessageSubject +import akka.remote.artery.OutboundControlJunction.OutboundControlIngress +import akka.remote.artery.compress._ +import akka.remote.artery.compress.CompressionProtocol.CompressionMessage +import akka.remote.transport.AkkaPduCodec +import akka.remote.transport.AkkaPduProtobufCodec +import akka.remote.transport.ThrottlerTransportAdapter.Blackhole +import akka.remote.transport.ThrottlerTransportAdapter.SetThrottle +import akka.remote.transport.ThrottlerTransportAdapter.Unthrottled +import akka.stream.AbruptTerminationException +import akka.stream.ActorMaterializer +import akka.stream.KillSwitches +import akka.stream.Materializer +import akka.stream.SharedKillSwitch +import akka.stream.scaladsl.BroadcastHub +import akka.stream.scaladsl.Flow +import akka.stream.scaladsl.Keep +import akka.stream.scaladsl.Sink +import akka.stream.scaladsl.Source +import akka.util.OptionVal +import akka.util.WildcardIndex +import io.aeron._ +import io.aeron.driver.MediaDriver +import io.aeron.driver.ThreadingMode +import io.aeron.exceptions.ConductorServiceTimeoutException +import io.aeron.exceptions.DriverTimeoutException +import org.agrona.ErrorHandler +import org.agrona.IoUtil +import org.agrona.concurrent.BackoffIdleStrategy +import akka.remote.artery.Association.OutboundStreamMatValues + +/** + * INTERNAL API + * Inbound API that is used by the stream stages. + * Separate trait to facilitate testing without real transport. + */ +private[remote] trait InboundContext { + /** + * The local inbound address. + */ + def localAddress: UniqueAddress + + /** + * An inbound stage can send control message, e.g. a reply, to the origin + * address with this method. It will be sent over the control sub-channel. + */ + def sendControl(to: Address, message: ControlMessage): Unit + + /** + * Lookup the outbound association for a given address. + */ + def association(remoteAddress: Address): OutboundContext + + /** + * Lookup the outbound association for a given UID. + * Will return `OptionVal.None` if the UID is unknown, i.e. + * handshake not completed. + */ + def association(uid: Long): OptionVal[OutboundContext] + + def completeHandshake(peer: UniqueAddress): Future[Done] + + def settings: ArterySettings + +} + +/** + * INTERNAL API + */ +private[remote] object AssociationState { + def apply(): AssociationState = + new AssociationState( + incarnation = 1, + uniqueRemoteAddressPromise = Promise(), + quarantined = ImmutableLongMap.empty[QuarantinedTimestamp]) + + final case class QuarantinedTimestamp(nanoTime: Long) { + override def toString: String = + s"Quarantined ${TimeUnit.NANOSECONDS.toSeconds(System.nanoTime() - nanoTime)} seconds ago" + } +} + +/** + * INTERNAL API + */ +private[remote] final class AssociationState( + val incarnation: Int, + val uniqueRemoteAddressPromise: Promise[UniqueAddress], + val quarantined: ImmutableLongMap[AssociationState.QuarantinedTimestamp]) { + + import AssociationState.QuarantinedTimestamp + + // doesn't have to be volatile since it's only a cache changed once + private var uniqueRemoteAddressValueCache: Option[UniqueAddress] = null + + /** + * Full outbound address with UID for this association. + * Completed when by the handshake. + */ + def uniqueRemoteAddress: Future[UniqueAddress] = uniqueRemoteAddressPromise.future + + def uniqueRemoteAddressValue(): Option[UniqueAddress] = { + if (uniqueRemoteAddressValueCache ne null) + uniqueRemoteAddressValueCache + else { + uniqueRemoteAddress.value match { + case Some(Success(peer)) ⇒ + uniqueRemoteAddressValueCache = Some(peer) + uniqueRemoteAddressValueCache + case _ ⇒ None + } + } + } + + def newIncarnation(remoteAddressPromise: Promise[UniqueAddress]): AssociationState = + new AssociationState(incarnation + 1, remoteAddressPromise, quarantined) + + def newQuarantined(): AssociationState = + uniqueRemoteAddressPromise.future.value match { + case Some(Success(a)) ⇒ + new AssociationState( + incarnation, + uniqueRemoteAddressPromise, + quarantined = quarantined.updated(a.uid, QuarantinedTimestamp(System.nanoTime()))) + case _ ⇒ this + } + + def isQuarantined(): Boolean = { + uniqueRemoteAddressValue match { + case Some(a) ⇒ isQuarantined(a.uid) + case _ ⇒ false // handshake not completed yet + } + } + + def isQuarantined(uid: Long): Boolean = quarantined.contains(uid) + + override def toString(): String = { + val a = uniqueRemoteAddressPromise.future.value match { + case Some(Success(a)) ⇒ a + case Some(Failure(e)) ⇒ s"Failure(${e.getMessage})" + case None ⇒ "unknown" + } + s"AssociationState($incarnation, $a)" + } + +} + +/** + * INTERNAL API + * Outbound association API that is used by the stream stages. + * Separate trait to facilitate testing without real transport. + */ +private[remote] trait OutboundContext { + /** + * The local inbound address. + */ + def localAddress: UniqueAddress + + /** + * The outbound address for this association. + */ + def remoteAddress: Address + + def associationState: AssociationState + + def quarantine(reason: String): Unit + + /** + * An inbound stage can send control message, e.g. a HandshakeReq, to the remote + * address of this association. It will be sent over the control sub-channel. + */ + def sendControl(message: ControlMessage): Unit + + /** + * An outbound stage can listen to control messages + * via this observer subject. + */ + def controlSubject: ControlMessageSubject + + def settings: ArterySettings + +} + +/** + * INTERNAL API + */ +private[remote] object FlushOnShutdown { + def props(done: Promise[Done], timeout: FiniteDuration, + inboundContext: InboundContext, associations: Set[Association]): Props = { + require(associations.nonEmpty) + Props(new FlushOnShutdown(done, timeout, inboundContext, associations)) + } + + case object Timeout +} + +/** + * INTERNAL API + */ +private[remote] class FlushOnShutdown(done: Promise[Done], timeout: FiniteDuration, + inboundContext: InboundContext, associations: Set[Association]) extends Actor { + + var remaining = Map.empty[UniqueAddress, Int] + + val timeoutTask = context.system.scheduler.scheduleOnce(timeout, self, FlushOnShutdown.Timeout)(context.dispatcher) + + override def preStart(): Unit = { + try { + associations.foreach { a ⇒ + val acksExpected = a.sendTerminationHint(self) + a.associationState.uniqueRemoteAddressValue() match { + case Some(address) ⇒ remaining += address → acksExpected + case None ⇒ // Ignore, handshake was not completed on this association + } + } + if (remaining.valuesIterator.sum == 0) { + done.trySuccess(Done) + context.stop(self) + } + } catch { + case NonFatal(e) ⇒ + // sendTerminationHint may throw + done.tryFailure(e) + throw e + } + } + + override def postStop(): Unit = { + timeoutTask.cancel() + done.trySuccess(Done) + } + + def receive = { + case ActorSystemTerminatingAck(from) ⇒ + // Just treat unexpected acks as systems from which zero acks are expected + val acksRemaining = remaining.getOrElse(from, 0) + if (acksRemaining <= 1) { + remaining -= from + } else { + remaining = remaining.updated(from, acksRemaining - 1) + } + + if (remaining.isEmpty) + context.stop(self) + case FlushOnShutdown.Timeout ⇒ + context.stop(self) + } +} + +/** + * INTERNAL API + */ +private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: RemoteActorRefProvider) + extends RemoteTransport(_system, _provider) with InboundContext { + import ArteryTransport.AeronTerminated + import ArteryTransport.ShutdownSignal + import ArteryTransport.InboundStreamMatValues + import FlightRecorderEvents._ + + // these vars are initialized once in the start method + @volatile private[this] var _localAddress: UniqueAddress = _ + @volatile private[this] var _bindAddress: UniqueAddress = _ + @volatile private[this] var _addresses: Set[Address] = _ + @volatile private[this] var materializer: Materializer = _ + @volatile private[this] var controlMaterializer: Materializer = _ + @volatile private[this] var controlSubject: ControlMessageSubject = _ + @volatile private[this] var messageDispatcher: MessageDispatcher = _ + private[this] val mediaDriver = new AtomicReference[Option[MediaDriver]](None) + @volatile private[this] var aeron: Aeron = _ + @volatile private[this] var aeronErrorLogTask: Cancellable = _ + @volatile private[this] var areonErrorLog: AeronErrorLog = _ + + @volatile private[this] var _inboundCompressions: Option[InboundCompressions] = None + def inboundCompressions: Option[InboundCompressions] = _inboundCompressions + + def bindAddress: UniqueAddress = _bindAddress + override def localAddress: UniqueAddress = _localAddress + override def defaultAddress: Address = localAddress.address + override def addresses: Set[Address] = _addresses + override def localAddressForRemote(remote: Address): Address = defaultAddress + override val log: LoggingAdapter = Logging(system, getClass.getName) + + private val codec: AkkaPduCodec = AkkaPduProtobufCodec + private val killSwitch: SharedKillSwitch = KillSwitches.shared("transportKillSwitch") + // keyed by the streamId + private[this] val streamMatValues = new AtomicReference(Map.empty[Int, InboundStreamMatValues]) + private[this] val hasBeenShutdown = new AtomicBoolean(false) + + private val testState = new SharedTestState + + private val inboundLanes = settings.Advanced.InboundLanes + + // TODO use WildcardIndex.isEmpty when merged from master + val largeMessageChannelEnabled = + !settings.LargeMessageDestinations.wildcardTree.isEmpty || !settings.LargeMessageDestinations.doubleWildcardTree.isEmpty + + private val priorityMessageDestinations = + WildcardIndex[NotUsed]() + // These destinations are not defined in configuration because it should not + // be possible to abuse the control channel + .insert(Array("system", "remote-watcher"), NotUsed) + // these belongs to cluster and should come from there + .insert(Array("system", "cluster", "core", "daemon", "heartbeatSender"), NotUsed) + .insert(Array("system", "cluster", "heartbeatReceiver"), NotUsed) + + private def inboundChannel = s"aeron:udp?endpoint=${_bindAddress.address.host.get}:${_bindAddress.address.port.get}" + private def outboundChannel(a: Address) = s"aeron:udp?endpoint=${a.host.get}:${a.port.get}" + + private val controlStreamId = 1 + private val ordinaryStreamId = 2 + private val largeStreamId = 3 + + private val taskRunner = new TaskRunner(system, settings.Advanced.IdleCpuLevel) + + private val restartCounter = new RestartCounter(settings.Advanced.InboundMaxRestarts, settings.Advanced.InboundRestartTimeout) + + private val envelopeBufferPool = new EnvelopeBufferPool(settings.Advanced.MaximumFrameSize, settings.Advanced.BufferPoolSize) + private val largeEnvelopeBufferPool = new EnvelopeBufferPool(settings.Advanced.MaximumLargeFrameSize, settings.Advanced.LargeBufferPoolSize) + + private val inboundEnvelopePool = ReusableInboundEnvelope.createObjectPool(capacity = 16) + // The outboundEnvelopePool is shared among all outbound associations + private val outboundEnvelopePool = ReusableOutboundEnvelope.createObjectPool(capacity = + settings.Advanced.OutboundMessageQueueSize * settings.Advanced.OutboundLanes * 3) + + val (afrFileChannel, afrFile, flightRecorder) = initializeFlightRecorder() match { + case None ⇒ (None, None, None) + case Some((c, f, r)) ⇒ (Some(c), Some(f), Some(r)) + } + + def createFlightRecorderEventSink(synchr: Boolean = false): EventSink = { + flightRecorder match { + case Some(f) ⇒ + val eventSink = f.createEventSink() + if (synchr) new SynchronizedEventSink(eventSink) + else eventSink + case None ⇒ + IgnoreEventSink + } + } + + private val topLevelFREvents = + createFlightRecorderEventSink(synchr = true) + + private val associationRegistry = new AssociationRegistry( + remoteAddress ⇒ new Association( + this, + materializer, + controlMaterializer, + remoteAddress, + controlSubject, + settings.LargeMessageDestinations, + priorityMessageDestinations, + outboundEnvelopePool)) + + override def settings = provider.remoteSettings.Artery + + override def start(): Unit = { + Runtime.getRuntime.addShutdownHook(shutdownHook) + startMediaDriver() + startAeron() + topLevelFREvents.loFreq(Transport_AeronStarted, NoMetaData) + startAeronErrorLog() + topLevelFREvents.loFreq(Transport_AeronErrorLogStarted, NoMetaData) + taskRunner.start() + topLevelFREvents.loFreq(Transport_TaskRunnerStarted, NoMetaData) + + val port = + if (settings.Canonical.Port == 0) { + if (settings.Bind.Port != 0) settings.Bind.Port // if bind port is set, use bind port instead of random + else ArteryTransport.autoSelectPort(settings.Canonical.Hostname) + } else settings.Canonical.Port + + val bindPort = if (settings.Bind.Port == 0) { + if (settings.Canonical.Port == 0) port // canonical and bind ports are zero. Use random port for both + else ArteryTransport.autoSelectPort(settings.Bind.Hostname) + } else settings.Bind.Port + + _localAddress = UniqueAddress( + Address(ArteryTransport.ProtocolName, system.name, settings.Canonical.Hostname, port), + AddressUidExtension(system).longAddressUid) + _addresses = Set(_localAddress.address) + + _bindAddress = UniqueAddress( + Address(ArteryTransport.ProtocolName, system.name, settings.Bind.Hostname, bindPort), + AddressUidExtension(system).longAddressUid) + + // TODO: This probably needs to be a global value instead of an event as events might rotate out of the log + topLevelFREvents.loFreq(Transport_UniqueAddressSet, _localAddress.toString().getBytes("US-ASCII")) + + materializer = ActorMaterializer.systemMaterializer(settings.Advanced.MaterializerSettings, "remote", system) + controlMaterializer = ActorMaterializer.systemMaterializer( + settings.Advanced.MaterializerSettings, + "remoteControl", system) + + messageDispatcher = new MessageDispatcher(system, provider) + topLevelFREvents.loFreq(Transport_MaterializerStarted, NoMetaData) + + runInboundStreams() + topLevelFREvents.loFreq(Transport_StartupFinished, NoMetaData) + + log.info("Remoting started; listening on address: [{}] with UID [{}]", localAddress.address, localAddress.uid) + } + + private lazy val shutdownHook = new Thread { + override def run(): Unit = { + if (hasBeenShutdown.compareAndSet(false, true)) { + log.debug("Shutting down [{}] via shutdownHook", localAddress) + Await.result(internalShutdown(), 20.seconds) + } + } + } + + private def startMediaDriver(): Unit = { + if (settings.Advanced.EmbeddedMediaDriver) { + val driverContext = new MediaDriver.Context + if (settings.Advanced.AeronDirectoryName.nonEmpty) { + driverContext.aeronDirectoryName(settings.Advanced.AeronDirectoryName) + } else { + // create a random name but include the actor system name for easier debugging + val uniquePart = UUID.randomUUID().toString + val randomName = s"${CommonContext.AERON_DIR_PROP_DEFAULT}-${system.name}-$uniquePart" + driverContext.aeronDirectoryName(randomName) + } + driverContext.clientLivenessTimeoutNs(settings.Advanced.ClientLivenessTimeout.toNanos) + driverContext.imageLivenessTimeoutNs(settings.Advanced.ImageLivenessTimeout.toNanos) + driverContext.driverTimeoutMs(settings.Advanced.DriverTimeout.toMillis) + + val idleCpuLevel = settings.Advanced.IdleCpuLevel + if (idleCpuLevel == 10) { + driverContext + .threadingMode(ThreadingMode.DEDICATED) + .conductorIdleStrategy(new BackoffIdleStrategy(1, 1, 1, 1)) + .receiverIdleStrategy(TaskRunner.createIdleStrategy(idleCpuLevel)) + .senderIdleStrategy(TaskRunner.createIdleStrategy(idleCpuLevel)) + } else if (idleCpuLevel == 1) { + driverContext + .threadingMode(ThreadingMode.SHARED) + .sharedIdleStrategy(TaskRunner.createIdleStrategy(idleCpuLevel)) + } else if (idleCpuLevel <= 7) { + driverContext + .threadingMode(ThreadingMode.SHARED_NETWORK) + .sharedNetworkIdleStrategy(TaskRunner.createIdleStrategy(idleCpuLevel)) + } else { + driverContext + .threadingMode(ThreadingMode.DEDICATED) + .receiverIdleStrategy(TaskRunner.createIdleStrategy(idleCpuLevel)) + .senderIdleStrategy(TaskRunner.createIdleStrategy(idleCpuLevel)) + } + + val driver = MediaDriver.launchEmbedded(driverContext) + log.info("Started embedded media driver in directory [{}]", driver.aeronDirectoryName) + topLevelFREvents.loFreq(Transport_MediaDriverStarted, driver.aeronDirectoryName().getBytes("US-ASCII")) + if (!mediaDriver.compareAndSet(None, Some(driver))) { + throw new IllegalStateException("media driver started more than once") + } + } + } + + private def aeronDir: String = mediaDriver.get match { + case Some(driver) ⇒ driver.aeronDirectoryName + case None ⇒ settings.Advanced.AeronDirectoryName + } + + private def stopMediaDriver(): Unit = { + // make sure we only close the driver once or we will crash the JVM + val maybeDriver = mediaDriver.getAndSet(None) + maybeDriver.foreach { driver ⇒ + // this is only for embedded media driver + driver.close() + + try { + if (settings.Advanced.DeleteAeronDirectory) { + IoUtil.delete(new File(driver.aeronDirectoryName), false) + topLevelFREvents.loFreq(Transport_MediaFileDeleted, NoMetaData) + } + } catch { + case NonFatal(e) ⇒ + log.warning( + "Couldn't delete Aeron embedded media driver files in [{}] due to [{}]", + driver.aeronDirectoryName, e.getMessage) + } + } + } + + // TODO: Add FR events + private def startAeron(): Unit = { + val ctx = new Aeron.Context + + ctx.driverTimeoutMs(settings.Advanced.DriverTimeout.toMillis) + + ctx.availableImageHandler(new AvailableImageHandler { + override def onAvailableImage(img: Image): Unit = { + if (log.isDebugEnabled) + log.debug(s"onAvailableImage from ${img.sourceIdentity} session ${img.sessionId}") + } + }) + ctx.unavailableImageHandler(new UnavailableImageHandler { + override def onUnavailableImage(img: Image): Unit = { + if (log.isDebugEnabled) + log.debug(s"onUnavailableImage from ${img.sourceIdentity} session ${img.sessionId}") + + // freeSessionBuffer in AeronSource FragmentAssembler + streamMatValues.get.valuesIterator.foreach { + case InboundStreamMatValues(resourceLife, _) ⇒ resourceLife.onUnavailableImage(img.sessionId) + } + } + }) + + ctx.errorHandler(new ErrorHandler { + private val fatalErrorOccured = new AtomicBoolean + + override def onError(cause: Throwable): Unit = { + cause match { + case e: ConductorServiceTimeoutException ⇒ handleFatalError(e) + case e: DriverTimeoutException ⇒ handleFatalError(e) + case _: AeronTerminated ⇒ // already handled, via handleFatalError + case _ ⇒ + log.error(cause, s"Aeron error, ${cause.getMessage}") + } + } + + private def handleFatalError(cause: Throwable): Unit = { + if (fatalErrorOccured.compareAndSet(false, true)) { + if (!isShutdown) { + log.error(cause, "Fatal Aeron error {}. Have to terminate ActorSystem because it lost contact with the " + + "{} Aeron media driver. Possible configuration properties to mitigate the problem are " + + "'client-liveness-timeout' or 'driver-timeout'. {}", + Logging.simpleName(cause), + if (settings.Advanced.EmbeddedMediaDriver) "embedded" else "external", + cause.getMessage) + taskRunner.stop() + aeronErrorLogTask.cancel() + system.terminate() + throw new AeronTerminated(cause) + } + } else + throw new AeronTerminated(cause) + } + }) + + ctx.aeronDirectoryName(aeronDir) + aeron = Aeron.connect(ctx) + } + + // TODO Add FR Events + private def startAeronErrorLog(): Unit = { + areonErrorLog = new AeronErrorLog(new File(aeronDir, CncFileDescriptor.CNC_FILE)) + val lastTimestamp = new AtomicLong(0L) + import system.dispatcher + aeronErrorLogTask = system.scheduler.schedule(3.seconds, 5.seconds) { + if (!isShutdown) { + val newLastTimestamp = areonErrorLog.logErrors(log, lastTimestamp.get) + lastTimestamp.set(newLastTimestamp + 1) + } + } + } + + private def runInboundStreams(): Unit = { + val noCompressions = NoInboundCompressions // TODO possibly enable on other channels too https://github.com/akka/akka/pull/20546/files#r68074082 + val compressions = createInboundCompressions(this) + _inboundCompressions = Some(compressions) + + runInboundControlStream(noCompressions) // TODO should understand compressions too + runInboundOrdinaryMessagesStream(compressions) + if (largeMessageChannelEnabled) { + runInboundLargeMessagesStream() + } + } + + private def runInboundControlStream(compression: InboundCompressions): Unit = { + if (isShutdown) throw ShuttingDown + val (resourceLife, (ctrl, completed)) = + aeronSource(controlStreamId, envelopeBufferPool) + .via(inboundFlow(compression)) + .toMat(inboundControlSink)(Keep.both) + .run()(controlMaterializer) + + controlSubject = ctrl + + controlSubject.attach(new ControlMessageObserver { + override def notify(inboundEnvelope: InboundEnvelope): Unit = { + try { + inboundEnvelope.message match { + case m: CompressionMessage ⇒ + import CompressionProtocol._ + m match { + case ActorRefCompressionAdvertisement(from, table) ⇒ + if (table.originUid == localAddress.uid) { + log.debug("Incoming ActorRef compression advertisement from [{}], table: [{}]", from, table) + val a = association(from.address) + // make sure uid is same for active association + if (a.associationState.uniqueRemoteAddressValue().contains(from)) { + import system.dispatcher + a.changeActorRefCompression(table).foreach { _ ⇒ + a.sendControl(ActorRefCompressionAdvertisementAck(localAddress, table.version)) + system.eventStream.publish(Events.ReceivedActorRefCompressionTable(from, table)) + } + } + } else + log.debug( + "Discarding incoming ActorRef compression advertisement from [{}] that was " + + "prepared for another incarnation with uid [{}] than current uid [{}], table: [{}]", + from, table.originUid, localAddress.uid, table) + case ActorRefCompressionAdvertisementAck(from, tableVersion) ⇒ + _inboundCompressions.foreach(_.confirmActorRefCompressionAdvertisement(from.uid, tableVersion)) + case ClassManifestCompressionAdvertisement(from, table) ⇒ + if (table.originUid == localAddress.uid) { + log.debug("Incoming Class Manifest compression advertisement from [{}], table: [{}]", from, table) + val a = association(from.address) + // make sure uid is same for active association + if (a.associationState.uniqueRemoteAddressValue().contains(from)) { + import system.dispatcher + a.changeClassManifestCompression(table).foreach { _ ⇒ + a.sendControl(ClassManifestCompressionAdvertisementAck(localAddress, table.version)) + system.eventStream.publish(Events.ReceivedClassManifestCompressionTable(from, table)) + } + } + } else + log.debug( + "Discarding incoming Class Manifest compression advertisement from [{}] that was " + + "prepared for another incarnation with uid [{}] than current uid [{}], table: [{}]", + from, table.originUid, localAddress.uid, table) + case ClassManifestCompressionAdvertisementAck(from, tableVersion) ⇒ + inboundCompressions.foreach(_.confirmClassManifestCompressionAdvertisement(from.uid, tableVersion)) + } + + case Quarantined(from, to) if to == localAddress ⇒ + // Don't quarantine the other system here, since that will result cluster member removal + // and can result in forming two separate clusters (cluster split). + // Instead, the downing strategy should act on ThisActorSystemQuarantinedEvent, e.g. + // use it as a STONITH signal. + val lifecycleEvent = ThisActorSystemQuarantinedEvent(localAddress.address, from.address) + system.eventStream.publish(lifecycleEvent) + + case _ ⇒ // not interesting + } + } catch { + case ShuttingDown ⇒ // silence it + } + } + }) + + updateStreamMatValues(controlStreamId, resourceLife, completed) + attachStreamRestart("Inbound control stream", completed, () ⇒ runInboundControlStream(compression)) + } + + private def runInboundOrdinaryMessagesStream(compression: InboundCompressions): Unit = { + if (isShutdown) throw ShuttingDown + val (resourceLife, completed) = + if (inboundLanes == 1) { + aeronSource(ordinaryStreamId, envelopeBufferPool) + .via(inboundFlow(compression)) + .toMat(inboundSink(envelopeBufferPool))(Keep.both) + .run()(materializer) + + } else { + val hubKillSwitch = KillSwitches.shared("hubKillSwitch") + val source = aeronSource(ordinaryStreamId, envelopeBufferPool) + .via(hubKillSwitch.flow) + .via(inboundFlow(compression)) + .map(env ⇒ (env.recipient, env)) + + val (resourceLife, broadcastHub) = + source + .toMat(BroadcastHub.sink(bufferSize = settings.Advanced.InboundBroadcastHubBufferSize))(Keep.both) + .run()(materializer) + + val lane = inboundSink(envelopeBufferPool) + + // select lane based on destination, to preserve message order + val partitionFun: OptionVal[ActorRef] ⇒ Int = { + _ match { + case OptionVal.Some(r) ⇒ math.abs(r.path.uid) % inboundLanes + case OptionVal.None ⇒ 0 + } + } + + val completedValues: Vector[Future[Done]] = + (0 until inboundLanes).map { i ⇒ + broadcastHub.runWith( + // TODO replace filter with "PartitionHub" when that is implemented + // must use a tuple here because envelope is pooled and must only be touched in the selected lane + Flow[(OptionVal[ActorRef], InboundEnvelope)].collect { + case (recipient, env) if partitionFun(recipient) == i ⇒ env + } + .toMat(lane)(Keep.right))(materializer) + }(collection.breakOut) + + import system.dispatcher + val completed = Future.sequence(completedValues).map(_ ⇒ Done) + + // tear down the upstream hub part if downstream lane fails + // lanes are not completed with success by themselves so we don't have to care about onSuccess + completed.onFailure { + case reason: Throwable ⇒ hubKillSwitch.abort(reason) + } + + (resourceLife, completed) + } + + updateStreamMatValues(ordinaryStreamId, resourceLife, completed) + attachStreamRestart("Inbound message stream", completed, () ⇒ runInboundOrdinaryMessagesStream(compression)) + } + + private def runInboundLargeMessagesStream(): Unit = { + if (isShutdown) throw ShuttingDown + val disableCompression = NoInboundCompressions // no compression on large message stream for now + + val (resourceLife, completed) = aeronSource(largeStreamId, largeEnvelopeBufferPool) + .via(inboundLargeFlow(disableCompression)) + .toMat(inboundSink(largeEnvelopeBufferPool))(Keep.both) + .run()(materializer) + + updateStreamMatValues(largeStreamId, resourceLife, completed) + attachStreamRestart("Inbound large message stream", completed, () ⇒ runInboundLargeMessagesStream()) + } + + private def attachStreamRestart(streamName: String, streamCompleted: Future[Done], restart: () ⇒ Unit): Unit = { + implicit val ec = materializer.executionContext + streamCompleted.onFailure { + case ShutdownSignal ⇒ // shutdown as expected + case _: AeronTerminated ⇒ // shutdown already in progress + case cause if isShutdown ⇒ + // don't restart after shutdown, but log some details so we notice + log.error(cause, s"{} failed after shutdown. {}", streamName, cause.getMessage) + case _: AbruptTerminationException ⇒ // ActorSystem shutdown + case cause ⇒ + _inboundCompressions.foreach(_.close()) + _inboundCompressions = None + if (restartCounter.restart()) { + log.error(cause, "{} failed. Restarting it. {}", streamName, cause.getMessage) + restart() + } else { + log.error(cause, "{} failed and restarted {} times within {} seconds. Terminating system. {}", + streamName, settings.Advanced.InboundMaxRestarts, settings.Advanced.InboundRestartTimeout.toSeconds, cause.getMessage) + system.terminate() + } + } + } + + override def shutdown(): Future[Done] = { + if (hasBeenShutdown.compareAndSet(false, true)) { + log.debug("Shutting down [{}]", localAddress) + val allAssociations = associationRegistry.allAssociations + val flushing: Future[Done] = + if (allAssociations.isEmpty) Future.successful(Done) + else { + val flushingPromise = Promise[Done]() + system.systemActorOf(FlushOnShutdown.props(flushingPromise, settings.Advanced.ShutdownFlushTimeout, + this, allAssociations), "remoteFlushOnShutdown") + flushingPromise.future + } + implicit val ec = system.dispatcher + flushing.recover { case _ ⇒ Done }.flatMap(_ ⇒ internalShutdown()) + } else { + Future.successful(Done) + } + } + + private def internalShutdown(): Future[Done] = { + import system.dispatcher + + killSwitch.abort(ShutdownSignal) + topLevelFREvents.loFreq(Transport_KillSwitchPulled, NoMetaData) + for { + _ ← streamsCompleted + _ ← taskRunner.stop() + } yield { + topLevelFREvents.loFreq(Transport_Stopped, NoMetaData) + + _inboundCompressions.foreach(_.close()) + _inboundCompressions = None + + if (aeronErrorLogTask != null) { + aeronErrorLogTask.cancel() + topLevelFREvents.loFreq(Transport_AeronErrorLogTaskStopped, NoMetaData) + } + if (aeron != null) aeron.close() + if (areonErrorLog != null) areonErrorLog.close() + if (mediaDriver.get.isDefined) { + stopMediaDriver() + + } + topLevelFREvents.loFreq(Transport_FlightRecorderClose, NoMetaData) + + flightRecorder.foreach(_.close()) + afrFileChannel.foreach(_.force(true)) + afrFileChannel.foreach(_.close()) + Done + } + } + + private def updateStreamMatValues(streamId: Int, aeronSourceLifecycle: AeronSource.ResourceLifecycle, completed: Future[Done]): Unit = { + implicit val ec = materializer.executionContext + updateStreamMatValues(streamId, InboundStreamMatValues(aeronSourceLifecycle, completed.recover { case _ ⇒ Done })) + } + + @tailrec private def updateStreamMatValues(streamId: Int, values: InboundStreamMatValues): Unit = { + val prev = streamMatValues.get() + if (!streamMatValues.compareAndSet(prev, prev + (streamId → values))) { + updateStreamMatValues(streamId, values) + } + } + + /** + * Exposed for orderly shutdown purposes, can not be trusted except for during shutdown as streams may restart. + * Will complete successfully even if one of the stream completion futures failed + */ + private def streamsCompleted: Future[Done] = { + implicit val ec = system.dispatcher + for { + _ ← Future.traverse(associationRegistry.allAssociations)(_.streamsCompleted) + _ ← Future.sequence(streamMatValues.get().valuesIterator.map { + case InboundStreamMatValues(_, done) ⇒ done + }) + } yield Done + } + + private[remote] def isShutdown: Boolean = hasBeenShutdown.get() + + override def managementCommand(cmd: Any): Future[Boolean] = { + cmd match { + case SetThrottle(address, direction, Blackhole) ⇒ + testState.blackhole(localAddress.address, address, direction) + case SetThrottle(address, direction, Unthrottled) ⇒ + testState.passThrough(localAddress.address, address, direction) + } + Future.successful(true) + } + + // InboundContext + override def sendControl(to: Address, message: ControlMessage) = + try { + association(to).sendControl(message) + } catch { + case ShuttingDown ⇒ // silence it + } + + override def send(message: Any, sender: OptionVal[ActorRef], recipient: RemoteActorRef): Unit = + try { + val cached = recipient.cachedAssociation + + val a = + if (cached ne null) cached + else { + val a2 = association(recipient.path.address) + recipient.cachedAssociation = a2 + a2 + } + + a.send(message, sender, OptionVal.Some(recipient)) + } catch { + case ShuttingDown ⇒ // silence it + } + + override def association(remoteAddress: Address): Association = { + require(remoteAddress != localAddress.address, "Attempted association with self address!") + // only look at isShutdown if there wasn't already an association + // races but better than nothing + associationRegistry.association(remoteAddress) + } + + override def association(uid: Long): OptionVal[Association] = + associationRegistry.association(uid) + + override def completeHandshake(peer: UniqueAddress): Future[Done] = { + try { + val a = associationRegistry.setUID(peer) + a.completeHandshake(peer) + } catch { + case ShuttingDown ⇒ Future.successful(Done) // silence it + } + } + + override def quarantine(remoteAddress: Address, uid: Option[Long], reason: String): Unit = { + try { + association(remoteAddress).quarantine(reason, uid) + } catch { + case ShuttingDown ⇒ // silence it + } + } + + def outboundLarge(outboundContext: OutboundContext): Sink[OutboundEnvelope, Future[Done]] = + createOutboundSink(largeStreamId, outboundContext, largeEnvelopeBufferPool) + .mapMaterializedValue { case (_, d) ⇒ d } + + def outbound(outboundContext: OutboundContext): Sink[OutboundEnvelope, (ChangeOutboundCompression, Future[Done])] = + createOutboundSink(ordinaryStreamId, outboundContext, envelopeBufferPool) + + private def createOutboundSink(streamId: Int, outboundContext: OutboundContext, + bufferPool: EnvelopeBufferPool): Sink[OutboundEnvelope, (ChangeOutboundCompression, Future[Done])] = { + + outboundLane(outboundContext, bufferPool) + .toMat(aeronSink(outboundContext, streamId))(Keep.both) + } + + def aeronSink(outboundContext: OutboundContext): Sink[EnvelopeBuffer, Future[Done]] = + aeronSink(outboundContext, ordinaryStreamId) + + private def aeronSink(outboundContext: OutboundContext, streamId: Int): Sink[EnvelopeBuffer, Future[Done]] = { + val giveUpAfter = + if (streamId == controlStreamId) settings.Advanced.GiveUpSystemMessageAfter + else settings.Advanced.GiveUpMessageAfter + Sink.fromGraph(new AeronSink(outboundChannel(outboundContext.remoteAddress), streamId, aeron, taskRunner, + envelopeBufferPool, giveUpAfter, createFlightRecorderEventSink())) + } + + def outboundLane(outboundContext: OutboundContext): Flow[OutboundEnvelope, EnvelopeBuffer, ChangeOutboundCompression] = + outboundLane(outboundContext, envelopeBufferPool) + + private def outboundLane( + outboundContext: OutboundContext, + bufferPool: EnvelopeBufferPool): Flow[OutboundEnvelope, EnvelopeBuffer, ChangeOutboundCompression] = { + + Flow.fromGraph(killSwitch.flow[OutboundEnvelope]) + .via(new OutboundHandshake(system, outboundContext, outboundEnvelopePool, settings.Advanced.HandshakeTimeout, + settings.Advanced.HandshakeRetryInterval, settings.Advanced.InjectHandshakeInterval)) + .viaMat(createEncoder(bufferPool))(Keep.right) + } + + def outboundControl(outboundContext: OutboundContext): Sink[OutboundEnvelope, (OutboundControlIngress, Future[Done])] = { + + Flow.fromGraph(killSwitch.flow[OutboundEnvelope]) + .via(new OutboundHandshake(system, outboundContext, outboundEnvelopePool, settings.Advanced.HandshakeTimeout, + settings.Advanced.HandshakeRetryInterval, settings.Advanced.InjectHandshakeInterval)) + .via(new SystemMessageDelivery(outboundContext, system.deadLetters, settings.Advanced.SystemMessageResendInterval, + settings.Advanced.SysMsgBufferSize)) + // note that System messages must not be dropped before the SystemMessageDelivery stage + .via(outboundTestFlow(outboundContext)) + .viaMat(new OutboundControlJunction(outboundContext, outboundEnvelopePool))(Keep.right) + .via(createEncoder(envelopeBufferPool)) + .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), controlStreamId, aeron, taskRunner, + envelopeBufferPool, Duration.Inf, createFlightRecorderEventSink()))(Keep.both) + + // TODO we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages + } + + private def createInboundCompressions(inboundContext: InboundContext): InboundCompressions = + new InboundCompressionsImpl(system, inboundContext, settings.Advanced.Compression) + + def createEncoder(pool: EnvelopeBufferPool): Flow[OutboundEnvelope, EnvelopeBuffer, ChangeOutboundCompression] = + Flow.fromGraph(new Encoder(localAddress, system, outboundEnvelopePool, pool, settings.LogSend)) + + def aeronSource(streamId: Int, pool: EnvelopeBufferPool): Source[EnvelopeBuffer, AeronSource.ResourceLifecycle] = + Source.fromGraph(new AeronSource(inboundChannel, streamId, aeron, taskRunner, pool, + createFlightRecorderEventSink())) + + val messageDispatcherSink: Sink[InboundEnvelope, Future[Done]] = Sink.foreach[InboundEnvelope] { m ⇒ + val originAddress = m.association match { + case OptionVal.Some(a) ⇒ OptionVal.Some(a.remoteAddress) + case OptionVal.None ⇒ OptionVal.None + } + messageDispatcher.dispatch(m.recipient.get, m.message, m.sender, originAddress) + m match { + case r: ReusableInboundEnvelope ⇒ inboundEnvelopePool.release(r) + case _ ⇒ + } + } + + def createDecoder(compression: InboundCompressions, bufferPool: EnvelopeBufferPool): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { + Flow.fromGraph(new Decoder(this, system, localAddress, compression, bufferPool, + inboundEnvelopePool)) + } + + def createDeserializer(bufferPool: EnvelopeBufferPool): Flow[InboundEnvelope, InboundEnvelope, NotUsed] = + Flow.fromGraph(new Deserializer(this, system, bufferPool)) + + // Checks for termination hint messages and sends an ACK for those (not processing them further) + // Purpose of this stage is flushing, the sender can wait for the ACKs up to try flushing + // pending messages. + def terminationHintReplier(): Flow[InboundEnvelope, InboundEnvelope, NotUsed] = { + Flow[InboundEnvelope].filter { envelope ⇒ + envelope.message match { + case _: ActorSystemTerminating ⇒ + envelope.sender match { + case OptionVal.Some(snd) ⇒ snd.tell(ActorSystemTerminatingAck(localAddress), ActorRef.noSender) + case OptionVal.None ⇒ log.error("Expected sender for ActorSystemTerminating message") + } + false + case _ ⇒ true + } + } + } + + def inboundSink(bufferPool: EnvelopeBufferPool): Sink[InboundEnvelope, Future[Done]] = + Flow[InboundEnvelope] + .via(createDeserializer(bufferPool)) + .via(new InboundTestStage(this, testState, settings.Advanced.TestMode)) + .via(terminationHintReplier()) + .via(new InboundHandshake(this, inControlStream = false)) + .via(new InboundQuarantineCheck(this)) + .toMat(messageDispatcherSink)(Keep.right) + + def inboundFlow(compression: InboundCompressions): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { + Flow[EnvelopeBuffer] + .via(killSwitch.flow) + .via(createDecoder(compression, envelopeBufferPool)) + } + + def inboundLargeFlow(compression: InboundCompressions): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { + Flow[EnvelopeBuffer] + .via(killSwitch.flow) + .via(createDecoder(compression, largeEnvelopeBufferPool)) + } + + def inboundControlSink: Sink[InboundEnvelope, (ControlMessageSubject, Future[Done])] = { + Flow[InboundEnvelope] + .via(createDeserializer(envelopeBufferPool)) + .via(new InboundTestStage(this, testState, settings.Advanced.TestMode)) + .via(terminationHintReplier()) + .via(new InboundHandshake(this, inControlStream = true)) + .via(new InboundQuarantineCheck(this)) + .viaMat(new InboundControlJunction)(Keep.right) + .via(new SystemMessageAcker(this)) + .toMat(messageDispatcherSink)(Keep.both) + } + + private def initializeFlightRecorder(): Option[(FileChannel, Path, FlightRecorder)] = { + if (settings.Advanced.FlightRecorderEnabled) { + val afrFile = FlightRecorder.createFlightRecorderFile(settings.Advanced.FlightRecorderDestination) + log.info("Flight recorder enabled, output can be found in '{}'", afrFile) + + val fileChannel = FlightRecorder.prepareFileForFlightRecorder(afrFile) + Some((fileChannel, afrFile, new FlightRecorder(fileChannel))) + } else + None + } + + def outboundTestFlow(outboundContext: OutboundContext): Flow[OutboundEnvelope, OutboundEnvelope, NotUsed] = + Flow.fromGraph(new OutboundTestStage(outboundContext, testState, settings.Advanced.TestMode)) + + /** INTERNAL API: for testing only. */ + private[remote] def triggerCompressionAdvertisements(actorRef: Boolean, manifest: Boolean) = { + _inboundCompressions.foreach { + case c: InboundCompressionsImpl if actorRef || manifest ⇒ + log.info("Triggering compression table advertisement for {}", c) + if (actorRef) c.runNextActorRefAdvertisement() + if (manifest) c.runNextClassManifestAdvertisement() + case _ ⇒ + } + } + +} + +/** + * INTERNAL API + */ +private[remote] object ArteryTransport { + + val ProtocolName = "akka" + + val Version: Byte = 0 + + class AeronTerminated(e: Throwable) extends RuntimeException(e) + + object ShutdownSignal extends RuntimeException with NoStackTrace + + // thrown when the transport is shutting down and something triggers a new association + object ShuttingDown extends RuntimeException with NoStackTrace + + final case class InboundStreamMatValues( + aeronSourceLifecycle: AeronSource.ResourceLifecycle, + completed: Future[Done]) + + def autoSelectPort(hostname: String): Int = { + val socket = DatagramChannel.open().socket() + socket.bind(new InetSocketAddress(hostname, 0)) + val port = socket.getLocalPort + socket.close() + port + } + +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala new file mode 100644 index 0000000000..6527ed7e17 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -0,0 +1,796 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.Queue +import java.util.concurrent.CountDownLatch +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.atomic.AtomicReference + +import scala.annotation.tailrec +import scala.concurrent.Future +import scala.concurrent.Promise +import scala.concurrent.duration._ +import scala.concurrent.duration.FiniteDuration +import akka.{ Done, NotUsed } +import akka.actor.ActorRef +import akka.actor.ActorSelectionMessage +import akka.actor.Address +import akka.dispatch.sysmsg.SystemMessage +import akka.event.Logging +import akka.pattern.after +import akka.remote._ +import akka.remote.DaemonMsgCreate +import akka.remote.QuarantinedEvent +import akka.remote.artery.AeronSink.GaveUpMessageException +import akka.remote.artery.ArteryTransport.{ AeronTerminated, ShuttingDown } +import akka.remote.artery.Encoder.ChangeOutboundCompression +import akka.remote.artery.Encoder.ChangeOutboundCompressionFailed +import akka.remote.artery.InboundControlJunction.ControlMessageSubject +import akka.remote.artery.OutboundControlJunction.OutboundControlIngress +import akka.remote.artery.OutboundHandshake.HandshakeTimeoutException +import akka.remote.artery.SystemMessageDelivery.ClearSystemMessageDelivery +import akka.remote.artery.compress.CompressionProtocol._ +import akka.remote.artery.compress.CompressionTable +import akka.stream.AbruptTerminationException +import akka.stream.KillSwitches +import akka.stream.Materializer +import akka.stream.scaladsl.Keep +import akka.stream.scaladsl.MergeHub +import akka.stream.scaladsl.Source +import akka.util.{ Unsafe, WildcardIndex } +import akka.util.OptionVal +import org.agrona.concurrent.ManyToOneConcurrentArrayQueue +import akka.stream.SharedKillSwitch +import scala.util.control.NoStackTrace +import akka.actor.Cancellable + +/** + * INTERNAL API + */ +private[remote] object Association { + sealed trait QueueWrapper extends SendQueue.ProducerApi[OutboundEnvelope] { + def queue: Queue[OutboundEnvelope] + } + + final case class QueueWrapperImpl(queue: Queue[OutboundEnvelope]) extends QueueWrapper { + override def offer(message: OutboundEnvelope): Boolean = queue.offer(message) + + override def isEnabled: Boolean = true + } + + object DisabledQueueWrapper extends QueueWrapper { + override def queue: java.util.Queue[OutboundEnvelope] = + throw new UnsupportedOperationException("The Queue is disabled") + + override def offer(message: OutboundEnvelope): Boolean = + throw new UnsupportedOperationException("The method offer() is illegal on a disabled queue") + + override def isEnabled: Boolean = false + } + + final case class LazyQueueWrapper(queue: Queue[OutboundEnvelope], materialize: () ⇒ Unit) extends QueueWrapper { + private val onlyOnce = new AtomicBoolean + + def runMaterialize(): Unit = { + if (onlyOnce.compareAndSet(false, true)) + materialize() + } + + override def offer(message: OutboundEnvelope): Boolean = { + runMaterialize() + queue.offer(message) + } + + override def isEnabled: Boolean = true + } + + final val ControlQueueIndex = 0 + final val LargeQueueIndex = 1 + final val OrdinaryQueueIndex = 2 + + private object OutboundStreamStopSignal extends RuntimeException with NoStackTrace + + final case class OutboundStreamMatValues(streamKillSwitch: SharedKillSwitch, completed: Future[Done]) +} + +/** + * INTERNAL API + * + * Thread-safe, mutable holder for association state. Main entry point for remote destined message to a specific + * remote address. + */ +private[remote] class Association( + val transport: ArteryTransport, + val materializer: Materializer, + val controlMaterializer: Materializer, + override val remoteAddress: Address, + override val controlSubject: ControlMessageSubject, + largeMessageDestinations: WildcardIndex[NotUsed], + priorityMessageDestinations: WildcardIndex[NotUsed], + outboundEnvelopePool: ObjectPool[ReusableOutboundEnvelope]) + extends AbstractAssociation with OutboundContext { + import Association._ + import FlightRecorderEvents._ + + private val log = Logging(transport.system, getClass.getName) + private val flightRecorder = transport.createFlightRecorderEventSink(synchr = true) + + override def settings = transport.settings + private def advancedSettings = transport.settings.Advanced + + private val restartCounter = new RestartCounter(advancedSettings.OutboundMaxRestarts, advancedSettings.OutboundRestartTimeout) + + // We start with the raw wrapped queue and then it is replaced with the materialized value of + // the `SendQueue` after materialization. Using same underlying queue. This makes it possible to + // start sending (enqueuing) to the Association immediate after construction. + + def createQueue(capacity: Int): Queue[OutboundEnvelope] = + new ManyToOneConcurrentArrayQueue[OutboundEnvelope](capacity) + + private val outboundLanes = advancedSettings.OutboundLanes + private val controlQueueSize = advancedSettings.OutboundControlQueueSize + private val queueSize = advancedSettings.OutboundMessageQueueSize + private val largeQueueSize = advancedSettings.OutboundLargeMessageQueueSize + + private[this] val queues: Array[SendQueue.ProducerApi[OutboundEnvelope]] = Array.ofDim(2 + outboundLanes) + queues(ControlQueueIndex) = QueueWrapperImpl(createQueue(controlQueueSize)) // control stream + queues(LargeQueueIndex) = + if (transport.largeMessageChannelEnabled) // large messages stream + QueueWrapperImpl(createQueue(largeQueueSize)) + else + DisabledQueueWrapper + + (0 until outboundLanes).foreach { i ⇒ + queues(OrdinaryQueueIndex + i) = QueueWrapperImpl(createQueue(queueSize)) // ordinary messages stream + } + @volatile private[this] var queuesVisibility = false + + private def controlQueue: SendQueue.ProducerApi[OutboundEnvelope] = queues(ControlQueueIndex) + private def largeQueue: SendQueue.ProducerApi[OutboundEnvelope] = queues(LargeQueueIndex) + + @volatile private[this] var _outboundControlIngress: OptionVal[OutboundControlIngress] = OptionVal.None + @volatile private[this] var materializing = new CountDownLatch(1) + @volatile private[this] var changeOutboundCompression: Vector[ChangeOutboundCompression] = Vector.empty + // in case there is a restart at the same time as a compression table update + private val changeCompressionTimeout = 5.seconds + + private[remote] def changeActorRefCompression(table: CompressionTable[ActorRef]): Future[Done] = { + import transport.system.dispatcher + val c = changeOutboundCompression + val result = + if (c.isEmpty) Future.successful(Done) + else if (c.size == 1) c.head.changeActorRefCompression(table) + else Future.sequence(c.map(_.changeActorRefCompression(table))).map(_ ⇒ Done) + timeoutAfter(result, changeCompressionTimeout, new ChangeOutboundCompressionFailed) + } + // keyed by stream queue index + private[this] val streamMatValues = new AtomicReference(Map.empty[Int, OutboundStreamMatValues]) + private[this] val idle = new AtomicReference[Option[Cancellable]](None) + + private[remote] def changeClassManifestCompression(table: CompressionTable[String]): Future[Done] = { + import transport.system.dispatcher + val c = changeOutboundCompression + val result = + if (c.isEmpty) Future.successful(Done) + else if (c.size == 1) c.head.changeClassManifestCompression(table) + else Future.sequence(c.map(_.changeClassManifestCompression(table))).map(_ ⇒ Done) + timeoutAfter(result, changeCompressionTimeout, new ChangeOutboundCompressionFailed) + } + + private def clearOutboundCompression(): Future[Done] = { + import transport.system.dispatcher + val c = changeOutboundCompression + val result = + if (c.isEmpty) Future.successful(Done) + else if (c.size == 1) c.head.clearCompression() + else Future.sequence(c.map(_.clearCompression())).map(_ ⇒ Done) + timeoutAfter(result, changeCompressionTimeout, new ChangeOutboundCompressionFailed) + } + + private def clearInboundCompression(originUid: Long): Unit = + transport.inboundCompressions.foreach(_.close(originUid)) + + private def timeoutAfter[T](f: Future[T], timeout: FiniteDuration, e: ⇒ Throwable): Future[T] = { + import transport.system.dispatcher + val f2 = after(timeout, transport.system.scheduler)(Future.failed(e)) + Future.firstCompletedOf(List(f, f2)) + } + + private def deadletters = transport.system.deadLetters + + def outboundControlIngress: OutboundControlIngress = { + _outboundControlIngress match { + case OptionVal.Some(o) ⇒ o + case OptionVal.None ⇒ + controlQueue match { + case w: LazyQueueWrapper ⇒ w.runMaterialize() + case _ ⇒ + } + // the outboundControlIngress may be accessed before the stream is materialized + // using CountDownLatch to make sure that materialization is completed + materializing.await(10, TimeUnit.SECONDS) + _outboundControlIngress match { + case OptionVal.Some(o) ⇒ o + case OptionVal.None ⇒ + if (transport.isShutdown) throw ShuttingDown + else throw new IllegalStateException("outboundControlIngress not initialized yet") + } + } + } + + override def localAddress: UniqueAddress = transport.localAddress + + /** + * Holds reference to shared state of Association - *access only via helper methods* + */ + @volatile + private[this] var _sharedStateDoNotCallMeDirectly: AssociationState = AssociationState() + + /** + * Helper method for access to underlying state via Unsafe + * + * @param oldState Previous state + * @param newState Next state on transition + * @return Whether the previous state matched correctly + */ + @inline + private[this] def swapState(oldState: AssociationState, newState: AssociationState): Boolean = + Unsafe.instance.compareAndSwapObject(this, AbstractAssociation.sharedStateOffset, oldState, newState) + + /** + * @return Reference to current shared state + */ + def associationState: AssociationState = + Unsafe.instance.getObjectVolatile(this, AbstractAssociation.sharedStateOffset).asInstanceOf[AssociationState] + + def completeHandshake(peer: UniqueAddress): Future[Done] = { + require( + remoteAddress == peer.address, + s"wrong remote address in completeHandshake, got ${peer.address}, expected $remoteAddress") + val current = associationState + + current.uniqueRemoteAddressValue() match { + case Some(`peer`) ⇒ + // handshake already completed + Future.successful(Done) + case _ ⇒ + // clear outbound compression, it's safe to do that several times if someone else + // completes handshake at same time, but it's important to clear it before + // we signal that the handshake is completed (uniqueRemoteAddressPromise.trySuccess) + import transport.system.dispatcher + clearOutboundCompression().map { _ ⇒ + current.uniqueRemoteAddressPromise.trySuccess(peer) + current.uniqueRemoteAddressValue() match { + case Some(`peer`) ⇒ + // our value + case _ ⇒ + val newState = current.newIncarnation(Promise.successful(peer)) + if (swapState(current, newState)) { + current.uniqueRemoteAddressValue() match { + case Some(old) ⇒ + cancelIdleTimer() + log.debug( + "Incarnation {} of association to [{}] with new UID [{}] (old UID [{}])", + newState.incarnation, peer.address, peer.uid, old.uid) + clearInboundCompression(old.uid) + case None ⇒ + // Failed, nothing to do + } + // if swap failed someone else completed before us, and that is fine + } + } + Done + } + } + } + + // OutboundContext + override def sendControl(message: ControlMessage): Unit = { + try { + if (!transport.isShutdown) { + if (associationState.isQuarantined()) { + log.debug("Send control message [{}] to quarantined [{}]", Logging.messageClassName(message), + remoteAddress) + startIdleTimer() + } + outboundControlIngress.sendControlMessage(message) + } + } catch { + case ShuttingDown ⇒ // silence it + } + } + + def send(message: Any, sender: OptionVal[ActorRef], recipient: OptionVal[RemoteActorRef]): Unit = { + + def createOutboundEnvelope(): OutboundEnvelope = + outboundEnvelopePool.acquire().init(recipient, message.asInstanceOf[AnyRef], sender) + + // volatile read to see latest queue array + val unused = queuesVisibility + + def dropped(queueIndex: Int, qSize: Int, env: OutboundEnvelope): Unit = { + log.debug( + "Dropping message [{}] from [{}] to [{}] due to overflow of send queue, size [{}]", + Logging.messageClassName(message), sender.getOrElse(deadletters), recipient.getOrElse(recipient), qSize) + flightRecorder.hiFreq(Transport_SendQueueOverflow, queueIndex) + deadletters ! env + } + + val quarantined = associationState.isQuarantined() + + // allow ActorSelectionMessage to pass through quarantine, to be able to establish interaction with new system + if (message.isInstanceOf[ActorSelectionMessage] || !quarantined || message == ClearSystemMessageDelivery) { + if (quarantined && message != ClearSystemMessageDelivery) { + log.debug("Quarantine piercing attempt with message [{}] to [{}]", Logging.messageClassName(message), recipient.getOrElse("")) + startIdleTimer() + } + try { + message match { + case _: SystemMessage ⇒ + val outboundEnvelope = createOutboundEnvelope() + if (!controlQueue.offer(createOutboundEnvelope())) { + quarantine(reason = s"Due to overflow of control queue, size [$controlQueueSize]") + dropped(ControlQueueIndex, controlQueueSize, outboundEnvelope) + } + case ActorSelectionMessage(_: PriorityMessage, _, _) | _: ControlMessage | ClearSystemMessageDelivery ⇒ + // ActorSelectionMessage with PriorityMessage is used by cluster and remote failure detector heartbeating + val outboundEnvelope = createOutboundEnvelope() + if (!controlQueue.offer(createOutboundEnvelope())) { + dropped(ControlQueueIndex, controlQueueSize, outboundEnvelope) + } + case _: DaemonMsgCreate ⇒ + // DaemonMsgCreate is not a SystemMessage, but must be sent over the control stream because + // remote deployment process depends on message ordering for DaemonMsgCreate and Watch messages. + // It must also be sent over the ordinary message stream so that it arrives (and creates the + // destination) before the first ordinary message arrives. + val outboundEnvelope1 = createOutboundEnvelope() + if (!controlQueue.offer(outboundEnvelope1)) + dropped(ControlQueueIndex, controlQueueSize, outboundEnvelope1) + (0 until outboundLanes).foreach { i ⇒ + val outboundEnvelope2 = createOutboundEnvelope() + if (!queues(OrdinaryQueueIndex + i).offer(outboundEnvelope2)) + dropped(OrdinaryQueueIndex + i, queueSize, outboundEnvelope2) + } + case _ ⇒ + val outboundEnvelope = createOutboundEnvelope() + val queueIndex = selectQueue(recipient) + val queue = queues(queueIndex) + val offerOk = queue.offer(outboundEnvelope) + if (!offerOk) + dropped(queueIndex, queueSize, outboundEnvelope) + } + } catch { + case ShuttingDown ⇒ // silence it + } + } else if (log.isDebugEnabled) + log.debug( + "Dropping message [{}] from [{}] to [{}] due to quarantined system [{}]", + Logging.messageClassName(message), sender.getOrElse(deadletters), recipient.getOrElse(recipient), remoteAddress) + } + + private def selectQueue(recipient: OptionVal[RemoteActorRef]): Int = { + recipient match { + case OptionVal.Some(r) ⇒ + r.cachedSendQueueIndex match { + case -1 ⇒ + // only happens when messages are sent to new remote destination + // and is then cached on the RemoteActorRef + val elements = r.path.elements + val idx = + if (priorityMessageDestinations.find(elements).isDefined) { + log.debug("Using priority message stream for {}", r.path) + ControlQueueIndex + } else if (transport.largeMessageChannelEnabled && largeMessageDestinations.find(elements).isDefined) { + log.debug("Using large message stream for {}", r.path) + LargeQueueIndex + } else if (outboundLanes == 1) { + OrdinaryQueueIndex + } else { + // select lane based on destination, to preserve message order + OrdinaryQueueIndex + (math.abs(r.path.uid) % outboundLanes) + } + r.cachedSendQueueIndex = idx + idx + case idx ⇒ idx + } + + case OptionVal.None ⇒ + OrdinaryQueueIndex + } + } + + def sendTerminationHint(replyTo: ActorRef): Int = { + if (!associationState.isQuarantined()) { + val msg = ActorSystemTerminating(localAddress) + var sent = 0 + queues.iterator.filter(_.isEnabled).foreach { queue ⇒ + try { + val envelope = outboundEnvelopePool.acquire() + .init(OptionVal.None, msg, OptionVal.Some(replyTo)) + + queue.offer(envelope) + sent += 1 + } catch { + case ShuttingDown ⇒ // can be thrown if `offer` triggers new materialization + } + } + sent + } else 0 + } + + // OutboundContext + override def quarantine(reason: String): Unit = { + val uid = associationState.uniqueRemoteAddressValue().map(_.uid) + quarantine(reason, uid) + } + + @tailrec final def quarantine(reason: String, uid: Option[Long]): Unit = { + uid match { + case Some(u) ⇒ + val current = associationState + current.uniqueRemoteAddressValue() match { + case Some(peer) if peer.uid == u ⇒ + if (!current.isQuarantined(u)) { + val newState = current.newQuarantined() + if (swapState(current, newState)) { + // quarantine state change was performed + log.warning( + "Association to [{}] with UID [{}] is irrecoverably failed. UID is now quarantined and all " + + "messages to this UID will be delivered to dead letters. " + + "Remote actorsystem must be restarted to recover from this situation. {}", + remoteAddress, u, reason) + transport.system.eventStream.publish(QuarantinedEvent(remoteAddress, u)) + clearOutboundCompression() + clearInboundCompression(u) + // end delivery of system messages to that incarnation after this point + send(ClearSystemMessageDelivery, OptionVal.None, OptionVal.None) + // try to tell the other system that we have quarantined it + sendControl(Quarantined(localAddress, peer)) + startIdleTimer() + } else + quarantine(reason, uid) // recursive + } + case Some(peer) ⇒ + log.debug( + "Quarantine of [{}] ignored due to non-matching UID, quarantine requested for [{}] but current is [{}]. {}", + remoteAddress, u, peer.uid, reason) + case None ⇒ + log.debug( + "Quarantine of [{}] ignored because handshake not completed, quarantine request was for old incarnation. {}", + remoteAddress, reason) + } + case None ⇒ + log.warning("Quarantine of [{}] ignored because unknown UID", remoteAddress) + } + + } + + private def cancelIdleTimer(): Unit = { + val current = idle.get + current.foreach(_.cancel()) + idle.compareAndSet(current, None) + } + + private def startIdleTimer(): Unit = { + cancelIdleTimer() + idle.set(Some(transport.system.scheduler.scheduleOnce(advancedSettings.StopQuarantinedAfterIdle) { + if (associationState.isQuarantined()) + streamMatValues.get.valuesIterator.foreach { + case OutboundStreamMatValues(killSwitch, _) ⇒ killSwitch.abort(OutboundStreamStopSignal) + } + }(transport.system.dispatcher))) + } + + /** + * Called once after construction when the `Association` instance + * wins the CAS in the `AssociationRegistry`. It will materialize + * the streams. It is possible to sending (enqueuing) to the association + * before this method is called. + * + * @throws ShuttingDown if called while the transport is shutting down + */ + def associate(): Unit = { + if (!controlQueue.isInstanceOf[QueueWrapper]) + throw new IllegalStateException("associate() must only be called once") + runOutboundStreams() + } + + private def runOutboundStreams(): Unit = { + + // it's important to materialize the outboundControl stream first, + // so that outboundControlIngress is ready when stages for all streams start + runOutboundControlStream() + runOutboundOrdinaryMessagesStream() + + if (transport.largeMessageChannelEnabled) + runOutboundLargeMessagesStream() + } + + private def runOutboundControlStream(): Unit = { + if (transport.isShutdown) throw ShuttingDown + log.debug("Starting outbound control stream to [{}]", remoteAddress) + + val wrapper = getOrCreateQueueWrapper(ControlQueueIndex, queueSize) + queues(ControlQueueIndex) = wrapper // use new underlying queue immediately for restarts + queuesVisibility = true // volatile write for visibility of the queues array + + val streamKillSwitch = KillSwitches.shared("outboundControlStreamKillSwitch") + + val (queueValue, (control, completed)) = + Source.fromGraph(new SendQueue[OutboundEnvelope]) + .via(streamKillSwitch.flow) + .toMat(transport.outboundControl(this))(Keep.both) + .run()(materializer) + + queueValue.inject(wrapper.queue) + // replace with the materialized value, still same underlying queue + queues(ControlQueueIndex) = queueValue + queuesVisibility = true // volatile write for visibility of the queues array + _outboundControlIngress = OptionVal.Some(control) + materializing.countDown() + + updateStreamMatValues(ControlQueueIndex, streamKillSwitch, completed) + attachStreamRestart("Outbound control stream", ControlQueueIndex, controlQueueSize, + completed, () ⇒ runOutboundControlStream()) + } + + private def getOrCreateQueueWrapper(queueIndex: Int, capacity: Int): QueueWrapper = { + val unused = queuesVisibility // volatile read to see latest queues array + queues(queueIndex) match { + case existing: QueueWrapper ⇒ existing + case _ ⇒ + // use new queue for restarts + QueueWrapperImpl(createQueue(capacity)) + } + } + + private def runOutboundOrdinaryMessagesStream(): Unit = { + if (transport.isShutdown) throw ShuttingDown + if (outboundLanes == 1) { + log.debug("Starting outbound message stream to [{}]", remoteAddress) + val queueIndex = OrdinaryQueueIndex + val wrapper = getOrCreateQueueWrapper(queueIndex, queueSize) + queues(queueIndex) = wrapper // use new underlying queue immediately for restarts + queuesVisibility = true // volatile write for visibility of the queues array + + val streamKillSwitch = KillSwitches.shared("outboundMessagesKillSwitch") + + val ((queueValue, testMgmt), (changeCompression, completed)) = + Source.fromGraph(new SendQueue[OutboundEnvelope]) + .via(streamKillSwitch.flow) + .viaMat(transport.outboundTestFlow(this))(Keep.both) + .toMat(transport.outbound(this))(Keep.both) + .run()(materializer) + + queueValue.inject(wrapper.queue) + // replace with the materialized value, still same underlying queue + queues(queueIndex) = queueValue + queuesVisibility = true // volatile write for visibility of the queues array + changeOutboundCompression = Vector(changeCompression) + + updateStreamMatValues(OrdinaryQueueIndex, streamKillSwitch, completed) + attachStreamRestart("Outbound message stream", OrdinaryQueueIndex, queueSize, + completed, () ⇒ runOutboundOrdinaryMessagesStream()) + + } else { + log.debug("Starting outbound message stream to [{}] with [{}] lanes", remoteAddress, outboundLanes) + val wrappers = (0 until outboundLanes).map { i ⇒ + val wrapper = getOrCreateQueueWrapper(OrdinaryQueueIndex + i, queueSize) + queues(OrdinaryQueueIndex + i) = wrapper // use new underlying queue immediately for restarts + queuesVisibility = true // volatile write for visibility of the queues array + wrapper + }.toVector + + val streamKillSwitch = KillSwitches.shared("outboundMessagesKillSwitch") + + val lane = Source.fromGraph(new SendQueue[OutboundEnvelope]) + .via(streamKillSwitch.flow) + .via(transport.outboundTestFlow(this)) + .viaMat(transport.outboundLane(this))(Keep.both) + .watchTermination()(Keep.both) + // recover to avoid error logging by MergeHub + .recoverWithRetries(-1, { case _: Throwable ⇒ Source.empty }) + .mapMaterializedValue { + case ((q, c), w) ⇒ (q, c, w) + } + + val (mergeHub, aeronSinkCompleted) = MergeHub.source[EnvelopeBuffer] + .via(streamKillSwitch.flow) + .toMat(transport.aeronSink(this))(Keep.both).run()(materializer) + + val values: Vector[(SendQueue.QueueValue[OutboundEnvelope], Encoder.ChangeOutboundCompression, Future[Done])] = + (0 until outboundLanes).map { _ ⇒ + lane.to(mergeHub).run()(materializer) + }(collection.breakOut) + + val (queueValues, changeCompressionValues, laneCompletedValues) = values.unzip3 + + import transport.system.dispatcher + val completed = Future.sequence(laneCompletedValues).flatMap(_ ⇒ aeronSinkCompleted) + + // tear down all parts if one part fails or completes + completed.onFailure { + case reason: Throwable ⇒ streamKillSwitch.abort(reason) + } + (laneCompletedValues :+ aeronSinkCompleted).foreach(_.onSuccess { case _ ⇒ streamKillSwitch.shutdown() }) + + queueValues.zip(wrappers).zipWithIndex.foreach { + case ((q, w), i) ⇒ + q.inject(w.queue) + queues(OrdinaryQueueIndex + i) = q // replace with the materialized value, still same underlying queue + } + queuesVisibility = true // volatile write for visibility of the queues array + + changeOutboundCompression = changeCompressionValues + + attachStreamRestart("Outbound message stream", OrdinaryQueueIndex, queueSize, + completed, () ⇒ runOutboundOrdinaryMessagesStream()) + } + } + + private def runOutboundLargeMessagesStream(): Unit = { + if (transport.isShutdown) throw ShuttingDown + log.debug("Starting outbound large message stream to [{}]", remoteAddress) + val wrapper = getOrCreateQueueWrapper(LargeQueueIndex, largeQueueSize) + queues(LargeQueueIndex) = wrapper // use new underlying queue immediately for restarts + queuesVisibility = true // volatile write for visibility of the queues array + + val streamKillSwitch = KillSwitches.shared("outboundLargeMessagesKillSwitch") + + val (queueValue, completed) = Source.fromGraph(new SendQueue[OutboundEnvelope]) + .via(streamKillSwitch.flow) + .via(transport.outboundTestFlow(this)) + .toMat(transport.outboundLarge(this))(Keep.both) + .run()(materializer) + + queueValue.inject(wrapper.queue) + // replace with the materialized value, still same underlying queue + queues(LargeQueueIndex) = queueValue + queuesVisibility = true // volatile write for visibility of the queues array + + updateStreamMatValues(LargeQueueIndex, streamKillSwitch, completed) + attachStreamRestart("Outbound large message stream", LargeQueueIndex, largeQueueSize, + completed, () ⇒ runOutboundLargeMessagesStream()) + } + + private def attachStreamRestart(streamName: String, queueIndex: Int, queueCapacity: Int, + streamCompleted: Future[Done], restart: () ⇒ Unit): Unit = { + + def lazyRestart(): Unit = { + changeOutboundCompression = Vector.empty + if (queueIndex == ControlQueueIndex) { + materializing = new CountDownLatch(1) + _outboundControlIngress = OptionVal.None + } + // LazyQueueWrapper will invoke the `restart` function when first message is offered + queues(queueIndex) = LazyQueueWrapper(createQueue(queueCapacity), restart) + queuesVisibility = true // volatile write for visibility of the queues array + } + + implicit val ec = materializer.executionContext + streamCompleted.onFailure { + case ArteryTransport.ShutdownSignal ⇒ + // shutdown as expected + // countDown the latch in case threads are waiting on the latch in outboundControlIngress method + materializing.countDown() + case _: AeronTerminated ⇒ // shutdown already in progress + case cause if transport.isShutdown ⇒ + // don't restart after shutdown, but log some details so we notice + log.error(cause, s"{} to [{}] failed after shutdown. {}", streamName, remoteAddress, cause.getMessage) + // countDown the latch in case threads are waiting on the latch in outboundControlIngress method + materializing.countDown() + case _: AbruptTerminationException ⇒ // ActorSystem shutdown + case OutboundStreamStopSignal ⇒ + // stop as expected due to quarantine + log.debug("{} to [{}] stopped. It will be restarted if used again.", streamName, remoteAddress) + lazyRestart() + case cause: GaveUpMessageException ⇒ + log.debug("{} to [{}] failed. Restarting it. {}", streamName, remoteAddress, cause.getMessage) + // restart unconditionally, without counting restarts + lazyRestart() + case cause ⇒ + if (queueIndex == ControlQueueIndex) { + cause match { + case _: HandshakeTimeoutException ⇒ // ok, quarantine not possible without UID + case _ ⇒ quarantine("Outbound control stream restarted") + } + } + + if (restartCounter.restart()) { + log.error(cause, "{} to [{}] failed. Restarting it. {}", streamName, remoteAddress, cause.getMessage) + lazyRestart() + } else { + log.error(cause, s"{} to [{}] failed and restarted {} times within {} seconds. Terminating system. ${cause.getMessage}", + streamName, remoteAddress, advancedSettings.OutboundMaxRestarts, advancedSettings.OutboundRestartTimeout.toSeconds) + transport.system.terminate() + } + } + } + + private def updateStreamMatValues(streamId: Int, streamKillSwitch: SharedKillSwitch, completed: Future[Done]): Unit = { + implicit val ec = materializer.executionContext + updateStreamMatValues(streamId, OutboundStreamMatValues(streamKillSwitch, completed.recover { case _ ⇒ Done })) + } + + @tailrec private def updateStreamMatValues(streamId: Int, values: OutboundStreamMatValues): Unit = { + val prev = streamMatValues.get() + if (!streamMatValues.compareAndSet(prev, prev + (streamId → values))) { + updateStreamMatValues(streamId, values) + } + } + + /** + * Exposed for orderly shutdown purposes, can not be trusted except for during shutdown as streams may restart. + * Will complete successfully even if one of the stream completion futures failed + */ + def streamsCompleted: Future[Done] = { + implicit val ec = materializer.executionContext + Future.sequence(streamMatValues.get().values.map { + case OutboundStreamMatValues(_, done) ⇒ done + }).map(_ ⇒ Done) + } + + override def toString: String = + s"Association($localAddress -> $remoteAddress with $associationState)" + +} + +/** + * INTERNAL API + */ +private[remote] class AssociationRegistry(createAssociation: Address ⇒ Association) { + private[this] val associationsByAddress = new AtomicReference[Map[Address, Association]](Map.empty) + private[this] val associationsByUid = new AtomicReference[ImmutableLongMap[Association]](ImmutableLongMap.empty) + + /** + * @throws ShuttingDown if called while the transport is shutting down + */ + @tailrec final def association(remoteAddress: Address): Association = { + val currentMap = associationsByAddress.get + currentMap.get(remoteAddress) match { + case Some(existing) ⇒ existing + case None ⇒ + val newAssociation = createAssociation(remoteAddress) + val newMap = currentMap.updated(remoteAddress, newAssociation) + if (associationsByAddress.compareAndSet(currentMap, newMap)) { + newAssociation.associate() // start it, only once + newAssociation + } else + association(remoteAddress) // lost CAS, retry + } + } + + def association(uid: Long): OptionVal[Association] = + associationsByUid.get.get(uid) + + /** + * @throws ShuttingDown if called while the transport is shutting down + */ + @tailrec final def setUID(peer: UniqueAddress): Association = { + val currentMap = associationsByUid.get + val a = association(peer.address) + + currentMap.get(peer.uid) match { + case OptionVal.Some(previous) ⇒ + if (previous eq a) + // associationsByUid Map already contains the right association + a + else + // make sure we don't overwrite same UID with different association + throw new IllegalArgumentException(s"UID collision old [$previous] new [$a]") + case _ ⇒ + // update associationsByUid Map with the uid -> assocation + val newMap = currentMap.updated(peer.uid, a) + if (associationsByUid.compareAndSet(currentMap, newMap)) + a + else + setUID(peer) // lost CAS, retry + } + } + + def allAssociations: Set[Association] = + associationsByAddress.get.values.toSet +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala new file mode 100644 index 0000000000..c2dcd094be --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -0,0 +1,539 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ + +package akka.remote.artery + +import java.nio.charset.Charset +import java.nio.{ ByteBuffer, ByteOrder } + +import akka.actor.{ ActorPath, ActorRef, Address, ChildActorPath } +import akka.io.DirectByteBufferPool +import akka.remote.artery.compress.CompressionProtocol._ +import akka.remote.artery.compress.{ CompressionTable, InboundCompressions } +import akka.serialization.Serialization +import org.agrona.concurrent.{ ManyToManyConcurrentArrayQueue, UnsafeBuffer } +import akka.util.{ ByteString, CompactByteString, OptionVal, Unsafe } + +import akka.remote.artery.compress.NoInboundCompressions +import akka.util.ByteString.ByteString1C + +import scala.annotation.tailrec + +/** + * INTERNAL API + */ +private[remote] class OutOfBuffersException extends RuntimeException("Out of usable ByteBuffers") + +/** + * INTERNAL API + */ +private[remote] class EnvelopeBufferPool(maximumPayload: Int, maximumBuffers: Int) { + private val availableBuffers = new ManyToManyConcurrentArrayQueue[EnvelopeBuffer](maximumBuffers) + + def acquire(): EnvelopeBuffer = { + val buf = availableBuffers.poll() + if (buf ne null) { + buf.byteBuffer.clear() + buf + } else { + val newBuf = new EnvelopeBuffer(ByteBuffer.allocateDirect(maximumPayload)) + newBuf.byteBuffer.order(ByteOrder.LITTLE_ENDIAN) + newBuf + } + } + + def release(buffer: EnvelopeBuffer) = if (!availableBuffers.offer(buffer)) buffer.tryCleanDirectByteBuffer() + +} + +/** INTERNAL API */ +private[remote] final class ByteFlag(val mask: Byte) extends AnyVal { + def isEnabled(byteFlags: Byte): Boolean = (byteFlags.toInt & mask) != 0 + override def toString = s"ByteFlag(${ByteFlag.binaryLeftPad(mask)})" +} +/** + * INTERNAL API + */ +private[remote] object ByteFlag { + def binaryLeftPad(byte: Byte): String = { + val string = Integer.toBinaryString(byte) + val pad = "0" * (8 - string.length) // leftPad + pad + string + } +} + +/** + * INTERNAL API + */ +private[remote] object EnvelopeBuffer { + + val TagTypeMask = 0xFF000000 + val TagValueMask = 0x0000FFFF + + // Flags (1 byte allocated for them) + val MetadataPresentFlag = new ByteFlag(0x1) + + val VersionOffset = 0 // Byte + val FlagsOffset = 1 // Byte + val ActorRefCompressionTableVersionOffset = 2 // Byte + val ClassManifestCompressionTableVersionOffset = 3 // Byte + + val UidOffset = 4 // Long + val SerializerOffset = 12 // Int + + val SenderActorRefTagOffset = 16 // Int + val RecipientActorRefTagOffset = 20 // Int + val ClassManifestTagOffset = 24 // Int + + // EITHER metadata followed by literals directly OR literals directly in this spot. + // Mode depends on the `MetadataPresentFlag`. + val MetadataContainerAndLiteralSectionOffset = 28 // Int + + val UsAscii = Charset.forName("US-ASCII") + + // accessing the internal char array of String when writing literal strings to ByteBuffer + val StringValueFieldOffset = Unsafe.instance.objectFieldOffset(classOf[String].getDeclaredField("value")) +} + +/** INTERNAL API */ +private[remote] object HeaderBuilder { + + // We really only use the Header builder on one "side" or the other, thus in order to avoid having to split its impl + // we inject no-op compression's of the "other side". + + def in(compression: InboundCompressions): HeaderBuilder = + new HeaderBuilderImpl(compression, CompressionTable.empty[ActorRef], CompressionTable.empty[String]) + + def out(): HeaderBuilder = + new HeaderBuilderImpl(NoInboundCompressions, CompressionTable.empty[ActorRef], CompressionTable.empty[String]) + + final val DeadLettersCode = -1 +} + +/** + * INTERNAL API + */ +private[remote] sealed trait HeaderBuilder { + def setVersion(v: Byte): Unit + def version: Byte + + def setFlags(v: Byte): Unit + def flags: Byte + def flag(byteFlag: ByteFlag): Boolean + def setFlag(byteFlag: ByteFlag, value: Boolean): Unit + + def inboundActorRefCompressionTableVersion: Byte + def inboundClassManifestCompressionTableVersion: Byte + + def useOutboundCompression(on: Boolean): Unit + + def outboundActorRefCompression: CompressionTable[ActorRef] + def setOutboundActorRefCompression(table: CompressionTable[ActorRef]): Unit + + def outboundClassManifestCompression: CompressionTable[String] + def setOutboundClassManifestCompression(table: CompressionTable[String]): Unit + + def setUid(u: Long): Unit + def uid: Long + + /** Metadata SPI, internally multiple metadata sections can be represented. */ + def metadataContainer: ByteString + + def setSenderActorRef(ref: ActorRef): Unit + /** + * Retrive the compressed ActorRef by the compressionId carried by this header. + * Returns `None` if ActorRef was not compressed, and then the literal [[senderActorRefPath]] should be used. + */ + def senderActorRef(originUid: Long): OptionVal[ActorRef] + /** + * Retrive the raw literal actor path, instead of using the compressed value. + * Returns `None` if ActorRef was compressed (!). To obtain the path in such case call [[senderActorRef]] and extract the path from it directly. + */ + def senderActorRefPath: OptionVal[String] + + def setNoSender(): Unit + def isNoSender: Boolean + + def setNoRecipient(): Unit + def isNoRecipient: Boolean + + def setRecipientActorRef(ref: ActorRef): Unit + /** + * Retrive the compressed ActorRef by the compressionId carried by this header. + * Returns `None` if ActorRef was not compressed, and then the literal [[recipientActorRefPath]] should be used. + */ + def recipientActorRef(originUid: Long): OptionVal[ActorRef] + /** + * Retrive the raw literal actor path, instead of using the compressed value. + * Returns `None` if ActorRef was compressed (!). To obtain the path in such case call [[recipientActorRefPath]] and extract the path from it directly. + */ + def recipientActorRefPath: OptionVal[String] + + def setMetadataContainer(container: ByteString): Unit + def clearMetadataContainer(): Unit + + def setSerializer(serializer: Int): Unit + def serializer: Int + + def setManifest(manifest: String): Unit + def manifest(originUid: Long): OptionVal[String] + + /** + * Reset all fields that are related to an outbound message, + * i.e. Encoder calls this as the first thing in onPush. + */ + def resetMessageFields(): Unit + +} + +/** + * INTERNAL API + */ +private[remote] final class SerializationFormatCache + extends LruBoundedCache[ActorRef, String](capacity = 1024, evictAgeThreshold = 600) { + + override protected def compute(ref: ActorRef): String = ref.path.toSerializationFormat + + // Not calling ref.hashCode since it does a path.hashCode if ActorCell.undefinedUid is encountered. + // Refs with ActorCell.undefinedUid will now collide all the time, but this is not a usual scenario anyway. + override protected def hash(ref: ActorRef): Int = ref.path.uid + + override protected def isCacheable(v: String): Boolean = true +} + +/** + * INTERNAL API + */ +private[remote] final class HeaderBuilderImpl( + inboundCompression: InboundCompressions, + var _outboundActorRefCompression: CompressionTable[ActorRef], + var _outboundClassManifestCompression: CompressionTable[String]) extends HeaderBuilder { + import HeaderBuilder.DeadLettersCode + + private[this] val toSerializationFormat: SerializationFormatCache = new SerializationFormatCache + + // Fields only available for EnvelopeBuffer + var _version: Byte = 0 + var _flags: Byte = 0 + var _uid: Long = 0 + var _inboundActorRefCompressionTableVersion: Byte = 0 + var _inboundClassManifestCompressionTableVersion: Byte = 0 + var _useOutboundCompression: Boolean = true + + var _senderActorRef: String = null + var _senderActorRefIdx: Int = -1 + var _recipientActorRef: String = null + var _recipientActorRefIdx: Int = -1 + + var _serializer: Int = 0 + var _manifest: String = null + var _manifestIdx: Int = -1 + + var _metadataContainer: ByteString = null + + override def resetMessageFields(): Unit = { + _flags = 0 + _senderActorRef = null + _senderActorRefIdx = -1 + _recipientActorRef = null + _recipientActorRefIdx = -1 + + _serializer = 0 + _manifest = null + _manifestIdx = -1 + } + + override def setVersion(v: Byte) = _version = v + override def version = _version + + override def setFlags(v: Byte) = _flags = v + override def flags = _flags + override def flag(byteFlag: ByteFlag): Boolean = (_flags.toInt & byteFlag.mask) != 0 + override def setFlag(byteFlag: ByteFlag, value: Boolean): Unit = + if (value) _flags = (flags | byteFlag.mask).toByte + else _flags = (flags & ~byteFlag.mask).toByte + + override def setUid(uid: Long) = _uid = uid + override def uid: Long = _uid + + override def inboundActorRefCompressionTableVersion: Byte = _inboundActorRefCompressionTableVersion + override def inboundClassManifestCompressionTableVersion: Byte = _inboundClassManifestCompressionTableVersion + + def useOutboundCompression(on: Boolean): Unit = + _useOutboundCompression = on + + def setOutboundActorRefCompression(table: CompressionTable[ActorRef]): Unit = { + _outboundActorRefCompression = table + } + override def outboundActorRefCompression: CompressionTable[ActorRef] = _outboundActorRefCompression + + def setOutboundClassManifestCompression(table: CompressionTable[String]): Unit = { + _outboundClassManifestCompression = table + } + def outboundClassManifestCompression: CompressionTable[String] = _outboundClassManifestCompression + + override def setSenderActorRef(ref: ActorRef): Unit = { + // serializedActorPath includes local address from `currentTransportInformation` + if (_useOutboundCompression) { + _senderActorRefIdx = outboundActorRefCompression.compress(ref) + if (_senderActorRefIdx == -1) _senderActorRef = Serialization.serializedActorPath(ref) + } else + _senderActorRef = Serialization.serializedActorPath(ref) + } + override def setNoSender(): Unit = { + _senderActorRef = null + _senderActorRefIdx = DeadLettersCode + } + override def isNoSender: Boolean = + (_senderActorRef eq null) && _senderActorRefIdx == DeadLettersCode + override def senderActorRef(originUid: Long): OptionVal[ActorRef] = { + // we treat deadLetters as always present, but not included in table + if ((_senderActorRef eq null) && !isNoSender) + inboundCompression.decompressActorRef(originUid, inboundActorRefCompressionTableVersion, _senderActorRefIdx) + else OptionVal.None + } + + def senderActorRefPath: OptionVal[String] = + OptionVal(_senderActorRef) + + def setNoRecipient(): Unit = { + _recipientActorRef = null + _recipientActorRefIdx = DeadLettersCode + } + def isNoRecipient: Boolean = + (_recipientActorRef eq null) && _recipientActorRefIdx == DeadLettersCode + + def setRecipientActorRef(ref: ActorRef): Unit = { + if (_useOutboundCompression) { + _recipientActorRefIdx = outboundActorRefCompression.compress(ref) + if (_recipientActorRefIdx == -1) _recipientActorRef = toSerializationFormat.getOrCompute(ref) + } else + _recipientActorRef = toSerializationFormat.getOrCompute(ref) + } + def recipientActorRef(originUid: Long): OptionVal[ActorRef] = { + // we treat deadLetters as always present, but not included in table + if ((_recipientActorRef eq null) && !isNoRecipient) + inboundCompression.decompressActorRef(originUid, inboundActorRefCompressionTableVersion, _recipientActorRefIdx) + else OptionVal.None + } + def recipientActorRefPath: OptionVal[String] = + OptionVal(_recipientActorRef) + + override def setSerializer(serializer: Int): Unit = { + _serializer = serializer + } + override def serializer: Int = + _serializer + + override def setManifest(manifest: String): Unit = { + if (_useOutboundCompression) { + _manifestIdx = outboundClassManifestCompression.compress(manifest) + if (_manifestIdx == -1) _manifest = manifest + } else + _manifest = manifest + } + override def manifest(originUid: Long): OptionVal[String] = { + if (_manifest ne null) OptionVal.Some(_manifest) + else { + inboundCompression.decompressClassManifest( + originUid, + inboundClassManifestCompressionTableVersion, _manifestIdx) + } + } + + /** Make sure to prefix the data with an Int-length */ + def setMetadataContainer(container: ByteString): Unit = { + setFlag(EnvelopeBuffer.MetadataPresentFlag, value = container != null) + _metadataContainer = container + } + /** Rendered metadata already contains int-length prefix, no need to add it manually */ + def metadataContainer: ByteString = + _metadataContainer + def clearMetadataContainer(): Unit = { + setFlag(EnvelopeBuffer.MetadataPresentFlag, value = false) + _metadataContainer = null + } + + override def toString = + "HeaderBuilderImpl(" + + "version:" + version + ", " + + "flags:" + ByteFlag.binaryLeftPad(flags) + ", " + + "UID:" + uid + ", " + + "_senderActorRef:" + _senderActorRef + ", " + + "_senderActorRefIdx:" + _senderActorRefIdx + ", " + + "_recipientActorRef:" + _recipientActorRef + ", " + + "_recipientActorRefIdx:" + _recipientActorRefIdx + ", " + + "_serializer:" + _serializer + ", " + + "_manifest:" + _manifest + ", " + + "_manifestIdx:" + _manifestIdx + ", " + + "_metadataContainer:" + _metadataContainer + ")" + +} + +/** + * INTERNAL API + */ +private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { + import EnvelopeBuffer._ + val aeronBuffer = new UnsafeBuffer(byteBuffer) + + private var literalChars = Array.ofDim[Char](64) + private var literalBytes = Array.ofDim[Byte](64) + + def writeHeader(h: HeaderBuilder): Unit = { + val header = h.asInstanceOf[HeaderBuilderImpl] + byteBuffer.clear() + + // Write fixed length parts + byteBuffer.put(VersionOffset, header.version) + byteBuffer.put(FlagsOffset, header.flags) + byteBuffer.putLong(UidOffset, header.uid) + byteBuffer.putInt(SerializerOffset, header.serializer) + + // compression table version numbers + byteBuffer.put(ActorRefCompressionTableVersionOffset, header.outboundActorRefCompression.version) + byteBuffer.put(ClassManifestCompressionTableVersionOffset, header.outboundClassManifestCompression.version) + + byteBuffer.position(MetadataContainerAndLiteralSectionOffset) + if (header.flag(MetadataPresentFlag)) { + // tag if we have metadata or not, as the layout next follows different patterns depending on that + header.metadataContainer.copyToBuffer(byteBuffer) + // after metadata is written, buffer is at correct position to continue writing literals (they "moved forward") + } + + // Serialize sender + if (header._senderActorRefIdx != -1) + byteBuffer.putInt(SenderActorRefTagOffset, header._senderActorRefIdx | TagTypeMask) + else + writeLiteral(SenderActorRefTagOffset, header._senderActorRef) + + // Serialize recipient + if (header._recipientActorRefIdx != -1) + byteBuffer.putInt(RecipientActorRefTagOffset, header._recipientActorRefIdx | TagTypeMask) + else + writeLiteral(RecipientActorRefTagOffset, header._recipientActorRef) + + // Serialize class manifest + if (header._manifestIdx != -1) + byteBuffer.putInt(ClassManifestTagOffset, header._manifestIdx | TagTypeMask) + else + writeLiteral(ClassManifestTagOffset, header._manifest) + + } + + def parseHeader(h: HeaderBuilder): Unit = { + val header = h.asInstanceOf[HeaderBuilderImpl] + + // Read fixed length parts + header.setVersion(byteBuffer.get(VersionOffset)) + header.setFlags(byteBuffer.get(FlagsOffset)) + header.setUid(byteBuffer.getLong(UidOffset)) + header.setSerializer(byteBuffer.getInt(SerializerOffset)) + + // compression table versions (stored in the Tag) + header._inboundActorRefCompressionTableVersion = byteBuffer.get(ActorRefCompressionTableVersionOffset) + header._inboundClassManifestCompressionTableVersion = byteBuffer.get(ClassManifestCompressionTableVersionOffset) + + if (header.flag(MetadataPresentFlag)) { + byteBuffer.position(MetadataContainerAndLiteralSectionOffset) + val totalMetadataLength = byteBuffer.getInt() + + ensureLiteralCharsLength(totalMetadataLength) + val bytes = literalBytes + + byteBuffer.get(bytes, 0, totalMetadataLength) + header._metadataContainer = ByteString(bytes).take(totalMetadataLength) + // the literals section starts here, right after the metadata has ended + // thus, no need to move position the buffer again + } else { + // No metadata present, we position the buffer on the place where literals start + byteBuffer.position(MetadataContainerAndLiteralSectionOffset) + } + + // Deserialize sender + val senderTag = byteBuffer.getInt(SenderActorRefTagOffset) + if ((senderTag & TagTypeMask) != 0) { + val idx = senderTag & TagValueMask + header._senderActorRef = null + header._senderActorRefIdx = idx + } else { + header._senderActorRef = emptyAsNull(readLiteral()) + } + + // Deserialize recipient + val recipientTag = byteBuffer.getInt(RecipientActorRefTagOffset) + if ((recipientTag & TagTypeMask) != 0) { + val idx = recipientTag & TagValueMask + header._recipientActorRef = null + header._recipientActorRefIdx = idx + } else { + header._recipientActorRef = emptyAsNull(readLiteral()) + } + + // Deserialize class manifest + val manifestTag = byteBuffer.getInt(ClassManifestTagOffset) + if ((manifestTag & TagTypeMask) != 0) { + val idx = manifestTag & TagValueMask + header._manifest = null + header._manifestIdx = idx + } else { + header._manifest = readLiteral() + } + } + + private def emptyAsNull(s: String): String = + if (s == "") null + else s + + private def readLiteral(): String = { + val length = byteBuffer.getShort + if (length == 0) "" + else { + ensureLiteralCharsLength(length) + val chars = literalChars + val bytes = literalBytes + byteBuffer.get(bytes, 0, length) + var i = 0 + while (i < length) { + // UsAscii + chars(i) = bytes(i).asInstanceOf[Char] + i += 1 + } + String.valueOf(chars, 0, length) + } + } + + private def writeLiteral(tagOffset: Int, literal: String): Unit = { + val length = if (literal eq null) 0 else literal.length + if (length > 65535) + throw new IllegalArgumentException("Literals longer than 65535 cannot be encoded in the envelope") + + byteBuffer.putInt(tagOffset, byteBuffer.position()) + + if (length == 0) { + byteBuffer.putShort(0) + } else { + byteBuffer.putShort(length.toShort) + ensureLiteralCharsLength(length) + val bytes = literalBytes + val chars = Unsafe.instance.getObject(literal, StringValueFieldOffset).asInstanceOf[Array[Char]] + var i = 0 + while (i < length) { + // UsAscii + bytes(i) = chars(i).asInstanceOf[Byte] + i += 1 + } + byteBuffer.put(bytes, 0, length) + } + } + + private def ensureLiteralCharsLength(length: Int): Unit = { + if (length > literalChars.length) { + literalChars = Array.ofDim[Char](length) + literalBytes = Array.ofDim[Byte](length) + } + } + + def tryCleanDirectByteBuffer(): Unit = DirectByteBufferPool.tryCleanDirectByteBuffer(byteBuffer) +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala new file mode 100644 index 0000000000..8ca2a0eae6 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -0,0 +1,566 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.duration._ +import scala.util.control.NonFatal +import akka.actor._ +import akka.remote.{ MessageSerializer, OversizedPayloadException, RemoteActorRefProvider, UniqueAddress } +import akka.remote.artery.SystemMessageDelivery.SystemMessageEnvelope +import akka.serialization.{ Serialization, SerializationExtension } +import akka.stream._ +import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler } +import akka.util.{ ByteString, OptionVal } +import akka.actor.EmptyLocalActorRef +import akka.remote.artery.compress.InboundCompressions +import akka.stream.stage.TimerGraphStageLogic +import java.util.concurrent.TimeUnit + +import scala.concurrent.Future +import akka.remote.artery.compress.CompressionTable +import akka.Done +import akka.stream.stage.GraphStageWithMaterializedValue + +import scala.concurrent.Promise +import akka.event.Logging + +/** + * INTERNAL API + */ +private[remote] object Encoder { + private[remote] trait ChangeOutboundCompression { + def changeActorRefCompression(table: CompressionTable[ActorRef]): Future[Done] + def changeClassManifestCompression(table: CompressionTable[String]): Future[Done] + def clearCompression(): Future[Done] + } + + private[remote] class ChangeOutboundCompressionFailed extends RuntimeException( + "Change of outbound compression table failed (will be retried), because materialization did not complete yet") + +} + +/** + * INTERNAL API + */ +private[remote] class Encoder( + uniqueLocalAddress: UniqueAddress, + system: ExtendedActorSystem, + outboundEnvelopePool: ObjectPool[ReusableOutboundEnvelope], + bufferPool: EnvelopeBufferPool, + debugLogSend: Boolean) + extends GraphStageWithMaterializedValue[FlowShape[OutboundEnvelope, EnvelopeBuffer], Encoder.ChangeOutboundCompression] { + import Encoder._ + + val in: Inlet[OutboundEnvelope] = Inlet("Artery.Encoder.in") + val out: Outlet[EnvelopeBuffer] = Outlet("Artery.Encoder.out") + val shape: FlowShape[OutboundEnvelope, EnvelopeBuffer] = FlowShape(in, out) + + override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, ChangeOutboundCompression) = { + val logic = new GraphStageLogic(shape) with InHandler with OutHandler with StageLogging with ChangeOutboundCompression { + + private val headerBuilder = HeaderBuilder.out() + headerBuilder setVersion ArteryTransport.Version + headerBuilder setUid uniqueLocalAddress.uid + private val localAddress = uniqueLocalAddress.address + private val serialization = SerializationExtension(system) + private val serializationInfo = Serialization.Information(localAddress, system) + + private val instruments: Vector[RemoteInstrument] = RemoteInstruments.create(system) + // by being backed by an Array, this allows us to not allocate any wrapper type for the metadata (since we need its ID) + private val serializedMetadatas: MetadataMap[ByteString] = MetadataMap() // TODO: possibly can be optimised a more for the specific access pattern (during write) + + private val changeActorRefCompressionCb = getAsyncCallback[(CompressionTable[ActorRef], Promise[Done])] { + case (table, done) ⇒ + headerBuilder.setOutboundActorRefCompression(table) + done.success(Done) + } + + private val changeClassManifsetCompressionCb = getAsyncCallback[(CompressionTable[String], Promise[Done])] { + case (table, done) ⇒ + headerBuilder.setOutboundClassManifestCompression(table) + done.success(Done) + } + + private val clearCompressionCb = getAsyncCallback[Promise[Done]] { done ⇒ + headerBuilder.setOutboundActorRefCompression(CompressionTable.empty[ActorRef]) + headerBuilder.setOutboundClassManifestCompression(CompressionTable.empty[String]) + done.success(Done) + } + + override protected def logSource = classOf[Encoder] + + private var debugLogSendEnabled = false + + override def preStart(): Unit = { + debugLogSendEnabled = debugLogSend && log.isDebugEnabled + } + + override def onPush(): Unit = { + val outboundEnvelope = grab(in) + val envelope = bufferPool.acquire() + + headerBuilder.resetMessageFields() + // don't use outbound compression for ArteryMessage, e.g. handshake messages must get through + // without depending on compression tables being in sync when systems are restarted + headerBuilder.useOutboundCompression(!outboundEnvelope.message.isInstanceOf[ArteryMessage]) + + // internally compression is applied by the builder: + outboundEnvelope.recipient match { + case OptionVal.Some(r) ⇒ headerBuilder setRecipientActorRef r + case OptionVal.None ⇒ headerBuilder.setNoRecipient() + } + + try { + // avoiding currentTransportInformation.withValue due to thunk allocation + val oldValue = Serialization.currentTransportInformation.value + try { + Serialization.currentTransportInformation.value = serializationInfo + + outboundEnvelope.sender match { + case OptionVal.None ⇒ headerBuilder.setNoSender() + case OptionVal.Some(s) ⇒ headerBuilder setSenderActorRef s + } + + applyAndRenderRemoteMessageSentMetadata(instruments, outboundEnvelope, headerBuilder) + MessageSerializer.serializeForArtery(serialization, outboundEnvelope.message, headerBuilder, envelope) + } finally Serialization.currentTransportInformation.value = oldValue + + envelope.byteBuffer.flip() + + if (debugLogSendEnabled) + log.debug( + "sending remote message [{}] to [{}] from [{}]", + Logging.messageClassName(outboundEnvelope.message), + outboundEnvelope.recipient.getOrElse(""), outboundEnvelope.sender.getOrElse("")) + + push(out, envelope) + + } catch { + case NonFatal(e) ⇒ + bufferPool.release(envelope) + outboundEnvelope.message match { + case _: SystemMessageEnvelope ⇒ + log.error(e, "Failed to serialize system message [{}].", + Logging.messageClassName(outboundEnvelope.message)) + throw e + case _ if e.isInstanceOf[java.nio.BufferOverflowException] ⇒ + val reason = new OversizedPayloadException("Discarding oversized payload sent to " + + s"${outboundEnvelope.recipient}: max allowed size ${envelope.byteBuffer.limit()} " + + s"bytes. Message type [${Logging.messageClassName(outboundEnvelope.message)}].") + log.error(reason, "Failed to serialize oversized message [{}].", + Logging.messageClassName(outboundEnvelope.message)) + pull(in) + case _ ⇒ + log.error(e, "Failed to serialize message [{}].", Logging.messageClassName(outboundEnvelope.message)) + pull(in) + } + } finally { + outboundEnvelope match { + case r: ReusableOutboundEnvelope ⇒ outboundEnvelopePool.release(r) + case _ ⇒ // no need to release it + } + } + } + + override def onPull(): Unit = pull(in) + + /** + * Renders metadata into `headerBuilder`. + * + * Replace all AnyRef's that were passed along with the [[OutboundEnvelope]] into their [[ByteString]] representations, + * by calling `remoteMessageSent` of each enabled instrumentation. If `context` was attached in the envelope it is passed + * into the instrument, otherwise it receives an OptionVal.None as context, and may still decide to attach rendered + * metadata by returning it. + */ + private def applyAndRenderRemoteMessageSentMetadata(instruments: Vector[RemoteInstrument], envelope: OutboundEnvelope, headerBuilder: HeaderBuilder): Unit = { + if (instruments.nonEmpty) { + val n = instruments.length + + var i = 0 + while (i < n) { + val instrument = instruments(i) + val instrumentId = instrument.identifier + + val metadata = instrument.remoteMessageSent(envelope.recipient.orNull, envelope.message, envelope.sender.orNull) + if (metadata ne null) serializedMetadatas.set(instrumentId, metadata) + + i += 1 + } + } + + if (serializedMetadatas.nonEmpty) { + MetadataEnvelopeSerializer.serialize(serializedMetadatas, headerBuilder) + serializedMetadatas.clear() + } + } + + /** + * External call from ChangeOutboundCompression materialized value + */ + override def changeActorRefCompression(table: CompressionTable[ActorRef]): Future[Done] = { + val done = Promise[Done]() + try changeActorRefCompressionCb.invoke((table, done)) catch { + // This is a harmless failure, it will be retried on next advertisement or handshake attempt. + // It will only occur when callback is invoked before preStart. That is highly unlikely to + // happen since advertisement is not done immediately and handshake involves network roundtrip. + case NonFatal(_) ⇒ done.tryFailure(new ChangeOutboundCompressionFailed) + } + done.future + } + + /** + * External call from ChangeOutboundCompression materialized value + */ + override def changeClassManifestCompression(table: CompressionTable[String]): Future[Done] = { + val done = Promise[Done]() + try changeClassManifsetCompressionCb.invoke((table, done)) catch { + // in case materialization not completed yet + case NonFatal(_) ⇒ done.tryFailure(new ChangeOutboundCompressionFailed) + } + done.future + } + + /** + * External call from ChangeOutboundCompression materialized value + */ + override def clearCompression(): Future[Done] = { + val done = Promise[Done]() + try clearCompressionCb.invoke(done) catch { + // in case materialization not completed yet + case NonFatal(_) ⇒ done.tryFailure(new ChangeOutboundCompressionFailed) + } + done.future + } + + setHandlers(in, out, this) + } + + (logic, logic) + } +} + +/** + * INTERNAL API + */ +private[remote] object Decoder { + private final case class RetryResolveRemoteDeployedRecipient( + attemptsLeft: Int, + recipientPath: String, + inboundEnvelope: InboundEnvelope) + + private object Tick +} + +/** + * INTERNAL API + */ +private[remote] final class ActorRefResolveCacheWithAddress(provider: RemoteActorRefProvider, localAddress: UniqueAddress) + extends LruBoundedCache[String, InternalActorRef](capacity = 1024, evictAgeThreshold = 600) { + + override protected def compute(k: String): InternalActorRef = + provider.resolveActorRefWithLocalAddress(k, localAddress.address) + + override protected def hash(k: String): Int = FastHash.ofString(k) + + override protected def isCacheable(v: InternalActorRef): Boolean = !v.isInstanceOf[EmptyLocalActorRef] +} + +/** + * INTERNAL API + */ +private[remote] class Decoder( + inboundContext: InboundContext, + system: ExtendedActorSystem, + uniqueLocalAddress: UniqueAddress, + compression: InboundCompressions, + bufferPool: EnvelopeBufferPool, + inEnvelopePool: ObjectPool[ReusableInboundEnvelope]) extends GraphStage[FlowShape[EnvelopeBuffer, InboundEnvelope]] { + import Decoder.Tick + val in: Inlet[EnvelopeBuffer] = Inlet("Artery.Decoder.in") + val out: Outlet[InboundEnvelope] = Outlet("Artery.Decoder.out") + val shape: FlowShape[EnvelopeBuffer, InboundEnvelope] = FlowShape(in, out) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new TimerGraphStageLogic(shape) with InHandler with OutHandler with StageLogging { + import Decoder.RetryResolveRemoteDeployedRecipient + private val localAddress = inboundContext.localAddress.address + private val headerBuilder = HeaderBuilder.in(compression) + private val actorRefResolver: ActorRefResolveCacheWithAddress = + new ActorRefResolveCacheWithAddress(system.provider.asInstanceOf[RemoteActorRefProvider], uniqueLocalAddress) + private val bannedRemoteDeployedActorRefs = new java.util.HashSet[String] + + private val retryResolveRemoteDeployedRecipientInterval = 50.millis + private val retryResolveRemoteDeployedRecipientAttempts = 20 + + // adaptive sampling when rate > 1000 msg/s + private var messageCount = 0L + private var heavyHitterMask = 0 // 0 => no sampling, otherwise power of two - 1 + private val adaptiveSamplingRateThreshold = 1000 + private var tickTimestamp = System.nanoTime() + private var tickMessageCount = 0L + + override protected def logSource = classOf[Decoder] + + override def preStart(): Unit = { + schedulePeriodically(Tick, 1.seconds) + } + + override def onPush(): Unit = { + messageCount += 1 + val envelope = grab(in) + headerBuilder.resetMessageFields() + envelope.parseHeader(headerBuilder) + + val originUid = headerBuilder.uid + val association = inboundContext.association(originUid) + + val recipient: OptionVal[InternalActorRef] = try headerBuilder.recipientActorRef(originUid) match { + case OptionVal.Some(ref) ⇒ + OptionVal(ref.asInstanceOf[InternalActorRef]) + case OptionVal.None if headerBuilder.recipientActorRefPath.isDefined ⇒ + resolveRecipient(headerBuilder.recipientActorRefPath.get) + case _ ⇒ + OptionVal.None + } catch { + case NonFatal(e) ⇒ + // probably version mismatch due to restarted system + log.warning("Couldn't decompress sender from originUid [{}]. {}", originUid, e.getMessage) + OptionVal.None + } + + val sender: OptionVal[InternalActorRef] = try headerBuilder.senderActorRef(originUid) match { + case OptionVal.Some(ref) ⇒ + OptionVal(ref.asInstanceOf[InternalActorRef]) + case OptionVal.None if headerBuilder.senderActorRefPath.isDefined ⇒ + OptionVal(actorRefResolver.getOrCompute(headerBuilder.senderActorRefPath.get)) + case _ ⇒ + OptionVal.None + } catch { + case NonFatal(e) ⇒ + // probably version mismatch due to restarted system + log.warning("Couldn't decompress sender from originUid [{}]. {}", originUid, e.getMessage) + OptionVal.None + } + + val classManifestOpt = try headerBuilder.manifest(originUid) catch { + case NonFatal(e) ⇒ + // probably version mismatch due to restarted system + log.warning("Couldn't decompress manifest from originUid [{}]. {}", originUid, e.getMessage) + OptionVal.None + } + + if ((recipient.isEmpty && headerBuilder.recipientActorRefPath.isEmpty && !headerBuilder.isNoRecipient) || + (sender.isEmpty && headerBuilder.senderActorRefPath.isEmpty && !headerBuilder.isNoSender)) { + log.debug("Dropping message for unknown recipient/sender. It was probably sent from system [{}] with compression " + + "table [{}] built for previous incarnation of the destination system, or it was compressed with a table " + + "that has already been discarded in the destination system.", originUid, + headerBuilder.inboundActorRefCompressionTableVersion) + pull(in) + } else if (classManifestOpt.isEmpty) { + log.debug("Dropping message with unknown manifest. It was probably sent from system [{}] with compression " + + "table [{}] built for previous incarnation of the destination system, or it was compressed with a table " + + "that has already been discarded in the destination system.", originUid, + headerBuilder.inboundActorRefCompressionTableVersion) + pull(in) + } else { + val classManifest = classManifestOpt.get + + if ((messageCount & heavyHitterMask) == 0) { + // --- hit refs and manifests for heavy-hitter counting + association match { + case OptionVal.Some(assoc) ⇒ + val remoteAddress = assoc.remoteAddress + sender match { + case OptionVal.Some(snd) ⇒ + compression.hitActorRef(originUid, remoteAddress, snd, 1) + case OptionVal.None ⇒ + } + + recipient match { + case OptionVal.Some(rcp) ⇒ + compression.hitActorRef(originUid, remoteAddress, rcp, 1) + case OptionVal.None ⇒ + } + + compression.hitClassManifest(originUid, remoteAddress, classManifest, 1) + + case _ ⇒ + // we don't want to record hits for compression while handshake is still in progress. + log.debug("Decoded message but unable to record hits for compression as no remoteAddress known. No association yet?") + } + // --- end of hit refs and manifests for heavy-hitter counting + } + + val decoded = inEnvelopePool.acquire().init( + recipient, + sender, + originUid, + headerBuilder.serializer, + classManifest, + headerBuilder.flags, + envelope, + association) + + if (recipient.isEmpty && !headerBuilder.isNoRecipient) { + + // The remote deployed actor might not be created yet when resolving the + // recipient for the first message that is sent to it, best effort retry. + // However, if the retried resolve isn't successful the ref is banned and + // we will not do the delayed retry resolve again. The reason for that is + // if many messages are sent to such dead refs the resolve process will slow + // down other messages. + val recipientActorRefPath = headerBuilder.recipientActorRefPath.get + if (bannedRemoteDeployedActorRefs.contains(recipientActorRefPath)) { + + headerBuilder.recipientActorRefPath match { + case OptionVal.Some(path) ⇒ + val ref = actorRefResolver.getOrCompute(path) + if (ref.isInstanceOf[EmptyLocalActorRef]) log.warning( + "Message for banned (terminated, unresolved) remote deployed recipient [{}].", + recipientActorRefPath) + push(out, decoded.withRecipient(ref)) + case OptionVal.None ⇒ + log.warning( + "Dropping message for banned (terminated, unresolved) remote deployed recipient [{}].", + recipientActorRefPath) + pull(in) + } + + } else + scheduleOnce(RetryResolveRemoteDeployedRecipient( + retryResolveRemoteDeployedRecipientAttempts, + recipientActorRefPath, decoded), retryResolveRemoteDeployedRecipientInterval) + } else { + push(out, decoded) + } + } + } + + private def resolveRecipient(path: String): OptionVal[InternalActorRef] = { + actorRefResolver.getOrCompute(path) match { + case empty: EmptyLocalActorRef ⇒ + val pathElements = empty.path.elements + if (pathElements.nonEmpty && pathElements.head == "remote") OptionVal.None + else OptionVal(empty) + case ref ⇒ OptionVal(ref) + } + } + + override def onPull(): Unit = pull(in) + + override protected def onTimer(timerKey: Any): Unit = { + timerKey match { + case Tick ⇒ + val now = System.nanoTime() + val d = math.max(1, now - tickTimestamp) + val rate = (messageCount - tickMessageCount) * TimeUnit.SECONDS.toNanos(1) / d + val oldHeavyHitterMask = heavyHitterMask + heavyHitterMask = + if (rate < adaptiveSamplingRateThreshold) 0 // no sampling + else if (rate < adaptiveSamplingRateThreshold * 10) (1 << 6) - 1 // sample every 64nth message + else if (rate < adaptiveSamplingRateThreshold * 100) (1 << 7) - 1 // sample every 128nth message + else (1 << 8) - 1 // sample every 256nth message + if (oldHeavyHitterMask > 0 && heavyHitterMask == 0) + log.debug("Turning off adaptive sampling of compression hit counting") + else if (oldHeavyHitterMask != heavyHitterMask) + log.debug("Turning on adaptive sampling ({}nth message) of compression hit counting", heavyHitterMask + 1) + tickMessageCount = messageCount + tickTimestamp = now + + case RetryResolveRemoteDeployedRecipient(attemptsLeft, recipientPath, inboundEnvelope) ⇒ + resolveRecipient(recipientPath) match { + case OptionVal.None ⇒ + if (attemptsLeft > 0) + scheduleOnce(RetryResolveRemoteDeployedRecipient( + attemptsLeft - 1, + recipientPath, inboundEnvelope), retryResolveRemoteDeployedRecipientInterval) + else { + // No more attempts left. If the retried resolve isn't successful the ref is banned and + // we will not do the delayed retry resolve again. The reason for that is + // if many messages are sent to such dead refs the resolve process will slow + // down other messages. + if (bannedRemoteDeployedActorRefs.size >= 100) { + // keep it bounded + bannedRemoteDeployedActorRefs.clear() + } + bannedRemoteDeployedActorRefs.add(recipientPath) + + val recipient = actorRefResolver.getOrCompute(recipientPath) + push(out, inboundEnvelope.withRecipient(recipient)) + } + case OptionVal.Some(recipient) ⇒ + push(out, inboundEnvelope.withRecipient(recipient)) + } + } + } + + setHandlers(in, out, this) + } +} + +/** + * INTERNAL API + */ +private[remote] class Deserializer( + inboundContext: InboundContext, + system: ExtendedActorSystem, + bufferPool: EnvelopeBufferPool) extends GraphStage[FlowShape[InboundEnvelope, InboundEnvelope]] { + + val in: Inlet[InboundEnvelope] = Inlet("Artery.Deserializer.in") + val out: Outlet[InboundEnvelope] = Outlet("Artery.Deserializer.out") + val shape: FlowShape[InboundEnvelope, InboundEnvelope] = FlowShape(in, out) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new GraphStageLogic(shape) with InHandler with OutHandler with StageLogging { + private val instruments: Vector[RemoteInstrument] = RemoteInstruments.create(system) + private val serialization = SerializationExtension(system) + + override protected def logSource = classOf[Deserializer] + + override def onPush(): Unit = { + val envelope = grab(in) + + try { + val deserializedMessage = MessageSerializer.deserializeForArtery( + system, envelope.originUid, serialization, envelope.serializer, envelope.classManifest, envelope.envelopeBuffer) + + val envelopeWithMessage = envelope.withMessage(deserializedMessage) + + applyIncomingInstruments(envelopeWithMessage) + + push(out, envelopeWithMessage) + } catch { + case NonFatal(e) ⇒ + log.warning( + "Failed to deserialize message with serializer id [{}] and manifest [{}]. {}", + envelope.serializer, envelope.classManifest, e.getMessage) + pull(in) + } finally { + val buf = envelope.envelopeBuffer + envelope.releaseEnvelopeBuffer() + bufferPool.release(buf) + } + } + + override def onPull(): Unit = pull(in) + + private def applyIncomingInstruments(envelope: InboundEnvelope): Unit = { + if (envelope.flag(EnvelopeBuffer.MetadataPresentFlag)) { + val length = instruments.length + if (length == 0) { + // TODO do we need to parse, or can we do a fast forward if debug logging is not enabled? + val metaMetadataEnvelope = MetadataMapParsing.parse(envelope) + if (log.isDebugEnabled) + log.debug("Incoming message envelope contains metadata for instruments: {}, " + + "however no RemoteInstrument was registered in local system!", metaMetadataEnvelope.metadataMap.keysWithValues.mkString("[", ",", "]")) + } else { + // we avoid emitting a MetadataMap and instead directly apply the instruments onto the received metadata + MetadataMapParsing.applyAllRemoteMessageReceived(instruments, envelope) + } + } + } + + setHandlers(in, out, this) + } +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/Control.scala b/akka-remote/src/main/scala/akka/remote/artery/Control.scala new file mode 100644 index 0000000000..3c8a2b38cc --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/Control.scala @@ -0,0 +1,230 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.ArrayDeque +import scala.concurrent.Future +import scala.concurrent.Promise +import akka.Done +import akka.stream.Attributes +import akka.stream.FlowShape +import akka.stream.Inlet +import akka.stream.Outlet +import akka.stream.stage.CallbackWrapper +import akka.stream.stage.GraphStageLogic +import akka.stream.stage.GraphStageWithMaterializedValue +import akka.stream.stage.InHandler +import akka.stream.stage.OutHandler +import akka.remote.UniqueAddress +import akka.util.OptionVal +import akka.event.Logging + +/** INTERNAL API: marker trait for protobuf-serializable artery messages */ +private[remote] trait ArteryMessage extends Serializable + +/** + * INTERNAL API: Marker trait for reply messages + */ +private[remote] trait Reply extends ControlMessage + +/** + * INTERNAL API + * Marker trait for control messages that can be sent via the system message sub-channel + * but don't need full reliable delivery. E.g. `HandshakeReq` and `Reply`. + */ +private[remote] trait ControlMessage extends ArteryMessage + +/** + * INTERNAL API + */ +private[remote] final case class Quarantined(from: UniqueAddress, to: UniqueAddress) extends ControlMessage + +/** + * INTERNAL API + */ +private[remote] case class ActorSystemTerminating(from: UniqueAddress) extends ControlMessage + +/** + * INTERNAL API + */ +private[remote] case class ActorSystemTerminatingAck(from: UniqueAddress) extends ArteryMessage + +/** + * INTERNAL API + */ +private[remote] object InboundControlJunction { + + /** + * Observer subject for inbound control messages. + * Interested observers can attach themselves to the + * subject to get notification of incoming control + * messages. + */ + private[remote] trait ControlMessageSubject { + def attach(observer: ControlMessageObserver): Future[Done] + def detach(observer: ControlMessageObserver): Unit + def stopped: Future[Done] + } + + private[remote] trait ControlMessageObserver { + + /** + * Notification of incoming control message. The message + * of the envelope is always a `ControlMessage`. + */ + def notify(inboundEnvelope: InboundEnvelope): Unit + } + + // messages for the CallbackWrapper + private[InboundControlJunction] sealed trait CallbackMessage + private[InboundControlJunction] final case class Attach(observer: ControlMessageObserver, done: Promise[Done]) + extends CallbackMessage + private[InboundControlJunction] final case class Dettach(observer: ControlMessageObserver) extends CallbackMessage +} + +/** + * INTERNAL API + */ +private[remote] class InboundControlJunction + extends GraphStageWithMaterializedValue[FlowShape[InboundEnvelope, InboundEnvelope], InboundControlJunction.ControlMessageSubject] { + import InboundControlJunction._ + + val in: Inlet[InboundEnvelope] = Inlet("InboundControlJunction.in") + val out: Outlet[InboundEnvelope] = Outlet("InboundControlJunction.out") + override val shape: FlowShape[InboundEnvelope, InboundEnvelope] = FlowShape(in, out) + + override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = { + val stoppedPromise = Promise[Done]() + // FIXME see issue #20503 related to CallbackWrapper, we might implement this in a better way + val logic = new GraphStageLogic(shape) with CallbackWrapper[CallbackMessage] with InHandler with OutHandler { + + private var observers: Vector[ControlMessageObserver] = Vector.empty + + private val callback = getAsyncCallback[CallbackMessage] { + case Attach(observer, done) ⇒ + observers :+= observer + done.success(Done) + case Dettach(observer) ⇒ + observers = observers.filterNot(_ == observer) + } + + override def preStart(): Unit = { + initCallback(callback.invoke) + } + + override def postStop(): Unit = stoppedPromise.success(Done) + + // InHandler + override def onPush(): Unit = { + grab(in) match { + case env: InboundEnvelope if env.message.isInstanceOf[ControlMessage] ⇒ + observers.foreach(_.notify(env)) + pull(in) + case env ⇒ + push(out, env) + } + } + + // OutHandler + override def onPull(): Unit = pull(in) + + setHandlers(in, out, this) + } + + // materialized value + val controlSubject: ControlMessageSubject = new ControlMessageSubject { + override def attach(observer: ControlMessageObserver): Future[Done] = { + val p = Promise[Done]() + logic.invoke(Attach(observer, p)) + p.future + } + + override def detach(observer: ControlMessageObserver): Unit = + logic.invoke(Dettach(observer)) + + override def stopped: Future[Done] = + stoppedPromise.future + } + + (logic, controlSubject) + } +} + +/** + * INTERNAL API + */ +private[remote] object OutboundControlJunction { + private[remote] trait OutboundControlIngress { + def sendControlMessage(message: ControlMessage): Unit + } +} + +/** + * INTERNAL API + */ +private[remote] class OutboundControlJunction( + outboundContext: OutboundContext, outboundEnvelopePool: ObjectPool[ReusableOutboundEnvelope]) + extends GraphStageWithMaterializedValue[FlowShape[OutboundEnvelope, OutboundEnvelope], OutboundControlJunction.OutboundControlIngress] { + import OutboundControlJunction._ + val in: Inlet[OutboundEnvelope] = Inlet("OutboundControlJunction.in") + val out: Outlet[OutboundEnvelope] = Outlet("OutboundControlJunction.out") + override val shape: FlowShape[OutboundEnvelope, OutboundEnvelope] = FlowShape(in, out) + + override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = { + // FIXME see issue #20503 related to CallbackWrapper, we might implement this in a better way + val logic = new GraphStageLogic(shape) with CallbackWrapper[ControlMessage] with InHandler with OutHandler with StageLogging { + import OutboundControlJunction._ + + private val sendControlMessageCallback = getAsyncCallback[ControlMessage](internalSendControlMessage) + private val maxControlMessageBufferSize: Int = outboundContext.settings.Advanced.OutboundControlQueueSize + private val buffer = new ArrayDeque[OutboundEnvelope] + + override def preStart(): Unit = { + initCallback(sendControlMessageCallback.invoke) + } + + // InHandler + override def onPush(): Unit = { + if (buffer.isEmpty && isAvailable(out)) + push(out, grab(in)) + else + buffer.offer(grab(in)) + } + + // OutHandler + override def onPull(): Unit = { + if (buffer.isEmpty && !hasBeenPulled(in)) + pull(in) + else if (!buffer.isEmpty) + push(out, buffer.poll()) + } + + private def internalSendControlMessage(message: ControlMessage): Unit = { + if (buffer.isEmpty && isAvailable(out)) + push(out, wrap(message)) + else if (buffer.size < maxControlMessageBufferSize) + buffer.offer(wrap(message)) + else { + // it's alright to drop control messages + log.debug("Dropping control message [{}] due to full buffer.", Logging.messageClassName(message)) + } + } + + private def wrap(message: ControlMessage): OutboundEnvelope = + outboundEnvelopePool.acquire().init( + recipient = OptionVal.None, message = message, sender = OptionVal.None) + + setHandlers(in, out, this) + } + + // materialized value + val outboundControlIngress = new OutboundControlIngress { + override def sendControlMessage(message: ControlMessage): Unit = + logic.invoke(message) + } + + (logic, outboundControlIngress) + } + +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala new file mode 100644 index 0000000000..0a20fe437d --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala @@ -0,0 +1,439 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.io.RandomAccessFile +import java.nio.channels.FileChannel +import java.nio.file._ +import java.nio.{ ByteBuffer, ByteOrder } +import java.util.concurrent.atomic.AtomicReference +import java.util.concurrent.{ CountDownLatch, TimeUnit } + +import org.agrona.BitUtil +import org.agrona.concurrent.MappedResizeableBuffer + +import scala.annotation.tailrec + +/** + * INTERNAL API + */ +private[remote] trait EventSink { + def alert(code: Int, metadata: Array[Byte]): Unit + def loFreq(code: Int, metadata: Array[Byte]): Unit + def hiFreq(code: Long, param: Long): Unit + + def flushHiFreqBatch(): Unit +} + +/** + * INTERNAL API + */ +private[remote] object IgnoreEventSink extends EventSink { + override def alert(code: Int, metadata: Array[Byte]): Unit = () + override def loFreq(code: Int, metadata: Array[Byte]): Unit = () + override def flushHiFreqBatch(): Unit = () + override def hiFreq(code: Long, param: Long): Unit = () +} + +/** + * INTERNAL API + */ +private[remote] class SynchronizedEventSink(delegate: EventSink) extends EventSink { + override def alert(code: Int, metadata: Array[Byte]): Unit = synchronized { + delegate.alert(code, metadata) + } + + override def loFreq(code: Int, metadata: Array[Byte]): Unit = synchronized { + delegate.loFreq(code, metadata) + } + + override def flushHiFreqBatch(): Unit = synchronized { + delegate.flushHiFreqBatch() + } + + override def hiFreq(code: Long, param: Long): Unit = synchronized { + delegate.hiFreq(code, param) + } +} + +/** + * INTERNAL API + * + * Update clock at various resolutions and aquire the resulting timestamp. + */ +private[remote] trait EventClock { + + def updateWallClock(): Unit + def updateHighSpeedClock(): Unit + + def wallClockPart: Long + def highSpeedPart: Long + +} + +/** + * INTERNAL API + * + * This class is not thread-safe + */ +private[remote] class EventClockImpl extends EventClock { + + private[this] var wallClock: Long = 0 + private[this] var highSpeedClock: Long = 0 + private[this] var highSpeedClockOffset: Long = 0 + + updateWallClock() + + override def updateWallClock(): Unit = { + wallClock = System.currentTimeMillis() + highSpeedClockOffset = System.nanoTime() + highSpeedClock = 0 + } + + override def updateHighSpeedClock(): Unit = { + // TODO: Update wall clock once in a while + highSpeedClock = System.nanoTime() - highSpeedClockOffset + } + + override def wallClockPart: Long = wallClock + override def highSpeedPart: Long = highSpeedClock +} + +/** + * INTERNAL API + */ +private[remote] object RollingEventLogSection { + val HeadPointerOffset = 0L + val LogStateOffset = 8L + val RecordsOffset = 16L + val LogOffset = 0L + + // Log states + val Empty = 0 + val Live = 1 + val Snapshot = 2 + + // Slot states + val Committed = 0 + val Dirty = 1 + + val CommitEntrySize = 4 +} + +/** + * INTERNAL API + */ +private[remote] class RollingEventLogSection( + fileChannel: FileChannel, + offset: Long, + entryCount: Long, + logBufferSize: Long, + recordSize: Int) { + import RollingEventLogSection._ + + require(entryCount > 0, "entryCount must be greater than 0") + require((entryCount & (entryCount - 1)) == 0, "entryCount must be power of two") + private[this] val LogMask: Long = entryCount - 1L + + private[this] val buffers: Array[MappedResizeableBuffer] = Array.tabulate(FlightRecorder.SnapshotCount) { logId ⇒ + val buffer = new MappedResizeableBuffer(fileChannel, offset + logId * logBufferSize, logBufferSize) + // Clear old data + buffer.setMemory(0, logBufferSize.toInt, 0.toByte) + if (logId == 0) buffer.putLong(LogStateOffset, Live) + buffer + } + + def clear(logId: Int): Unit = buffers(logId).setMemory(0, logBufferSize.toInt, 0.toByte) + + /* + * The logic here MUST be kept in sync with its simulated version in RollingEventLogSimulationSpec as it + * is currently the best place to do in-depth stress-testing of this logic. Unfortunately currently there is no + * sane way to use the same code here and in the test, too. + */ + def write(logId: Int, recordBuffer: ByteBuffer): Unit = { + val logBuffer = buffers(logId) + + @tailrec def writeRecord(): Unit = { + // Advance the head + val recordOffset = RecordsOffset + ((logBuffer.getAndAddLong(HeadPointerOffset, 1L) & LogMask) * recordSize) + val payloadOffset = recordOffset + CommitEntrySize + // Signal that we write to the record. This is to prevent concurrent writes to the same slot + // if the head *wraps over* and points again to this location. Without this we would end up with partial or corrupted + // writes to the slot. + if (logBuffer.compareAndSetInt(recordOffset, Committed, Dirty)) { + logBuffer.putBytes(payloadOffset, recordBuffer, recordSize) + //println(logBuffer.getLong(recordOffset + 4)) + + // Now this is free to be overwritten + logBuffer.putIntVolatile(recordOffset, Committed) + } else writeRecord() // Try to claim a new slot + } + + writeRecord() + } + + def markSnapshot(logId: Int): Unit = buffers(logId).putLongVolatile(LogStateOffset, Snapshot) + def markLive(logId: Int): Unit = buffers(logId).putLongVolatile(LogStateOffset, Live) + + def close(): Unit = buffers.foreach(_.close()) +} + +/** + * INTERNAL API + */ +private[remote] object FlightRecorder { + + /** + * @return A created file where the flight recorder file can be written. There are three options, depending + * on ``destination``: + * 1. Empty: a file will be generated in the temporary directory of the OS + * 2. A relative or absolute path ending with ".afr": this file will be used + * 3. A relative or absolute path: this directory will be used, the file will get a random file name + */ + def createFlightRecorderFile(destination: String, fs: FileSystem = FileSystems.getDefault): Path = { + + // TODO safer file permissions (e.g. only user readable on POSIX)? + destination match { + // not defined, use temporary directory + case "" ⇒ Files.createTempFile("artery", ".afr") + + case directory if directory.endsWith(".afr") ⇒ + val path = fs.getPath(directory).toAbsolutePath + if (!Files.exists(path)) { + Files.createDirectories(path.getParent) + Files.createFile(path) + } + path + + case directory ⇒ + val path = fs.getPath(directory).toAbsolutePath + if (!Files.exists(path)) Files.createDirectories(path) + + Files.createTempFile(path, "artery", ".afr") + } + } + + def prepareFileForFlightRecorder(path: Path): FileChannel = { + // Force the size, otherwise memory mapping will fail on *nixes + val randomAccessFile = new RandomAccessFile(path.toFile, "rwd") + randomAccessFile.setLength(FlightRecorder.TotalSize) + randomAccessFile.close() + + FileChannel.open(path, StandardOpenOption.CREATE, StandardOpenOption.WRITE, StandardOpenOption.READ) + } + + val Alignment = 64 * 1024 // Windows is picky about mapped section alignments + + val MagicString = 0x31524641 // "AFR1", little-endian + val GlobalSectionSize = BitUtil.align(24, Alignment) + val StartTimeStampOffset = 4 + + val LogHeaderSize = 16 + val SnapshotCount = 4 + val SnapshotMask = SnapshotCount - 1 + + // TODO: Dummy values right now, format is under construction + val AlertRecordSize = 128 + val LoFreqRecordSize = 128 + val HiFreqBatchSize = 62 + val HiFreqRecordSize = 16 * (HiFreqBatchSize + 2) // (batched events + header) + + val AlertWindow = 256 + val LoFreqWindow = 256 + val HiFreqWindow = 256 // This is counted in batches ! + + val AlertLogSize = BitUtil.align(LogHeaderSize + (AlertWindow * AlertRecordSize), Alignment) + val LoFreqLogSize = BitUtil.align(LogHeaderSize + (LoFreqWindow * LoFreqRecordSize), Alignment) + val HiFreqLogSize = BitUtil.align(LogHeaderSize + (HiFreqWindow * HiFreqRecordSize), Alignment) + + val AlertSectionSize = AlertLogSize * SnapshotCount + val LoFreqSectionSize = LoFreqLogSize * SnapshotCount + val HiFreqSectionSize = HiFreqLogSize * SnapshotCount + + val AlertSectionOffset = GlobalSectionSize + val LoFreqSectionOffset = GlobalSectionSize + AlertSectionSize + val HiFreqSectionOffset = GlobalSectionSize + AlertSectionSize + LoFreqSectionSize + + val TotalSize = GlobalSectionSize + AlertSectionSize + LoFreqSectionSize + HiFreqSectionSize + + val HiFreqEntryCountFieldOffset = 16 +} + +/** + * INTERNAL API + */ +private[remote] sealed trait FlightRecorderStatus +private[remote] case object Running extends FlightRecorderStatus +private[remote] case object ShutDown extends FlightRecorderStatus +private[remote] final case class SnapshotInProgress(latch: CountDownLatch) extends FlightRecorderStatus + +/** + * INTERNAL API + */ +private[remote] class FlightRecorder(val fileChannel: FileChannel) extends AtomicReference[FlightRecorderStatus](Running) { + import FlightRecorder._ + + private[this] val globalSection = new MappedResizeableBuffer(fileChannel, 0, GlobalSectionSize) + + require(SnapshotCount > 0, "SnapshotCount must be greater than 0") + require((SnapshotCount & (SnapshotCount - 1)) == 0, "SnapshotCount must be power of two") + private[this] val SnapshotMask = SnapshotCount - 1 + private[this] val alertLogs = + new RollingEventLogSection( + fileChannel = fileChannel, + offset = AlertSectionOffset, + entryCount = AlertWindow, + logBufferSize = AlertLogSize, + recordSize = AlertRecordSize) + private[this] val loFreqLogs = + new RollingEventLogSection( + fileChannel = fileChannel, + offset = LoFreqSectionOffset, + entryCount = LoFreqWindow, + logBufferSize = LoFreqLogSize, + recordSize = LoFreqRecordSize) + private[this] val hiFreqLogs = + new RollingEventLogSection( + fileChannel = fileChannel, + offset = HiFreqSectionOffset, + entryCount = HiFreqWindow, + logBufferSize = HiFreqLogSize, + recordSize = HiFreqRecordSize) + // No need for volatile, guarded by atomic CAS and set + @volatile private var currentLog = 0 + + init() + + private def init(): Unit = { + globalSection.putInt(0, MagicString) + globalSection.putLong(StartTimeStampOffset, System.currentTimeMillis()) + } + + def snapshot(): Unit = { + // Coalesce concurrent snapshot requests into one, i.e. ignore the "late-comers". + // In other words, this is a critical section in which participants either enter, or just + // simply skip ("Hm, seems someone else already does it. ¯\_(ツ)_/¯ ") + val snapshotLatch = new CountDownLatch(1) + val snapshotInProgress = SnapshotInProgress(snapshotLatch) + if (compareAndSet(Running, snapshotInProgress)) { + val previousLog = currentLog + val nextLog = (currentLog + 1) & SnapshotMask + // Mark new log as Live + hiFreqLogs.clear(nextLog) + loFreqLogs.clear(nextLog) + alertLogs.clear(nextLog) + hiFreqLogs.markLive(nextLog) + loFreqLogs.markLive(nextLog) + alertLogs.markLive(nextLog) + // Redirect traffic to newly allocated log + currentLog = nextLog + // Mark previous log as snapshot + hiFreqLogs.markSnapshot(previousLog) + loFreqLogs.markSnapshot(previousLog) + alertLogs.markSnapshot(previousLog) + fileChannel.force(true) + snapshotLatch.countDown() + compareAndSet(snapshotInProgress, Running) + // At this point it is NOT GUARANTEED that all writers have finished writing to the currently snapshotted + // buffer! + } + } + + def close(): Unit = { + getAndSet(ShutDown) match { + case SnapshotInProgress(latch) ⇒ latch.await(3, TimeUnit.SECONDS) + case _ ⇒ // Nothing to unlock + } + alertLogs.close() + hiFreqLogs.close() + loFreqLogs.close() + globalSection.close() + } + + def createEventSink(): EventSink = new EventSink { + private[this] val clock = new EventClockImpl + private[this] val alertRecordBuffer = ByteBuffer.allocate(AlertRecordSize).order(ByteOrder.LITTLE_ENDIAN) + private[this] val loFreqRecordBuffer = ByteBuffer.allocate(LoFreqRecordSize).order(ByteOrder.LITTLE_ENDIAN) + private[this] val hiFreqBatchBuffer = ByteBuffer.allocate(HiFreqRecordSize).order(ByteOrder.LITTLE_ENDIAN) + private[this] var hiFreqBatchedEntries = 0L + + startHiFreqBatch() + + override def alert(code: Int, metadata: Array[Byte]): Unit = { + if (FlightRecorder.this.get eq Running) { + clock.updateWallClock() + prepareRichRecord(alertRecordBuffer, code, metadata) + alertLogs.write(currentLog, alertRecordBuffer) + flushHiFreqBatch() + snapshot() + } + } + + override def loFreq(code: Int, metadata: Array[Byte]): Unit = { + val status = FlightRecorder.this.get + if (status eq Running) { + clock.updateHighSpeedClock() + prepareRichRecord(loFreqRecordBuffer, code, metadata) + loFreqLogs.write(currentLog, loFreqRecordBuffer) + } + } + + private def prepareRichRecord(recordBuffer: ByteBuffer, code: Int, metadata: Array[Byte]): Unit = { + recordBuffer.clear() + // TODO: This is a bit overkill, needs some smarter scheme later, no need to always store the wallclock + recordBuffer.putLong(clock.wallClockPart) + recordBuffer.putLong(clock.highSpeedPart) + recordBuffer.putInt(code) + // Truncate if necessary + val metadataLength = math.min(LoFreqRecordSize - 32, metadata.length) + recordBuffer.put(metadataLength.toByte) + if (metadataLength > 0) + recordBuffer.put(metadata, 0, metadataLength) + // Don't flip here! We always write fixed size records + recordBuffer.position(0) + } + + // TODO: Try to save as many bytes here as possible! We will see crazy throughput here + override def hiFreq(code: Long, param: Long): Unit = { + val status = FlightRecorder.this.get + if (status eq Running) { + hiFreqBatchedEntries += 1 + hiFreqBatchBuffer.putLong(code) + hiFreqBatchBuffer.putLong(param) + + // If batch is full, time to flush + if (!hiFreqBatchBuffer.hasRemaining) flushHiFreqBatch() + } + } + + private def startHiFreqBatch(): Unit = { + hiFreqBatchBuffer.clear() + // Refresh the nanotime + clock.updateHighSpeedClock() + // Header of the batch will contain our most accurate knowledge of the clock, individual entries do not + // contain any timestamp + hiFreqBatchBuffer.putLong(clock.wallClockPart) + hiFreqBatchBuffer.putLong(clock.highSpeedPart) + // Leave space for the size field + hiFreqBatchBuffer.putLong(0L) + // Reserved for now + hiFreqBatchBuffer.putLong(0L) + // Mow ready to write some more events... + } + + override def flushHiFreqBatch(): Unit = { + val status = FlightRecorder.this.get + if (status eq Running) { + if (hiFreqBatchedEntries > 0) { + hiFreqBatchBuffer.putLong(HiFreqEntryCountFieldOffset, hiFreqBatchedEntries) + hiFreqBatchedEntries = 0 + hiFreqBatchBuffer.position(0) + hiFreqLogs.write(currentLog, hiFreqBatchBuffer) + startHiFreqBatch() + } + } + } + + } +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala new file mode 100644 index 0000000000..1285aa6a36 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala @@ -0,0 +1,92 @@ +package akka.remote.artery + +/** + * INTERNAL API + */ +private[remote] object FlightRecorderEvents { + + // Note: Remember to update dictionary when adding new events! + + val NoMetaData = Array.empty[Byte] + + // Top level remoting events + val Transport_MediaDriverStarted = 0 + val Transport_AeronStarted = 1 + val Transport_AeronErrorLogStarted = 2 + val Transport_TaskRunnerStarted = 3 + val Transport_UniqueAddressSet = 4 + val Transport_MaterializerStarted = 5 + val Transport_StartupFinished = 6 + val Transport_OnAvailableImage = 7 + val Transport_KillSwitchPulled = 8 + val Transport_Stopped = 9 + val Transport_AeronErrorLogTaskStopped = 10 + val Transport_MediaFileDeleted = 11 + val Transport_FlightRecorderClose = 12 + val Transport_SendQueueOverflow = 13 + + // Aeron Sink events + val AeronSink_Started = 50 + val AeronSink_TaskRunnerRemoved = 51 + val AeronSink_PublicationClosed = 52 + val AeronSink_Stopped = 53 + val AeronSink_EnvelopeGrabbed = 54 + val AeronSink_EnvelopeOffered = 55 + val AeronSink_GaveUpEnvelope = 56 + val AeronSink_DelegateToTaskRunner = 57 + val AeronSink_ReturnFromTaskRunner = 58 + + // Aeron Source events + val AeronSource_Started = 70 + val AeronSource_Stopped = 71 + val AeronSource_Received = 72 + val AeronSource_DelegateToTaskRunner = 73 + val AeronSource_ReturnFromTaskRunner = 74 + + // Compression events + val Compression_CompressedActorRef = 90 + val Compression_AllocatedActorRefCompressionId = 91 + val Compression_CompressedManifest = 91 + val Compression_AllocatedManifestCompressionId = 92 + + // Used for presentation of the entries in the flight recorder + lazy val eventDictionary = Map( + Transport_MediaDriverStarted → "Transport: Media driver started", + Transport_AeronStarted → "Transport: Aeron started", + Transport_AeronErrorLogStarted → "Transport: Aeron error log started", + Transport_TaskRunnerStarted → "Transport: Task runner started", + Transport_UniqueAddressSet → "Transport: Unique address set", + Transport_MaterializerStarted → "Transport: Materializer started", + Transport_StartupFinished → "Transport: Startup finished", + Transport_OnAvailableImage → "Transport: onAvailableImage", + Transport_KillSwitchPulled → "Transport: KillSwitch pulled", + Transport_Stopped → "Transport: Stopped", + Transport_AeronErrorLogTaskStopped → "Transport: Aeron errorLog task stopped", + Transport_MediaFileDeleted → "Transport: Media file deleted", + Transport_FlightRecorderClose → "Transport: Flight recorder closed", + Transport_SendQueueOverflow → "Transport: Send queue overflow", + + // Aeron Sink events + AeronSink_Started → "AeronSink: Started", + AeronSink_TaskRunnerRemoved → "AeronSink: Task runner removed", + AeronSink_PublicationClosed → "AeronSink: Publication closed", + AeronSink_Stopped → "AeronSink: Stopped", + AeronSink_EnvelopeGrabbed → "AeronSink: Envelope grabbed", + AeronSink_EnvelopeOffered → "AeronSink: Envelope offered", + AeronSink_GaveUpEnvelope → "AeronSink: Gave up envelope", + AeronSink_DelegateToTaskRunner → "AeronSink: Delegate to task runner", + AeronSink_ReturnFromTaskRunner → "AeronSink: Return from task runner", + + // Aeron Source events + AeronSource_Started → "AeronSource: Started", + AeronSource_Stopped → "AeronSource: Stopped", + AeronSource_Received → "AeronSource: Received", + AeronSource_DelegateToTaskRunner → "AeronSource: Delegate to task runner", + AeronSource_ReturnFromTaskRunner → "AeronSource: Return from task runner", + + // Compression events + Compression_CompressedActorRef → "Compression: Compressed ActorRef", + Compression_AllocatedActorRefCompressionId → "Compression: Allocated ActorRef compression id", + Compression_CompressedManifest → "Compression: Compressed manifest", + Compression_AllocatedManifestCompressionId → "Compression: Allocated manifest compression id").map { case (int, str) ⇒ int.toLong → str } +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderReader.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderReader.scala new file mode 100644 index 0000000000..722755f39f --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderReader.scala @@ -0,0 +1,266 @@ +package akka.remote.artery + +import java.io.{ IOException, RandomAccessFile } +import java.nio.channels.FileChannel +import java.nio.file.{ FileSystems, Path } +import java.time.Instant + +import org.agrona.concurrent.MappedResizeableBuffer + +import scala.collection.{ SortedSet, immutable } + +/** + * Internal API + * + * Minimal utility for dumping a given afr file as text to stdout + */ +object FlightRecorderDump extends App { + require(args.size == 1, "Usage: FlightRecorderDump afr-file") + val path = FileSystems.getDefault.getPath(args(0)) + FlightRecorderReader.dumpToStdout(path) +} + +/** + * Internal API + */ +private[akka] object FlightRecorderReader { + import FlightRecorder._ + + sealed trait LogState + case object Empty extends LogState + case object Live extends LogState + case object Snapshot extends LogState + + case class SectionParameters( + offset: Long, + sectionSize: Long, + logSize: Long, + window: Long, + recordSize: Long, + entriesPerRecord: Long) { + override def toString: String = + s""" + | offset = $offset + | size = $sectionSize + | log size = $logSize + | window = $window + | record size = $recordSize + | max Entries/Record = $entriesPerRecord + | max Total Entries = ${entriesPerRecord * window} + """.stripMargin + } + + val AlertSectionParameters = SectionParameters( + offset = AlertSectionOffset, + sectionSize = AlertSectionSize, + logSize = AlertLogSize, + window = AlertWindow, + recordSize = AlertRecordSize, + entriesPerRecord = 1) + + val LoFreqSectionParameters = SectionParameters( + offset = LoFreqSectionOffset, + sectionSize = LoFreqSectionSize, + logSize = LoFreqLogSize, + window = LoFreqWindow, + recordSize = LoFreqRecordSize, + entriesPerRecord = 1) + + val HiFreqSectionParameters = SectionParameters( + offset = HiFreqSectionOffset, + sectionSize = HiFreqSectionSize, + logSize = HiFreqLogSize, + window = HiFreqWindow, + recordSize = HiFreqRecordSize, + entriesPerRecord = HiFreqBatchSize) + + def dumpToStdout(flightRecorderFile: Path): Unit = { + var raFile: RandomAccessFile = null + var channel: FileChannel = null + var reader: FlightRecorderReader = null + try { + + raFile = new RandomAccessFile(flightRecorderFile.toFile, "rw") + channel = raFile.getChannel + reader = new FlightRecorderReader(channel) + val alerts: Seq[FlightRecorderReader#Entry] = reader.structure.alertLog.logs.flatMap(_.richEntries) + val hiFreq: Seq[FlightRecorderReader#Entry] = reader.structure.hiFreqLog.logs.flatMap(_.compactEntries) + val loFreq: Seq[FlightRecorderReader#Entry] = reader.structure.loFreqLog.logs.flatMap(_.richEntries) + + implicit val ordering = Ordering.fromLessThan[FlightRecorderReader#Entry]((a, b) ⇒ a.timeStamp.isBefore(b.timeStamp)) + val sorted = SortedSet[FlightRecorderReader#Entry](alerts: _*) ++ hiFreq ++ loFreq + + println("--- FLIGHT RECORDER LOG") + sorted.foreach(println) + + } finally { + if (reader ne null) reader.close() + if (channel ne null) channel.close() + if (raFile ne null) raFile.close() + } + } + +} + +/** + * Internal API + */ +private[akka] final class FlightRecorderReader(fileChannel: FileChannel) { + import FlightRecorder._ + import FlightRecorderReader._ + + case class Structure(startTime: Instant, alertLog: RollingLog, loFreqLog: RollingLog, hiFreqLog: RollingLog) { + override def toString: String = + s""" + |AFR file created at $startTime + |Total size : $TotalSize + | + |--- ALERTS + |$alertLog + |--- LOW FREQUENCY EVENTS + |$loFreqLog + |--- HIGH FREQUENCY EVENTS + |$hiFreqLog + """.stripMargin + } + + case class RollingLog(sectionParameters: SectionParameters, logs: immutable.Seq[Log]) { + override def toString: String = + s""" + |$sectionParameters + | + |${logs.mkString(" ", "\n ", "")} + """.stripMargin + } + + case class Log(sectionParameters: SectionParameters, offset: Long, id: Int, state: LogState, head: Long) { + override def toString: String = s"$id: $state \thead = $head (Offset: $offset Size: ${sectionParameters.logSize})" + + def richEntries: Iterator[RichEntry] = { + new Iterator[RichEntry] { + var recordOffset = offset + RollingEventLogSection.RecordsOffset + var recordsLeft = math.min(head, sectionParameters.window) + + override def hasNext: Boolean = recordsLeft > 0 + + override def next(): RichEntry = { + val recordStartOffset = recordOffset + RollingEventLogSection.CommitEntrySize + + // FIXME: extract magic numbers + val metadata = Array.ofDim[Byte](fileBuffer.getByte(recordStartOffset + 20)) + fileBuffer.getBytes(recordStartOffset + 21, metadata) + + val entry = RichEntry( + timeStamp = Instant.ofEpochMilli(fileBuffer.getLong(recordStartOffset)).plusNanos(fileBuffer.getLong(recordStartOffset + 8)), + dirty = fileBuffer.getLong(recordOffset) == RollingEventLogSection.Dirty, + code = fileBuffer.getInt(recordStartOffset + 16), + metadata = metadata) + recordOffset += sectionParameters.recordSize + recordsLeft -= 1 + entry + } + } + } + + def compactEntries: Iterator[CompactEntry] = { + new Iterator[CompactEntry] { + var recordOffset = offset + RollingEventLogSection.RecordsOffset + var entryOffset = recordOffset + RollingEventLogSection.CommitEntrySize + var recordsLeft = math.min(head, sectionParameters.window) + var entriesLeft = -1L + var dirty = false + var timeStamp: Instant = _ + + private def readHeader(): Unit = { + dirty = fileBuffer.getLong(recordOffset) == RollingEventLogSection.Dirty + val entiresHeaderOffset = recordOffset + RollingEventLogSection.CommitEntrySize + entriesLeft = fileBuffer.getLong(entiresHeaderOffset + HiFreqEntryCountFieldOffset) + timeStamp = Instant.ofEpochMilli(fileBuffer.getLong(entiresHeaderOffset)) + .plusNanos(fileBuffer.getLong(entiresHeaderOffset + 8)) + entryOffset = entiresHeaderOffset + 32 + } + + override def hasNext: Boolean = recordsLeft > 0 + + override def next(): CompactEntry = { + if (entriesLeft == -1L) readHeader() + + val entry = CompactEntry( + timeStamp, + dirty, + code = fileBuffer.getLong(entryOffset), + param = fileBuffer.getLong(entryOffset + 8)) + + entriesLeft -= 1 + if (entriesLeft == 0) { + recordOffset += sectionParameters.recordSize + recordsLeft -= 1 + readHeader() + } else { + entryOffset += 16 + } + + entry + } + } + } + } + + trait Entry { + def timeStamp: Instant + } + + case class RichEntry(timeStamp: Instant, dirty: Boolean, code: Long, metadata: Array[Byte]) extends Entry { + override def toString: String = { + val textualCode = FlightRecorderEvents.eventDictionary.getOrElse(code, "").take(34) + val metadataString = new String(metadata, "US-ASCII") + f"[$timeStamp] ${if (dirty) "#" else ""} $code%3s $textualCode%-34s | $metadataString" + } + } + + case class CompactEntry(timeStamp: Instant, dirty: Boolean, code: Long, param: Long) extends Entry { + override def toString: String = { + val textualCode = FlightRecorderEvents.eventDictionary.getOrElse(code, "").take(34) + f"[$timeStamp] ${if (dirty) "#" else ""} $code%3s $textualCode%-34s | $param" + } + } + + private val fileBuffer = new MappedResizeableBuffer(fileChannel, 0, TotalSize) + private var _structure: Structure = _ + rereadStructure() + + def rereadStructure(): Unit = { + if (fileBuffer.getInt(0) != MagicString) { + fileBuffer.close() + throw new IOException(s"Expected magic string AFR1 (0x31524641) but got ${fileBuffer.getInt(0)}") + } + + val alertLog = readRollingLog(AlertSectionParameters) + val loFreqLog = readRollingLog(LoFreqSectionParameters) + val hiFreqLog = readRollingLog(HiFreqSectionParameters) + + _structure = Structure(Instant.ofEpochMilli(fileBuffer.getLong(4)), alertLog, loFreqLog, hiFreqLog) + } + + private def readRollingLog(sectionParameters: SectionParameters): RollingLog = { + val logs = Vector.tabulate(SnapshotCount) { idx ⇒ + readLog(idx, sectionParameters.offset + (idx * sectionParameters.logSize), sectionParameters) + } + RollingLog(sectionParameters, logs) + } + + private def readLog(id: Int, offset: Long, sectionParameters: SectionParameters): Log = { + val state = fileBuffer.getLong(offset + RollingEventLogSection.LogStateOffset) match { + case RollingEventLogSection.Empty ⇒ Empty + case RollingEventLogSection.Live ⇒ Live + case RollingEventLogSection.Snapshot ⇒ Snapshot + case other ⇒ throw new IOException(s"Unrecognized log state: $other in log at offset $offset") + } + Log(sectionParameters, offset, id, state, fileBuffer.getLong(offset + RollingEventLogSection.HeadPointerOffset)) + } + + def structure: Structure = _structure + + def close(): Unit = fileBuffer.close() + +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala new file mode 100644 index 0000000000..1c3abacbbc --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -0,0 +1,265 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.actor.ActorSystem + +import scala.concurrent.duration._ +import scala.util.control.NoStackTrace +import akka.remote.UniqueAddress +import akka.stream.Attributes +import akka.stream.FlowShape +import akka.stream.Inlet +import akka.stream.Outlet +import akka.stream.stage.GraphStage +import akka.stream.stage.GraphStageLogic +import akka.stream.stage.InHandler +import akka.stream.stage.OutHandler +import akka.stream.stage.TimerGraphStageLogic +import akka.util.OptionVal +import akka.Done +import scala.concurrent.Future +import akka.actor.Address + +/** + * INTERNAL API + */ +private[remote] object OutboundHandshake { + + /** + * Stream is failed with this exception if the handshake is not completed + * within the handshake timeout. + */ + class HandshakeTimeoutException(msg: String) extends RuntimeException(msg) with NoStackTrace + + final case class HandshakeReq(from: UniqueAddress, to: Address) extends ControlMessage + final case class HandshakeRsp(from: UniqueAddress) extends Reply + + private sealed trait HandshakeState + private case object Start extends HandshakeState + private case object ReqInProgress extends HandshakeState + private case object Completed extends HandshakeState + + private case object HandshakeTimeout + private case object HandshakeRetryTick + private case object InjectHandshakeTick + +} + +/** + * INTERNAL API + */ +private[remote] class OutboundHandshake( + system: ActorSystem, + outboundContext: OutboundContext, + outboundEnvelopePool: ObjectPool[ReusableOutboundEnvelope], + timeout: FiniteDuration, + retryInterval: FiniteDuration, injectHandshakeInterval: FiniteDuration) + extends GraphStage[FlowShape[OutboundEnvelope, OutboundEnvelope]] { + + val in: Inlet[OutboundEnvelope] = Inlet("OutboundHandshake.in") + val out: Outlet[OutboundEnvelope] = Outlet("OutboundHandshake.out") + override val shape: FlowShape[OutboundEnvelope, OutboundEnvelope] = FlowShape(in, out) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new TimerGraphStageLogic(shape) with InHandler with OutHandler { + import OutboundHandshake._ + + private var handshakeState: HandshakeState = Start + private var pendingMessage: OutboundEnvelope = null + private var injectHandshakeTickScheduled = false + + // InHandler + override def onPush(): Unit = { + if (handshakeState != Completed) + throw new IllegalStateException(s"onPush before handshake completed, was [$handshakeState]") + + // inject a HandshakeReq once in a while to trigger a new handshake when destination + // system has been restarted + if (injectHandshakeTickScheduled) { + push(out, grab(in)) + } else { + pushHandshakeReq() + pendingMessage = grab(in) + } + } + + // OutHandler + override def onPull(): Unit = { + handshakeState match { + case Completed ⇒ + if (pendingMessage eq null) + pull(in) + else { + push(out, pendingMessage) + pendingMessage = null + } + + case Start ⇒ + val uniqueRemoteAddress = outboundContext.associationState.uniqueRemoteAddress + if (uniqueRemoteAddress.isCompleted) { + handshakeState = Completed + } else { + // will pull when handshake reply is received (uniqueRemoteAddress completed) + handshakeState = ReqInProgress + scheduleOnce(HandshakeTimeout, timeout) + schedulePeriodically(HandshakeRetryTick, retryInterval) + + // The InboundHandshake stage will complete the uniqueRemoteAddress future + // when it receives the HandshakeRsp reply + implicit val ec = materializer.executionContext + uniqueRemoteAddress.foreach { + getAsyncCallback[UniqueAddress] { a ⇒ + if (handshakeState != Completed) { + handshakeCompleted() + if (isAvailable(out)) + pull(in) + } + }.invoke + } + } + + // always push a HandshakeReq as the first message + pushHandshakeReq() + + case ReqInProgress ⇒ // will pull when handshake reply is received + } + } + + private def pushHandshakeReq(): Unit = { + injectHandshakeTickScheduled = true + scheduleOnce(InjectHandshakeTick, injectHandshakeInterval) + val env: OutboundEnvelope = outboundEnvelopePool.acquire().init( + recipient = OptionVal.None, message = HandshakeReq(outboundContext.localAddress, outboundContext.remoteAddress), sender = OptionVal.None) + push(out, env) + } + + private def handshakeCompleted(): Unit = { + handshakeState = Completed + cancelTimer(HandshakeRetryTick) + cancelTimer(HandshakeTimeout) + } + + override protected def onTimer(timerKey: Any): Unit = + timerKey match { + case InjectHandshakeTick ⇒ + // next onPush message will trigger sending of HandshakeReq + injectHandshakeTickScheduled = false + case HandshakeRetryTick ⇒ + if (isAvailable(out)) + pushHandshakeReq() + case HandshakeTimeout ⇒ + failStage(new HandshakeTimeoutException( + s"Handshake with [${outboundContext.remoteAddress}] did not complete within ${timeout.toMillis} ms")) + } + + setHandlers(in, out, this) + } + +} + +/** + * INTERNAL API + */ +private[remote] class InboundHandshake(inboundContext: InboundContext, inControlStream: Boolean) extends GraphStage[FlowShape[InboundEnvelope, InboundEnvelope]] { + val in: Inlet[InboundEnvelope] = Inlet("InboundHandshake.in") + val out: Outlet[InboundEnvelope] = Outlet("InboundHandshake.out") + override val shape: FlowShape[InboundEnvelope, InboundEnvelope] = FlowShape(in, out) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new TimerGraphStageLogic(shape) with OutHandler with StageLogging { + import OutboundHandshake._ + + // InHandler + if (inControlStream) + setHandler(in, new InHandler { + override def onPush(): Unit = { + val env = grab(in) + env.message match { + case HandshakeReq(from, to) ⇒ onHandshakeReq(from, to) + case HandshakeRsp(from) ⇒ + after(inboundContext.completeHandshake(from)) { + pull(in) + } + case _ ⇒ + onMessage(env) + } + } + }) + else + setHandler(in, new InHandler { + override def onPush(): Unit = { + val env = grab(in) + env.message match { + case HandshakeReq(from, to) ⇒ onHandshakeReq(from, to) + case _ ⇒ + onMessage(env) + } + } + }) + + private def onHandshakeReq(from: UniqueAddress, to: Address): Unit = { + if (to == inboundContext.localAddress.address) { + after(inboundContext.completeHandshake(from)) { + inboundContext.sendControl(from.address, HandshakeRsp(inboundContext.localAddress)) + pull(in) + } + } else { + log.warning( + "Dropping Handshake Request from [{}] addressed to unknown local address [{}]. " + + "Local address is [{}]. Check that the sending system uses the same " + + "address to contact recipient system as defined in the " + + "'akka.remote.artery.canonical.hostname' of the recipient system. " + + "The name of the ActorSystem must also match.", + from, to, inboundContext.localAddress.address) + + pull(in) + } + } + + private def after(first: Future[Done])(thenInside: ⇒ Unit): Unit = { + first.value match { + case Some(_) ⇒ + // This in the normal case (all but the first). The future will be completed + // because handshake was already completed. Note that we send those HandshakeReq + // periodically. + thenInside + case None ⇒ + implicit val ec = materializer.executionContext + first.onComplete { _ ⇒ + getAsyncCallback[Done](_ ⇒ thenInside).invoke(Done) + } + } + + } + + private def onMessage(env: InboundEnvelope): Unit = { + if (isKnownOrigin(env)) + push(out, env) + else { + if (log.isDebugEnabled) + log.debug( + s"Dropping message [{}] from unknown system with UID [{}]. " + + "This system with UID [{}] was probably restarted. " + + "Messages will be accepted when new handshake has been completed.", + env.message.getClass.getName, env.originUid, inboundContext.localAddress.uid) + pull(in) + } + } + + private def isKnownOrigin(env: InboundEnvelope): Boolean = { + // the association is passed in the envelope from the Decoder stage to avoid + // additional lookup. The second OR case is because if we didn't use fusing it + // would be possible that it was not found by Decoder (handshake not completed yet) + env.association.isDefined || inboundContext.association(env.originUid).isDefined + } + + // OutHandler + override def onPull(): Unit = pull(in) + + setHandler(out, this) + + } + +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/ImmutableLongMap.scala b/akka-remote/src/main/scala/akka/remote/artery/ImmutableLongMap.scala new file mode 100644 index 0000000000..4ce664b94a --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/ImmutableLongMap.scala @@ -0,0 +1,136 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.annotation.tailrec +import akka.util.OptionVal +import scala.reflect.ClassTag +import java.util.Arrays +import akka.util.HashCode + +/** + * INTERNAL API + */ +private[akka] object ImmutableLongMap { + def empty[A >: Null](implicit t: ClassTag[A]): ImmutableLongMap[A] = + new ImmutableLongMap(Array.emptyLongArray, Array.empty) + + private val MaxScanLength = 10 +} + +/** + * INTERNAL API + * Specialized Map for primitive `Long` keys to avoid allocations (boxing). + * Keys and values are backed by arrays and lookup is performed with binary + * search. It's intended for rather small (<1000) maps. + */ +private[akka] class ImmutableLongMap[A >: Null] private ( + private val keys: Array[Long], private val values: Array[A])(implicit t: ClassTag[A]) { + import ImmutableLongMap.MaxScanLength + + val size: Int = keys.length + + /** + * Worst case `O(log n)`, allocation free. + */ + def get(key: Long): OptionVal[A] = { + val i = Arrays.binarySearch(keys, key) + if (i >= 0) OptionVal(values(i)) + else OptionVal.None + } + + /** + * Worst case `O(log n)`, allocation free. + */ + def contains(key: Long): Boolean = { + Arrays.binarySearch(keys, key) >= 0 + } + + /** + * Worst case `O(log n)`, creates new `ImmutableLongMap` + * with copies of the internal arrays for the keys and + * values. + */ + def updated(key: Long, value: A): ImmutableLongMap[A] = { + if (size == 0) + new ImmutableLongMap(Array(key), Array(value)) + else { + val i = Arrays.binarySearch(keys, key) + if (i >= 0) { + // existing key, replace value + val newValues = Array.ofDim[A](values.length) + System.arraycopy(values, 0, newValues, 0, values.length) + newValues(i) = value + new ImmutableLongMap(keys, newValues) + } else { + // insert the entry at the right position, and keep the arrays sorted + val j = -(i + 1) + val newKeys = Array.ofDim[Long](size + 1) + System.arraycopy(keys, 0, newKeys, 0, j) + newKeys(j) = key + System.arraycopy(keys, j, newKeys, j + 1, keys.length - j) + + val newValues = Array.ofDim[A](size + 1) + System.arraycopy(values, 0, newValues, 0, j) + newValues(j) = value + System.arraycopy(values, j, newValues, j + 1, values.length - j) + + new ImmutableLongMap(newKeys, newValues) + } + } + } + + def remove(key: Long): ImmutableLongMap[A] = { + val i = Arrays.binarySearch(keys, key) + if (i >= 0) { + if (size == 1) + ImmutableLongMap.empty + else { + val newKeys = Array.ofDim[Long](size - 1) + System.arraycopy(keys, 0, newKeys, 0, i) + System.arraycopy(keys, i + 1, newKeys, i, keys.length - i - 1) + + val newValues = Array.ofDim[A](size - 1) + System.arraycopy(values, 0, newValues, 0, i) + System.arraycopy(values, i + 1, newValues, i, values.length - i - 1) + + new ImmutableLongMap(newKeys, newValues) + } + } else + this + } + + /** + * All keys + */ + def keysIterator: Iterator[Long] = + keys.iterator + + override def toString: String = + keysIterator.map(key ⇒ s"$key -> ${get(key).get}").mkString("ImmutableLongMap(", ", ", ")") + + override def hashCode: Int = { + var result = HashCode.SEED + result = HashCode.hash(result, keys) + result = HashCode.hash(result, values) + result + } + + override def equals(obj: Any): Boolean = obj match { + case other: ImmutableLongMap[A] ⇒ + if (other eq this) true + else if (size != other.size) false + else if (size == 0 && other.size == 0) true + else { + @tailrec def check(i: Int): Boolean = { + if (i == size) true + else if (keys(i) == other.keys(i) && values(i) == other.values(i)) + check(i + 1) // recur, next elem + else false + } + check(0) + } + case _ ⇒ false + } +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala b/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala new file mode 100644 index 0000000000..cf2e8b3b54 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala @@ -0,0 +1,135 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.actor.InternalActorRef +import akka.util.OptionVal +import akka.actor.Address +import akka.actor.ActorRef + +/** + * INTERNAL API + */ +private[remote] object InboundEnvelope { + /** + * Only used in tests + */ + def apply( + recipient: OptionVal[InternalActorRef], + message: AnyRef, + sender: OptionVal[ActorRef], + originUid: Long, + association: OptionVal[OutboundContext]): InboundEnvelope = { + val env = new ReusableInboundEnvelope + env.init(recipient, sender, originUid, -1, "", 0, null, association) + .withMessage(message) + } + +} + +/** + * INTERNAL API + */ +private[remote] trait InboundEnvelope { + def recipient: OptionVal[InternalActorRef] + def sender: OptionVal[ActorRef] + def originUid: Long + def association: OptionVal[OutboundContext] + + def serializer: Int + def classManifest: String + def message: AnyRef + def envelopeBuffer: EnvelopeBuffer + + def flags: Byte + def flag(byteFlag: ByteFlag): Boolean + + def withMessage(message: AnyRef): InboundEnvelope + + def releaseEnvelopeBuffer(): InboundEnvelope + + def withRecipient(ref: InternalActorRef): InboundEnvelope +} + +/** + * INTERNAL API + */ +private[remote] object ReusableInboundEnvelope { + def createObjectPool(capacity: Int) = new ObjectPool[ReusableInboundEnvelope]( + capacity, + create = () ⇒ new ReusableInboundEnvelope, clear = inEnvelope ⇒ inEnvelope.asInstanceOf[ReusableInboundEnvelope].clear()) +} + +/** + * INTERNAL API + */ +private[remote] final class ReusableInboundEnvelope extends InboundEnvelope { + private var _recipient: OptionVal[InternalActorRef] = OptionVal.None + private var _sender: OptionVal[ActorRef] = OptionVal.None + private var _originUid: Long = 0L + private var _association: OptionVal[OutboundContext] = OptionVal.None + private var _serializer: Int = -1 + private var _classManifest: String = null + private var _flags: Byte = 0 + private var _message: AnyRef = null + private var _envelopeBuffer: EnvelopeBuffer = null + + override def recipient: OptionVal[InternalActorRef] = _recipient + override def sender: OptionVal[ActorRef] = _sender + override def originUid: Long = _originUid + override def association: OptionVal[OutboundContext] = _association + override def serializer: Int = _serializer + override def classManifest: String = _classManifest + override def message: AnyRef = _message + override def envelopeBuffer: EnvelopeBuffer = _envelopeBuffer + + override def flags: Byte = _flags + override def flag(byteFlag: ByteFlag): Boolean = byteFlag.isEnabled(_flags) + + override def withMessage(message: AnyRef): InboundEnvelope = { + _message = message + this + } + + def releaseEnvelopeBuffer(): InboundEnvelope = { + _envelopeBuffer = null + this + } + + def withRecipient(ref: InternalActorRef): InboundEnvelope = { + _recipient = OptionVal(ref) + this + } + + def clear(): Unit = { + _recipient = OptionVal.None + _message = null + _sender = OptionVal.None + _originUid = 0L + _association = OptionVal.None + } + + def init( + recipient: OptionVal[InternalActorRef], + sender: OptionVal[ActorRef], + originUid: Long, + serializer: Int, + classManifest: String, + flags: Byte, + envelopeBuffer: EnvelopeBuffer, + association: OptionVal[OutboundContext]): InboundEnvelope = { + _recipient = recipient + _sender = sender + _originUid = originUid + _serializer = serializer + _classManifest = classManifest + _flags = flags + _envelopeBuffer = envelopeBuffer + _association = association + this + } + + override def toString: String = + s"InboundEnvelope($recipient, $message, $sender, $originUid, $association)" +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala b/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala new file mode 100644 index 0000000000..edbbc289e5 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala @@ -0,0 +1,68 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.stream.Attributes +import akka.stream.FlowShape +import akka.stream.Inlet +import akka.stream.Outlet +import akka.stream.stage.GraphStage +import akka.stream.stage.GraphStageLogic +import akka.stream.stage.InHandler +import akka.stream.stage.OutHandler +import akka.remote.UniqueAddress +import akka.util.OptionVal +import akka.event.Logging +import akka.remote.HeartbeatMessage +import akka.actor.ActorSelectionMessage + +/** + * INTERNAL API + */ +private[remote] class InboundQuarantineCheck(inboundContext: InboundContext) extends GraphStage[FlowShape[InboundEnvelope, InboundEnvelope]] { + val in: Inlet[InboundEnvelope] = Inlet("InboundQuarantineCheck.in") + val out: Outlet[InboundEnvelope] = Outlet("InboundQuarantineCheck.out") + override val shape: FlowShape[InboundEnvelope, InboundEnvelope] = FlowShape(in, out) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new GraphStageLogic(shape) with InHandler with OutHandler with StageLogging { + + override protected def logSource = classOf[InboundQuarantineCheck] + + // InHandler + override def onPush(): Unit = { + val env = grab(in) + env.association match { + case OptionVal.None ⇒ + // unknown, handshake not completed + push(out, env) + case OptionVal.Some(association) ⇒ + if (association.associationState.isQuarantined(env.originUid)) { + if (log.isDebugEnabled) + log.debug( + "Dropping message [{}] from [{}#{}] because the system is quarantined", + Logging.messageClassName(env.message), association.remoteAddress, env.originUid) + // avoid starting outbound stream for heartbeats + if (!env.message.isInstanceOf[Quarantined] && !isHeartbeat(env.message)) + inboundContext.sendControl( + association.remoteAddress, + Quarantined(inboundContext.localAddress, UniqueAddress(association.remoteAddress, env.originUid))) + pull(in) + } else + push(out, env) + } + } + + private def isHeartbeat(msg: Any): Boolean = msg match { + case _: HeartbeatMessage ⇒ true + case ActorSelectionMessage(_: HeartbeatMessage, _, _) ⇒ true + case _ ⇒ false + } + + // OutHandler + override def onPull(): Unit = pull(in) + + setHandlers(in, out, this) + } +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/LruBoundedCache.scala b/akka-remote/src/main/scala/akka/remote/artery/LruBoundedCache.scala new file mode 100644 index 0000000000..ae4fa13b29 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/LruBoundedCache.scala @@ -0,0 +1,224 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.util.{ OptionVal, Unsafe } + +import scala.annotation.tailrec +import scala.reflect.ClassTag + +/** + * INTERNAL API + */ +private[remote] object FastHash { + + // Fast hash based on the 128 bit Xorshift128+ PRNG. Mixes in character bits into the random generator state. + def ofString(s: String): Int = { + val chars = Unsafe.instance.getObject(s, EnvelopeBuffer.StringValueFieldOffset).asInstanceOf[Array[Char]] + var s0: Long = 391408 + var s1: Long = 601258 + var i = 0 + + while (i < chars.length) { + var x = s0 ^ chars(i).toLong // Mix character into PRNG state + var y = s1 + + // Xorshift128+ round + s0 = y + x ^= x << 23 + y ^= (y >>> 26) + x ^= (x >>> 17) + s1 = x ^ y + + i += 1 + } + + (s0 + s1).toInt + } + +} + +/** + * INTERNAL API + */ +private[akka] case class CacheStatistics(entries: Int, maxProbeDistance: Int, averageProbeDistance: Double) + +/** + * INTERNAL API + * + * This class is based on a Robin-Hood hashmap + * (http://www.sebastiansylvan.com/post/robin-hood-hashing-should-be-your-default-hash-table-implementation/) + * with backshift (http://codecapsule.com/2013/11/17/robin-hood-hashing-backward-shift-deletion/). + * + * The main modification compared to an RH hashmap is that it never grows the map (no rehashes) instead it is allowed + * to kick out entires that are considered old. The implementation tries to keep the map close to full, only evicting + * old entries when needed. + */ +private[akka] abstract class LruBoundedCache[K: ClassTag, V <: AnyRef: ClassTag](capacity: Int, evictAgeThreshold: Int) { + require(capacity > 0, "Capacity must be larger than zero") + require((capacity & (capacity - 1)) == 0, "Capacity must be power of two") + require(evictAgeThreshold <= capacity, "Age threshold must be less than capacity.") + + private[this] val Mask = capacity - 1 + + // Practically guarantee an overflow + private[this] var epoch = Int.MaxValue - 1 + + private[this] val keys = Array.ofDim[K](capacity) + private[this] val values = Array.ofDim[V](capacity) + private[this] val hashes = Array.ofDim[Int](capacity) + private[this] val epochs = Array.fill[Int](capacity)(epoch - evictAgeThreshold) // Guarantee existing "values" are stale + + final def get(k: K): Option[V] = { + val h = hash(k) + + @tailrec def find(position: Int, probeDistance: Int): Option[V] = { + val otherProbeDistance = probeDistanceOf(position) + if (values(position) eq null) { + None + } else if (probeDistance > otherProbeDistance) { + None + } else if (hashes(position) == h && k == keys(position)) { + Some(values(position)) + } else { + find((position + 1) & Mask, probeDistance + 1) + } + } + + find(position = h & Mask, probeDistance = 0) + } + + final def stats: CacheStatistics = { + var i = 0 + var sum = 0 + var count = 0 + var max = 0 + while (i < hashes.length) { + if (values(i) ne null) { + val dist = probeDistanceOf(i) + sum += dist + count += 1 + max = math.max(dist, max) + } + i += 1 + } + CacheStatistics(count, max, sum.toDouble / count) + } + + final def getOrCompute(k: K): V = { + val h = hash(k) + epoch += 1 + + @tailrec def findOrCalculate(position: Int, probeDistance: Int): V = { + if (values(position) eq null) { + val value = compute(k) + if (isCacheable(value)) { + keys(position) = k + values(position) = value + hashes(position) = h + epochs(position) = epoch + } + value + } else { + val otherProbeDistance = probeDistanceOf(position) + // If probe distance of the element we try to get is larger than the current slot's, then the element cannot be in + // the table since because of the Robin-Hood property we would have swapped it with the current element. + if (probeDistance > otherProbeDistance) { + val value = compute(k) + if (isCacheable(value)) move(position, k, h, value, epoch, probeDistance) + value + } else if (hashes(position) == h && k == keys(position)) { + // Update usage + epochs(position) = epoch + values(position) + } else { + // This is not our slot yet + findOrCalculate((position + 1) & Mask, probeDistance + 1) + } + } + } + + findOrCalculate(position = h & Mask, probeDistance = 0) + } + + @tailrec private def removeAt(position: Int): Unit = { + val next = (position + 1) & Mask + if ((values(next) eq null) || probeDistanceOf(next) == 0) { + // Next is not movable, just empty this slot + values(position) = null.asInstanceOf[V] + } else { + // Shift the next slot here + keys(position) = keys(next) + values(position) = values(next) + hashes(position) = hashes(next) + epochs(position) = epochs(next) + // remove the shifted slot + removeAt(next) + } + } + + // Wraparound distance of the element that is in this slot. (X + capacity) & Mask ensures that there are no + // negative numbers on wraparound + private def probeDistanceOf(slot: Int): Int = probeDistanceOf(idealSlot = hashes(slot) & Mask, actualSlot = slot) + + // Protected for exposing it to unit tests + protected def probeDistanceOf(idealSlot: Int, actualSlot: Int) = ((actualSlot - idealSlot) + capacity) & Mask + + @tailrec private def move(position: Int, k: K, h: Int, value: V, elemEpoch: Int, probeDistance: Int): Unit = { + if (values(position) eq null) { + // Found an empty place, done. + keys(position) = k + values(position) = value + hashes(position) = h + epochs(position) = elemEpoch // Do NOT update the epoch of the elem. It was not touched, just moved + } else { + val otherEpoch = epochs(position) + // Check if the current entry is too old + if (epoch - otherEpoch >= evictAgeThreshold) { + // Remove old entry to make space + removeAt(position) + // Try to insert our element in hand to its ideal slot + move(h & Mask, k, h, value, elemEpoch, 0) + } else { + val otherProbeDistance = probeDistanceOf(position) + val otherEpoch = epochs(position) + + // Check whose probe distance is larger. The one with the larger one wins the slot. + if (probeDistance > otherProbeDistance) { + // Due to the Robin-Hood property, we now take away this slot from the "richer" and take it for ourselves + val otherKey = keys(position) + val otherValue = values(position) + val otherHash = hashes(position) + + keys(position) = k + values(position) = value + hashes(position) = h + epochs(position) = elemEpoch + + // Move out the old one + move((position + 1) & Mask, otherKey, otherHash, otherValue, otherEpoch, otherProbeDistance + 1) + } else { + // We are the "richer" so we need to find another slot + move((position + 1) & Mask, k, h, value, elemEpoch, probeDistance + 1) + } + + } + } + + } + + protected def compute(k: K): V + + protected def hash(k: K): Int + + protected def isCacheable(v: V): Boolean + + override def toString = + s"LruBoundedCache(" + + s" values = ${values.mkString("[", ",", "]")}," + + s" hashes = ${hashes.map(_ & Mask).mkString("[", ",", "]")}," + + s" epochs = ${epochs.mkString("[", ",", "]")}," + + s" distances = ${(0 until hashes.length).map(probeDistanceOf).mkString("[", ",", "]")}," + + s" $epoch)" +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala b/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala new file mode 100644 index 0000000000..d60f43f476 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala @@ -0,0 +1,96 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.actor.ActorRef +import akka.actor.ActorSelection +import akka.actor.ActorSelectionMessage +import akka.actor.Address +import akka.actor.ExtendedActorSystem +import akka.actor.InternalActorRef +import akka.actor.LocalRef +import akka.actor.PossiblyHarmful +import akka.actor.RepointableRef +import akka.dispatch.sysmsg.SystemMessage +import akka.event.Logging +import akka.remote.RemoteActorRefProvider +import akka.remote.RemoteRef +import akka.util.OptionVal +import akka.event.LoggingReceive + +/** + * INTERNAL API + */ +private[remote] class MessageDispatcher( + system: ExtendedActorSystem, + provider: RemoteActorRefProvider) { + + private val remoteDaemon = provider.remoteDaemon + private val log = Logging(system, getClass.getName) + private val debugLogEnabled = log.isDebugEnabled + + def dispatch( + recipient: InternalActorRef, + message: AnyRef, + senderOption: OptionVal[ActorRef], + originAddress: OptionVal[Address]): Unit = { + + import provider.remoteSettings.Artery._ + import Logging.messageClassName + + val sender: ActorRef = senderOption.getOrElse(system.deadLetters) + val originalReceiver = recipient.path + + recipient match { + + case `remoteDaemon` ⇒ + if (UntrustedMode) { + if (debugLogEnabled) log.debug( + "dropping daemon message [{}] in untrusted mode", + messageClassName(message)) + } else { + if (LogReceive && debugLogEnabled) log.debug( + "received daemon message [{}] from [{}]", + messageClassName(message), senderOption.getOrElse(originAddress.getOrElse(""))) + remoteDaemon ! message + } + + case l @ (_: LocalRef | _: RepointableRef) if l.isLocal ⇒ + if (LogReceive && debugLogEnabled) log.debug( + "received message [{}] to [{}] from [{}]", + messageClassName(message), recipient, senderOption.getOrElse("")) + message match { + case sel: ActorSelectionMessage ⇒ + if (UntrustedMode && (!TrustedSelectionPaths.contains(sel.elements.mkString("/", "/", "")) || + sel.msg.isInstanceOf[PossiblyHarmful] || l != provider.rootGuardian)) { + if (debugLogEnabled) log.debug( + "operating in UntrustedMode, dropping inbound actor selection to [{}], " + + "allow it by adding the path to 'akka.remote.trusted-selection-paths' configuration", + sel.elements.mkString("/", "/", "")) + } else + // run the receive logic for ActorSelectionMessage here to make sure it is not stuck on busy user actor + ActorSelection.deliverSelection(l, sender, sel) + case msg: PossiblyHarmful if UntrustedMode ⇒ + if (debugLogEnabled) log.debug( + "operating in UntrustedMode, dropping inbound PossiblyHarmful message of type [{}] to [{}] from [{}]", + messageClassName(msg), recipient, senderOption.getOrElse(originAddress.getOrElse(""))) + case msg: SystemMessage ⇒ l.sendSystemMessage(msg) + case msg ⇒ l.!(msg)(sender) + } + + case r @ (_: RemoteRef | _: RepointableRef) if !r.isLocal && !UntrustedMode ⇒ + if (LogReceive && debugLogEnabled) log.debug( + "received remote-destined message [{}] to [{}] from [{}]", + messageClassName(message), recipient, senderOption.getOrElse(originAddress.getOrElse(""))) + // if it was originally addressed to us but is in fact remote from our point of view (i.e. remote-deployed) + r.!(message)(sender) + + case r ⇒ log.error( + "dropping message [{}] for unknown recipient [{}] from [{}]", + messageClassName(message), r, senderOption.getOrElse(originAddress.getOrElse(""))) + + } + } + +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/MetadataEnvelopeSerializer.scala b/akka-remote/src/main/scala/akka/remote/artery/MetadataEnvelopeSerializer.scala new file mode 100644 index 0000000000..4e2be61121 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/MetadataEnvelopeSerializer.scala @@ -0,0 +1,173 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery + +import java.nio.{ ByteBuffer, ByteOrder } + +import akka.actor.ExtendedActorSystem +import akka.serialization.Serialization +import akka.util.ByteString.ByteString1C +import akka.util.{ ByteString, ByteStringBuilder } + +/** + * INTERNAL API + */ +private[akka] object MetadataEnvelopeSerializer { + + private[akka] val EmptyRendered = { + implicit val _ByteOrder = ByteOrder.LITTLE_ENDIAN + + val bsb = new ByteStringBuilder + bsb.putInt(0) // 0 length + bsb.result + } + + // key/length of a metadata element are encoded within a single integer: + // supports keys in the range of <0-31> + final val EntryKeyMask = Integer.parseInt("1111 1000 0000 0000 0000 0000 0000 000".replace(" ", ""), 2) + def maskEntryKey(k: Byte): Int = (k.toInt << 26) & EntryKeyMask + def unmaskEntryKey(kv: Int): Byte = ((kv & EntryKeyMask) >> 26).toByte + + final val EntryLengthMask = ~EntryKeyMask + + def maskEntryLength(k: Int): Int = k & EntryLengthMask + def unmaskEntryLength(kv: Int): Int = kv & EntryLengthMask + + def muxEntryKeyLength(k: Byte, l: Int): Int = { + maskEntryKey(k) | maskEntryLength(l) + } + + def serialize(metadatas: MetadataMap[ByteString], headerBuilder: HeaderBuilder): Unit = { + if (metadatas.isEmpty) headerBuilder.clearMetadataContainer() + else { + val container = new MetadataMapRendering(metadatas) + headerBuilder.setMetadataContainer(container.render()) + } + } + + def deserialize(system: ExtendedActorSystem, originUid: Long, serialization: Serialization, + serializer: Int, classManifest: String, envelope: EnvelopeBuffer): AnyRef = { + serialization.deserializeByteBuffer(envelope.byteBuffer, serializer, classManifest) + } +} + +/** + * INTERNAL API + * + * The metadata section is stored as ByteString (prefixed with Int length field, + * the same way as any other literal), however the internal structure of it is as follows: + * + * {{{ + * Metadata entry: + * + * 0 1 2 3 + * 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + * | Key | Metadata entry length | + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + * | ... metadata entry ... | + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + * }}} + * + */ +private[akka] final class MetadataMapRendering(val metadataMap: MetadataMap[ByteString]) extends AnyVal { + import MetadataEnvelopeSerializer._ + + def render(): ByteString = + if (metadataMap.isEmpty) { + // usually no-one will want to render an empty metadata section - it should not be there at all + EmptyRendered + } else { + implicit val _ByteOrder = ByteOrder.LITTLE_ENDIAN + + // TODO optimise this, we could just count along the way and then prefix with the length + val totalSize = 4 /* length int field */ + metadataMap.usedSlots * 4 /* metadata length */ + metadataMap.foldLeftValues(0)(_ + _.length) + val b = new ByteStringBuilder // TODO could we reuse one? + b.sizeHint(totalSize) + + b.putInt(totalSize - 4 /* don't count itself, the length prefix */ ) + // TODO: move through and then prepend length + metadataMap.foreach { (key: Byte, value: ByteString) ⇒ + // TODO try to remove allocation? Iterator otherwise, but that's also allocation + val kl = muxEntryKeyLength(key, value.length) + b.putInt(kl) + value match { + case c: ByteString1C ⇒ c.appendToBuilder(b) // uses putByteArrayUnsafe + case _ ⇒ b ++= value + } + } + b.result() + } +} + +/** INTERNAL API */ +private[akka] object MetadataMapParsing { + import MetadataEnvelopeSerializer._ + + /** Allocates an MetadataMap */ + def parse(envelope: InboundEnvelope): MetadataMapRendering = { + val buf = envelope.envelopeBuffer.byteBuffer + buf.position(EnvelopeBuffer.MetadataContainerAndLiteralSectionOffset) + parseRaw(buf) + } + + /** + * INTERNAL API, only for testing + * The buffer MUST be already at the right position where the Metadata container starts. + */ + private[akka] def parseRaw(buf: ByteBuffer) = { + buf.order(ByteOrder.LITTLE_ENDIAN) + val metadataContainerLength = buf.getInt() + val endOfMetadataPos = metadataContainerLength + buf.position() + val map = MetadataMap[ByteString]() + + while (buf.position() < endOfMetadataPos) { + val kl = buf.getInt() + val k = unmaskEntryKey(kl) // k + val l = unmaskEntryLength(kl) // l + + val arr = Array.ofDim[Byte](l) + buf.get(arr) + val metadata = ByteString1C(arr) // avoids copying again + map.set(k, metadata) + } + + new MetadataMapRendering(map) + } + + /** Implemented in a way to avoid allocations of any envelopes or arrays */ + def applyAllRemoteMessageReceived(instruments: Vector[RemoteInstrument], envelope: InboundEnvelope): Unit = { + val buf = envelope.envelopeBuffer.byteBuffer + buf.position(EnvelopeBuffer.MetadataContainerAndLiteralSectionOffset) + applyAllRemoteMessageReceivedRaw(instruments, envelope, buf) + } + + /** + * INTERNAL API, only for testing + * The buffer MUST be already at the right position where the Metadata container starts. + */ + private[akka] def applyAllRemoteMessageReceivedRaw(instruments: Vector[RemoteInstrument], envelope: InboundEnvelope, buf: ByteBuffer): Unit = { + buf.order(ByteOrder.LITTLE_ENDIAN) + + val metadataContainerLength = buf.getInt() + val endOfMetadataPos = metadataContainerLength + buf.position() + + while (buf.position() < endOfMetadataPos) { + val keyAndLength = buf.getInt() + val key = unmaskEntryKey(keyAndLength) + val length = unmaskEntryLength(keyAndLength) + + val arr = Array.ofDim[Byte](length) // TODO can be optimised to re-cycle this array instead + buf.get(arr) + val data = ByteString(arr) // bytes + + instruments.find(_.identifier == key) match { + case Some(instr) ⇒ instr.remoteMessageReceived(envelope.recipient.orNull, envelope.message, envelope.sender.orNull, data) + case _ ⇒ throw new Exception(s"No RemoteInstrument for id $key available!") + + } + } + } +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/ObjectPool.scala b/akka-remote/src/main/scala/akka/remote/artery/ObjectPool.scala new file mode 100644 index 0000000000..90f6f74d24 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/ObjectPool.scala @@ -0,0 +1,24 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import org.agrona.concurrent.ManyToManyConcurrentArrayQueue + +/** + * INTERNAL API + */ +private[remote] class ObjectPool[A <: AnyRef](capacity: Int, create: () ⇒ A, clear: A ⇒ Unit) { + private val pool = new ManyToManyConcurrentArrayQueue[A](capacity) + + def acquire(): A = { + val obj = pool.poll() + if (obj eq null) create() + else obj + } + + def release(obj: A): Boolean = { + clear(obj) + (!pool.offer(obj)) + } +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/OutboundEnvelope.scala b/akka-remote/src/main/scala/akka/remote/artery/OutboundEnvelope.scala new file mode 100644 index 0000000000..45e8fc8ea5 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/OutboundEnvelope.scala @@ -0,0 +1,84 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.actor.ActorRef +import akka.remote.RemoteActorRef +import akka.util.OptionVal + +/** + * INTERNAL API + */ +private[remote] object OutboundEnvelope { + def apply( + recipient: OptionVal[RemoteActorRef], + message: AnyRef, + sender: OptionVal[ActorRef]): OutboundEnvelope = { + val env = new ReusableOutboundEnvelope + env.init(recipient, message, sender) + } + +} + +/** + * INTERNAL API + */ +private[remote] trait OutboundEnvelope { + def recipient: OptionVal[RemoteActorRef] + def message: AnyRef + def sender: OptionVal[ActorRef] + + def withMessage(message: AnyRef): OutboundEnvelope + + def copy(): OutboundEnvelope +} + +/** + * INTERNAL API + */ +private[remote] object ReusableOutboundEnvelope { + def createObjectPool(capacity: Int) = new ObjectPool[ReusableOutboundEnvelope]( + capacity, + create = () ⇒ new ReusableOutboundEnvelope, clear = outEnvelope ⇒ outEnvelope.clear()) +} + +/** + * INTERNAL API + */ +private[remote] final class ReusableOutboundEnvelope extends OutboundEnvelope { + private var _recipient: OptionVal[RemoteActorRef] = OptionVal.None + private var _message: AnyRef = null + private var _sender: OptionVal[ActorRef] = OptionVal.None + + override def recipient: OptionVal[RemoteActorRef] = _recipient + override def message: AnyRef = _message + override def sender: OptionVal[ActorRef] = _sender + + override def withMessage(message: AnyRef): OutboundEnvelope = { + _message = message + this + } + + def copy(): OutboundEnvelope = + (new ReusableOutboundEnvelope).init(_recipient, _message, _sender) + + def clear(): Unit = { + _recipient = OptionVal.None + _message = null + _sender = OptionVal.None + } + + def init( + recipient: OptionVal[RemoteActorRef], + message: AnyRef, + sender: OptionVal[ActorRef]): OutboundEnvelope = { + _recipient = recipient + _message = message + _sender = sender + this + } + + override def toString: String = + s"OutboundEnvelope($recipient, $message, $sender)" +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/RemoteInstrument.scala b/akka-remote/src/main/scala/akka/remote/artery/RemoteInstrument.scala new file mode 100644 index 0000000000..9da058947e --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/RemoteInstrument.scala @@ -0,0 +1,192 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery + +import akka.actor.{ ActorRef, ExtendedActorSystem } +import akka.util.{ ByteString, OptionVal } + +/** + * INTERNAL API + * + * Part of the monitoring SPI which allows attaching metadata to outbound remote messages, + * and reading in metadata from incoming messages. + * + * Multiple instruments are automatically handled, however they MUST NOT overlap in their idenfitiers. + * + * Instances of `RemoteInstrument` are created from configuration. A new instance of RemoteInstrument + * will be created for each encoder and decoder. It's only called from the stage, so if it dosn't + * delegate to any shared instance it doesn't have to be thread-safe. + */ +abstract class RemoteInstrument { + /** + * Instrument identifier. + * + * MUST be >=0 and <32. + * + * Values between 0 and 7 are reserved for Akka internal use. + */ + def identifier: Byte + + /** + * Called right before putting the message onto the wire. + * Parameters MAY be `null` (except `message`)! + * + * @return `metadata` rendered to be serialized into the remove envelope, or `null` if no metadata should be attached + */ + def remoteMessageSent(recipient: ActorRef, message: Object, sender: ActorRef): ByteString + + /** + * Called once a message (containing a metadata field designated for this instrument) has been deserialized from the wire. + */ + def remoteMessageReceived(recipient: ActorRef, message: Object, sender: ActorRef, metadata: ByteString): Unit + +} + +object NoopRemoteInstrument extends RemoteInstrument { + override def identifier: Byte = + -1 + + override def remoteMessageSent(recipient: ActorRef, message: Object, sender: ActorRef): ByteString = + null + + override def remoteMessageReceived(recipient: ActorRef, message: Object, sender: ActorRef, metadata: ByteString): Unit = + () +} + +/** INTERNAL API */ +private[remote] object RemoteInstruments { + def create(system: ExtendedActorSystem): Vector[RemoteInstrument] = { + val c = system.settings.config + val path = "akka.remote.artery.advanced.instruments" + import scala.collection.JavaConverters._ + c.getStringList(path).asScala.map { fqcn ⇒ + system + .dynamicAccess.createInstanceFor[RemoteInstrument](fqcn, Nil) + .orElse(system.dynamicAccess.createInstanceFor[RemoteInstrument](fqcn, List(classOf[ExtendedActorSystem] → system))) + .get + }(collection.breakOut) + } +} + +/** + * INTERNAL API + * + * This datastructure is specialized for addressing directly into a known IDs slot. + * It is used when deserializing/serializing metadata and we know the ID we want to reach into. + * + * Mutable & NOT thread-safe. + * + * Fixed-size: 32-slots array-backed Map-like structure. + * Lazy: The backing array is allocated lazily only when needed, thus we pay no cost for the metadata array if + * the system is not using metadata. + * Life-cycle: Owned and linked to the lifecycle of an [[OutboundEnvelope]]. + * Re-cycled: Aimed to be re-cycled to produce the least possible GC-churn, by calling `clear()` when done with it. + * + * Reserved keys: The keys 0–7 are reserved for Akka internal purposes and future extensions. + */ +private[remote] final class MetadataMap[T >: Null] { + val capacity = 32 + + protected var backing: Array[T] = null // TODO re-think if a plain LinkedList wouldn't be fine here? + + private var _usedSlots = 0 + + def usedSlots = _usedSlots + + def apply(i: Int): OptionVal[T] = + if (backing == null) OptionVal.None + else OptionVal[T](backing(i)) + + def isEmpty = usedSlots == 0 + + def nonEmpty = !isEmpty + + def hasValueFor(i: Int) = nonEmpty && backing(i) != null + + // FIXME too specialized... + def foldLeftValues[A](zero: A)(f: (A, T) ⇒ A): A = { + var acc: A = zero + var hit = 0 + var i = 0 + while (i < capacity && hit < _usedSlots) { + val it = backing(i) + if (it != null) { + acc = f(acc, it) + hit += 1 + } + i += 1 + } + acc + } + + /** Heavy operation, only used for error logging */ + def keysWithValues: List[Int] = backing.zipWithIndex.filter({ case (t, id) ⇒ t != null }).map(_._2).toList + + def foreach(f: (Byte, T) ⇒ Unit) = { + var i = 0 + var hit = 0 + while (i < capacity && hit < _usedSlots) { + val t = backing(i) + if (t != null) { + f(i.toByte, t) + hit += 1 + } + i += 1 + } + } + + private def allocateIfNeeded(): Unit = + if (backing == null) backing = Array.ofDim[Object](capacity).asInstanceOf[Array[T]] + + /** + * Set a value at given index. + * Setting a null value removes the entry (the slot is marked as not used). + */ + def set(i: Int, t: T): Unit = + if (t == null) { + if (backing == null) () + else { + if (backing(i) != null) _usedSlots -= 1 // we're clearing a spot + backing(i) = null.asInstanceOf[T] + } + } else { + allocateIfNeeded() + + if (backing(i) == null) { + // was empty previously + _usedSlots += 1 + } else { + // replacing previous value, usedSlots remains unchanged + } + + backing(i) = t + } + + /** + * If the backing array was already allocated clears it, if it wasn't does nothing (no-op). + * This is so in order to pay no-cost when not using metadata - clearing then is instant. + */ + def clear() = + if (isEmpty) () + else { + var i = 0 + while (i < capacity) { + backing(i) = null.asInstanceOf[T] + i += 1 + } + _usedSlots = 0 + } + + override def toString() = + if (backing == null) s"MetadataMap()" + else s"MetadataMap(${backing.toList.mkString("[", ",", "]")})" +} + +/** + * INTERNAL API + */ +private[remote] object MetadataMap { + def apply[T >: Null]() = new MetadataMap[T] +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/RestartCounter.scala b/akka-remote/src/main/scala/akka/remote/artery/RestartCounter.scala new file mode 100644 index 0000000000..e11637e4c8 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/RestartCounter.scala @@ -0,0 +1,52 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.duration.Deadline +import java.util.concurrent.atomic.AtomicReference +import scala.concurrent.duration.FiniteDuration +import scala.annotation.tailrec + +/** + * INTERNAL API + */ +private[remote] object RestartCounter { + final case class State(count: Int, deadline: Deadline) +} + +/** + * INTERNAL API: Thread safe "restarts with duration" counter + */ +private[remote] class RestartCounter(maxRestarts: Int, restartTimeout: FiniteDuration) { + import RestartCounter._ + + private val state = new AtomicReference[State](State(0, Deadline.now + restartTimeout)) + + /** + * Current number of restarts. + */ + def count(): Int = state.get.count + + /** + * Increment the restart counter, or reset the counter to 1 if the + * `restartTimeout` has elapsed. The latter also resets the timeout. + * @return `true` if number of restarts, including this one, is less + * than or equal to `maxRestarts` + */ + @tailrec final def restart(): Boolean = { + val s = state.get + + val newState = + if (s.deadline.hasTimeLeft()) + s.copy(count = s.count + 1) + else + State(1, Deadline.now + restartTimeout) + + if (state.compareAndSet(s, newState)) + newState.count <= maxRestarts + else + restart() // recur + } + +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala b/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala new file mode 100644 index 0000000000..075a6064ab --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala @@ -0,0 +1,138 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.Queue +import akka.stream.stage.GraphStage +import akka.stream.stage.OutHandler +import akka.stream.Attributes +import akka.stream.Outlet +import akka.stream.SourceShape +import akka.stream.stage.GraphStageLogic +import org.agrona.concurrent.ManyToOneConcurrentArrayQueue +import akka.stream.stage.GraphStageWithMaterializedValue +import org.agrona.concurrent.ManyToOneConcurrentLinkedQueueTail +import org.agrona.concurrent.ManyToOneConcurrentLinkedQueue +import java.util.concurrent.atomic.AtomicInteger +import scala.annotation.tailrec +import scala.concurrent.Promise +import scala.util.Try +import scala.util.Success +import scala.util.Failure + +/** + * INTERNAL API + */ +private[remote] object SendQueue { + trait ProducerApi[T] { + def offer(message: T): Boolean + + def isEnabled: Boolean + } + + trait QueueValue[T] extends ProducerApi[T] { + def inject(queue: Queue[T]): Unit + } + + private trait WakeupSignal { + def wakeup(): Unit + } +} + +/** + * INTERNAL API + */ +private[remote] final class SendQueue[T] extends GraphStageWithMaterializedValue[SourceShape[T], SendQueue.QueueValue[T]] { + import SendQueue._ + + val out: Outlet[T] = Outlet("SendQueue.out") + override val shape: SourceShape[T] = SourceShape(out) + + override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, QueueValue[T]) = { + @volatile var needWakeup = false + val queuePromise = Promise[Queue[T]]() + + val logic = new GraphStageLogic(shape) with OutHandler with WakeupSignal { + + // using a local field for the consumer side of queue to avoid volatile access + private var consumerQueue: Queue[T] = null + + private val wakeupCallback = getAsyncCallback[Unit] { _ ⇒ + if (isAvailable(out)) + tryPush() + } + + override def preStart(): Unit = { + implicit val ec = materializer.executionContext + queuePromise.future.onComplete(getAsyncCallback[Try[Queue[T]]] { + case Success(q) ⇒ + consumerQueue = q + needWakeup = true + if (isAvailable(out)) + tryPush() + case Failure(e) ⇒ + failStage(e) + }.invoke) + } + + override def onPull(): Unit = { + if (consumerQueue ne null) + tryPush() + } + + @tailrec private def tryPush(firstAttempt: Boolean = true): Unit = { + consumerQueue.poll() match { + case null ⇒ + needWakeup = true + // additional poll() to grab any elements that might missed the needWakeup + // and have been enqueued just after it + if (firstAttempt) + tryPush(firstAttempt = false) + case elem ⇒ + needWakeup = false // there will be another onPull + push(out, elem) + } + } + + // external call + override def wakeup(): Unit = { + wakeupCallback.invoke(()) + } + + override def postStop(): Unit = { + // TODO quarantine will currently always be done when control stream is terminated, see issue #21359 + if (consumerQueue ne null) + consumerQueue.clear() + super.postStop() + } + + setHandler(out, this) + } + + val queueValue = new QueueValue[T] { + @volatile private var producerQueue: Queue[T] = null + + override def inject(q: Queue[T]): Unit = { + producerQueue = q + queuePromise.success(q) + } + + override def offer(message: T): Boolean = { + val q = producerQueue + if (q eq null) throw new IllegalStateException("offer not allowed before injecting the queue") + val result = q.offer(message) + if (result && needWakeup) { + needWakeup = false + logic.wakeup() + } + result + } + + override def isEnabled: Boolean = true + } + + (logic, queueValue) + + } +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/StageLogging.scala b/akka-remote/src/main/scala/akka/remote/artery/StageLogging.scala new file mode 100644 index 0000000000..8f9e768299 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/StageLogging.scala @@ -0,0 +1,34 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.stream.stage.GraphStageLogic +import akka.event.LoggingAdapter +import akka.stream.ActorMaterializer +import akka.event.NoLogging + +// TODO this can be removed when https://github.com/akka/akka/issues/18793 has been implemented +/** + * INTERNAL API + */ +private[akka] trait StageLogging { self: GraphStageLogic ⇒ + + private var _log: LoggingAdapter = _ + + protected def logSource: Class[_] = this.getClass + + def log: LoggingAdapter = { + // only used in StageLogic, i.e. thread safe + if (_log eq null) { + materializer match { + case a: ActorMaterializer ⇒ + _log = akka.event.Logging(a.system, logSource) + case _ ⇒ + _log = NoLogging + } + } + _log + } + +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala new file mode 100644 index 0000000000..931a2b3b5b --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -0,0 +1,311 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.util.PrettyDuration.PrettyPrintableDuration +import java.util.ArrayDeque +import scala.annotation.tailrec +import scala.concurrent.duration._ +import scala.util.Failure +import scala.util.Success +import scala.util.Try +import akka.Done +import akka.remote.UniqueAddress +import akka.remote.artery.InboundControlJunction.ControlMessageObserver +import akka.stream.Attributes +import akka.stream.FlowShape +import akka.stream.Inlet +import akka.stream.Outlet +import akka.stream.stage.GraphStage +import akka.stream.stage.GraphStageLogic +import akka.stream.stage.InHandler +import akka.stream.stage.OutHandler +import akka.stream.stage.TimerGraphStageLogic +import akka.remote.artery.OutboundHandshake.HandshakeReq +import akka.actor.ActorRef +import akka.dispatch.sysmsg.SystemMessage +import scala.util.control.NoStackTrace + +/** + * INTERNAL API + */ +private[remote] object SystemMessageDelivery { + // FIXME serialization of these messages + final case class SystemMessageEnvelope(message: AnyRef, seqNo: Long, ackReplyTo: UniqueAddress) extends ArteryMessage + final case class Ack(seqNo: Long, from: UniqueAddress) extends Reply + final case class Nack(seqNo: Long, from: UniqueAddress) extends Reply + + final case object ClearSystemMessageDelivery + + final class GaveUpSystemMessageException(msg: String) extends RuntimeException(msg) with NoStackTrace + + private case object ResendTick + + // If other message types than SystemMesage need acked delivery they can extend this trait. + // Used in tests since real SystemMessage are somewhat cumbersome to create. + trait AckedDeliveryMessage + +} + +/** + * INTERNAL API + */ +private[remote] class SystemMessageDelivery( + outboundContext: OutboundContext, + deadLetters: ActorRef, + resendInterval: FiniteDuration, + maxBufferSize: Int) + extends GraphStage[FlowShape[OutboundEnvelope, OutboundEnvelope]] { + + import SystemMessageDelivery._ + + val in: Inlet[OutboundEnvelope] = Inlet("SystemMessageDelivery.in") + val out: Outlet[OutboundEnvelope] = Outlet("SystemMessageDelivery.out") + override val shape: FlowShape[OutboundEnvelope, OutboundEnvelope] = FlowShape(in, out) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new TimerGraphStageLogic(shape) with InHandler with OutHandler with ControlMessageObserver { + + private var replyObserverAttached = false + private var seqNo = 0L // sequence number for the first message will be 1 + private val unacknowledged = new ArrayDeque[OutboundEnvelope] + private var resending = new ArrayDeque[OutboundEnvelope] + private var resendingFromSeqNo = -1L + private var stopping = false + + private val giveUpAfterNanos = outboundContext.settings.Advanced.GiveUpSystemMessageAfter.toNanos + private var ackTimestamp = System.nanoTime() + + private def localAddress = outboundContext.localAddress + private def remoteAddress = outboundContext.remoteAddress + + override def preStart(): Unit = { + implicit val ec = materializer.executionContext + outboundContext.controlSubject.attach(this).foreach { + getAsyncCallback[Done] { _ ⇒ + replyObserverAttached = true + if (isAvailable(out)) + pull(in) // onPull from downstream already called + }.invoke + } + + outboundContext.controlSubject.stopped.onComplete { + getAsyncCallback[Try[Done]] { + case Success(_) ⇒ completeStage() + case Failure(cause) ⇒ failStage(cause) + }.invoke + } + } + + override def postStop(): Unit = { + // TODO quarantine will currently always be done when control stream is terminated, see issue #21359 + sendUnacknowledgedToDeadLetters() + unacknowledged.clear() + outboundContext.controlSubject.detach(this) + } + + override def onUpstreamFinish(): Unit = { + if (unacknowledged.isEmpty) + super.onUpstreamFinish() + else + stopping = true + } + + override protected def onTimer(timerKey: Any): Unit = + timerKey match { + case ResendTick ⇒ + checkGiveUp() + if (resending.isEmpty && !unacknowledged.isEmpty) { + resending = unacknowledged.clone() + tryResend() + } + if (!unacknowledged.isEmpty) + scheduleOnce(ResendTick, resendInterval) + } + + // ControlMessageObserver, external call + override def notify(inboundEnvelope: InboundEnvelope): Unit = { + inboundEnvelope.message match { + case ack: Ack ⇒ if (ack.from.address == remoteAddress) ackCallback.invoke(ack) + case nack: Nack ⇒ if (nack.from.address == remoteAddress) nackCallback.invoke(nack) + case _ ⇒ // not interested + } + } + + private val ackCallback = getAsyncCallback[Ack] { reply ⇒ + ack(reply.seqNo) + } + + private val nackCallback = getAsyncCallback[Nack] { reply ⇒ + if (reply.seqNo <= seqNo) { + ack(reply.seqNo) + if (reply.seqNo > resendingFromSeqNo) + resending = unacknowledged.clone() + tryResend() + } + } + + private def ack(n: Long): Unit = { + ackTimestamp = System.nanoTime() + if (n <= seqNo) + clearUnacknowledged(n) + } + + @tailrec private def clearUnacknowledged(ackedSeqNo: Long): Unit = { + if (!unacknowledged.isEmpty && + unacknowledged.peek().message.asInstanceOf[SystemMessageEnvelope].seqNo <= ackedSeqNo) { + unacknowledged.removeFirst() + if (unacknowledged.isEmpty) + cancelTimer(resendInterval) + + if (stopping && unacknowledged.isEmpty) + completeStage() + else + clearUnacknowledged(ackedSeqNo) + } + } + + private def tryResend(): Unit = { + if (isAvailable(out) && !resending.isEmpty) + pushCopy(resending.poll()) + } + + // important to not send the buffered instance, since it's mutable + private def pushCopy(outboundEnvelope: OutboundEnvelope): Unit = { + push(out, outboundEnvelope.copy()) + } + + // InHandler + override def onPush(): Unit = { + val outboundEnvelope = grab(in) + outboundEnvelope.message match { + case msg @ (_: SystemMessage | _: AckedDeliveryMessage) ⇒ + if (unacknowledged.size < maxBufferSize) { + seqNo += 1 + if (unacknowledged.isEmpty) + ackTimestamp = System.nanoTime() + else + checkGiveUp() + val sendEnvelope = outboundEnvelope.withMessage(SystemMessageEnvelope(msg, seqNo, localAddress)) + unacknowledged.offer(sendEnvelope) + scheduleOnce(ResendTick, resendInterval) + if (resending.isEmpty && isAvailable(out)) + pushCopy(sendEnvelope) + else { + resending.offer(sendEnvelope) + tryResend() + } + } else { + // buffer overflow + outboundContext.quarantine(reason = s"System message delivery buffer overflow, size [$maxBufferSize]") + deadLetters ! outboundEnvelope + pull(in) + } + case _: HandshakeReq ⇒ + // pass on HandshakeReq + if (isAvailable(out)) + pushCopy(outboundEnvelope) + case ClearSystemMessageDelivery ⇒ + clear() + pull(in) + case _ ⇒ + // e.g. ActorSystemTerminating or ActorSelectionMessage with PriorityMessage, no need for acked delivery + if (resending.isEmpty && isAvailable(out)) + push(out, outboundEnvelope) + else { + resending.offer(outboundEnvelope) + tryResend() + } + } + } + + private def checkGiveUp(): Unit = { + if (!unacknowledged.isEmpty && (System.nanoTime() - ackTimestamp > giveUpAfterNanos)) + throw new GaveUpSystemMessageException( + s"Gave up sending system message to [${outboundContext.remoteAddress}] after " + + s"${outboundContext.settings.Advanced.GiveUpSystemMessageAfter.pretty}.") + } + + private def clear(): Unit = { + sendUnacknowledgedToDeadLetters() + seqNo = 0L // sequence number for the first message will be 1 + unacknowledged.clear() + resending.clear() + resendingFromSeqNo = -1L + cancelTimer(resendInterval) + } + + private def sendUnacknowledgedToDeadLetters(): Unit = { + val iter = unacknowledged.iterator + while (iter.hasNext()) { + deadLetters ! iter.next() + } + } + + // OutHandler + override def onPull(): Unit = { + if (replyObserverAttached) { // otherwise it will be pulled after attached + if (resending.isEmpty && !hasBeenPulled(in) && !stopping) + pull(in) + else + tryResend() + } + } + + setHandlers(in, out, this) + } +} + +/** + * INTERNAL API + */ +private[remote] class SystemMessageAcker(inboundContext: InboundContext) extends GraphStage[FlowShape[InboundEnvelope, InboundEnvelope]] { + import SystemMessageDelivery._ + + val in: Inlet[InboundEnvelope] = Inlet("SystemMessageAcker.in") + val out: Outlet[InboundEnvelope] = Outlet("SystemMessageAcker.out") + override val shape: FlowShape[InboundEnvelope, InboundEnvelope] = FlowShape(in, out) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new GraphStageLogic(shape) with InHandler with OutHandler { + + // TODO we might need have to prune old unused entries + var sequenceNumbers = Map.empty[UniqueAddress, Long] + + def localAddress = inboundContext.localAddress + + // InHandler + override def onPush(): Unit = { + val env = grab(in) + env.message match { + case sysEnv @ SystemMessageEnvelope(_, n, ackReplyTo) ⇒ + val expectedSeqNo = sequenceNumbers.get(ackReplyTo) match { + case None ⇒ 1L + case Some(seqNo) ⇒ seqNo + } + if (n == expectedSeqNo) { + inboundContext.sendControl(ackReplyTo.address, Ack(n, localAddress)) + sequenceNumbers = sequenceNumbers.updated(ackReplyTo, n + 1) + val unwrapped = env.withMessage(sysEnv.message) + push(out, unwrapped) + } else if (n < expectedSeqNo) { + inboundContext.sendControl(ackReplyTo.address, Ack(expectedSeqNo - 1, localAddress)) + pull(in) + } else { + inboundContext.sendControl(ackReplyTo.address, Nack(expectedSeqNo - 1, localAddress)) + pull(in) + } + case _ ⇒ + // messages that don't need acking + push(out, env) + } + } + + // OutHandler + override def onPull(): Unit = pull(in) + + setHandlers(in, out, this) + } +} + diff --git a/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala b/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala new file mode 100644 index 0000000000..1f6ca158d4 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala @@ -0,0 +1,197 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.concurrent.TimeUnit.{ MICROSECONDS, MILLISECONDS } + +import akka.Done +import akka.actor.ExtendedActorSystem +import akka.dispatch.{ AbstractNodeQueue, MonitorableThreadFactory } +import akka.event.Logging +import org.agrona.concurrent.{ BackoffIdleStrategy, BusySpinIdleStrategy, IdleStrategy, SleepingIdleStrategy } + +import scala.annotation.tailrec +import scala.concurrent.{ Future, Promise } +import scala.reflect.ClassTag +import scala.util.control.NonFatal + +/** + * INTERNAL API + */ +private[akka] object TaskRunner { + + type Task = () ⇒ Boolean + sealed trait Command + case object Shutdown extends Command + final case class Add(task: Task) extends Command + final case class Remove(task: Task) extends Command + + final class CommandQueue extends AbstractNodeQueue[Command] + + /** + * A specialized collection with allocation free add, remove and iterate of + * elements. The order of the elements is not important. + */ + private final class ArrayBag[T <: AnyRef: ClassTag] { + private var elements = Array.ofDim[T](16) + + def add(e: T): Unit = { + val size = elements.length + @tailrec def tryAdd(i: Int): Unit = { + if (i == size) { + doubleCapacity() + elements(i) = e + } else if (elements(i) eq null) + elements(i) = e + else + tryAdd(i + 1) //recursive + } + tryAdd(0) + } + + def remove(e: T): Unit = { + val size = elements.length + @tailrec def tryRemove(i: Int): Unit = { + if (i == size) + () // not found + else if (elements(i) == e) + elements(i) = null.asInstanceOf[T] + else + tryRemove(i + 1) //recursive + } + tryRemove(0) + } + + def removeAll(): Unit = { + var i = 0 + while (i < elements.length) { + elements(i) = null.asInstanceOf[T] + i += 1 + } + } + + /** + * All elements as an array for efficient iteration. + * The elements can be `null`. + */ + def all: Array[T] = elements + + private def doubleCapacity(): Unit = { + val newCapacity = elements.length << 1 + if (newCapacity < 0) + throw new IllegalStateException("Sorry, too big") + val a = Array.ofDim[T](newCapacity) + System.arraycopy(elements, 0, a, 0, elements.length) + elements = a + } + + override def toString(): String = + elements.filterNot(_ eq null).mkString("[", ",", "]") + } + + def createIdleStrategy(idleCpuLevel: Int): IdleStrategy = { + if (idleCpuLevel == 1) + new SleepingIdleStrategy(MILLISECONDS.toNanos(1)) + else if (idleCpuLevel == 10) + new BusySpinIdleStrategy + else { + // spin between 100 to 10000 depending on idleCpuLevel + val spinning = 1100 * idleCpuLevel - 1000 + val yielding = 5 * idleCpuLevel + val minParkNanos = 1 + // park between 250 and 10 micros depending on idleCpuLevel + val maxParkNanos = MICROSECONDS.toNanos(280 - 30 * idleCpuLevel) + new BackoffIdleStrategy(spinning, yielding, 1, maxParkNanos) + } + } +} + +/** + * INTERNAL API + */ +private[akka] class TaskRunner(system: ExtendedActorSystem, val idleCpuLevel: Int) extends Runnable { + import TaskRunner._ + + private val log = Logging(system, getClass) + private[this] var running = false + private[this] val cmdQueue = new CommandQueue + private[this] val tasks = new ArrayBag[Task] + private[this] val shutdown = Promise[Done]() + + private val idleStrategy = createIdleStrategy(idleCpuLevel) + private var reset = false + + def start(): Unit = { + val tf = system.threadFactory match { + case m: MonitorableThreadFactory ⇒ + m.withName(m.name + "-taskrunner") + case other ⇒ other + } + val thread = tf.newThread(this) + thread.start() + } + + def stop(): Future[Done] = { + command(Shutdown) + shutdown.future + } + + def command(cmd: Command): Unit = { + cmdQueue.add(cmd) + } + + override def run(): Unit = { + try { + running = true + while (running) { + processCommand(cmdQueue.poll()) + if (running) { + executeTasks() + if (reset) { + reset = false + idleStrategy.reset() + } + idleStrategy.idle() + } + } + } catch { + case NonFatal(e) ⇒ + log.error(e, e.getMessage) + } + } + + private def executeTasks(): Unit = { + val elements = tasks.all + var i = 0 + val size = elements.length + while (i < size) { + val task = elements(i) + if (task ne null) try { + if (task()) { + tasks.remove(task) + reset = true + } + } catch { + case NonFatal(e) ⇒ + log.error(e, "Task failed") + tasks.remove(task) + } + i += 1 + } + } + + private def processCommand(cmd: Command): Unit = { + cmd match { + case null ⇒ // no command + case Add(task) ⇒ tasks.add(task) + case Remove(task) ⇒ tasks.remove(task) + case Shutdown ⇒ + running = false + tasks.removeAll() // gc friendly + while (cmdQueue.poll() != null) () // gc friendly + shutdown.trySuccess(Done) + } + } + +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala b/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala new file mode 100644 index 0000000000..2b7ca5e27c --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala @@ -0,0 +1,170 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.concurrent.atomic.AtomicReference + +import scala.annotation.tailrec +import scala.concurrent.duration._ + +import akka.actor.Address +import akka.remote.transport.ThrottlerTransportAdapter.Direction +import akka.stream.Attributes +import akka.stream.FlowShape +import akka.stream.Inlet +import akka.stream.Outlet +import akka.stream.stage.GraphStage +import akka.stream.stage.GraphStageLogic +import akka.stream.stage.InHandler +import akka.stream.stage.OutHandler +import akka.stream.stage.TimerGraphStageLogic +import akka.util.OptionVal +import akka.event.Logging + +/** + * INTERNAL API: Thread safe mutable state that is shared among + * the test stages. + */ +private[remote] class SharedTestState { + private val state = new AtomicReference[TestState](TestState(Map.empty)) + + def isBlackhole(from: Address, to: Address): Boolean = + state.get.blackholes.get(from) match { + case Some(destinations) ⇒ destinations(to) + case None ⇒ false + } + + def blackhole(a: Address, b: Address, direction: Direction): Unit = + direction match { + case Direction.Send ⇒ addBlackhole(a, b) + case Direction.Receive ⇒ addBlackhole(b, a) + case Direction.Both ⇒ + addBlackhole(a, b) + addBlackhole(b, a) + } + + @tailrec private def addBlackhole(from: Address, to: Address): Unit = { + val current = state.get + val newState = current.blackholes.get(from) match { + case Some(destinations) ⇒ current.copy(blackholes = current.blackholes.updated(from, destinations + to)) + case None ⇒ current.copy(blackholes = current.blackholes.updated(from, Set(to))) + } + if (!state.compareAndSet(current, newState)) + addBlackhole(from, to) + } + + def passThrough(a: Address, b: Address, direction: Direction): Unit = + direction match { + case Direction.Send ⇒ removeBlackhole(a, b) + case Direction.Receive ⇒ removeBlackhole(b, a) + case Direction.Both ⇒ + removeBlackhole(a, b) + removeBlackhole(b, a) + } + + @tailrec private def removeBlackhole(from: Address, to: Address): Unit = { + val current = state.get + val newState = current.blackholes.get(from) match { + case Some(destinations) ⇒ current.copy(blackholes = current.blackholes.updated(from, destinations - to)) + case None ⇒ current + } + if (!state.compareAndSet(current, newState)) + removeBlackhole(from, to) + } + +} + +/** + * INTERNAL API + */ +private[remote] final case class TestState(blackholes: Map[Address, Set[Address]]) + +/** + * INTERNAL API + */ +private[remote] class OutboundTestStage(outboundContext: OutboundContext, state: SharedTestState, enabled: Boolean) + extends GraphStage[FlowShape[OutboundEnvelope, OutboundEnvelope]] { + val in: Inlet[OutboundEnvelope] = Inlet("OutboundTestStage.in") + val out: Outlet[OutboundEnvelope] = Outlet("OutboundTestStage.out") + override val shape: FlowShape[OutboundEnvelope, OutboundEnvelope] = FlowShape(in, out) + + override def createLogic(inheritedAttributes: Attributes) = { + if (enabled) { + new TimerGraphStageLogic(shape) with InHandler with OutHandler with StageLogging { + + // InHandler + override def onPush(): Unit = { + val env = grab(in) + if (state.isBlackhole(outboundContext.localAddress.address, outboundContext.remoteAddress)) { + log.debug( + "dropping outbound message [{}] to [{}] because of blackhole", + Logging.messageClassName(env.message), outboundContext.remoteAddress) + pull(in) // drop message + } else + push(out, env) + } + + // OutHandler + override def onPull(): Unit = pull(in) + + setHandlers(in, out, this) + } + } else { + new GraphStageLogic(shape) with InHandler with OutHandler { + override def onPush(): Unit = push(out, grab(in)) + override def onPull(): Unit = pull(in) + setHandlers(in, out, this) + } + } + } + +} + +/** + * INTERNAL API + */ +private[remote] class InboundTestStage(inboundContext: InboundContext, state: SharedTestState, enabled: Boolean) + extends GraphStage[FlowShape[InboundEnvelope, InboundEnvelope]] { + val in: Inlet[InboundEnvelope] = Inlet("InboundTestStage.in") + val out: Outlet[InboundEnvelope] = Outlet("InboundTestStage.out") + override val shape: FlowShape[InboundEnvelope, InboundEnvelope] = FlowShape(in, out) + + override def createLogic(inheritedAttributes: Attributes) = { + if (enabled) { + new TimerGraphStageLogic(shape) with InHandler with OutHandler with StageLogging { + + // InHandler + override def onPush(): Unit = { + val env = grab(in) + env.association match { + case OptionVal.None ⇒ + // unknown, handshake not completed + push(out, env) + case OptionVal.Some(association) ⇒ + if (state.isBlackhole(inboundContext.localAddress.address, association.remoteAddress)) { + log.debug( + "dropping inbound message [{}] from [{}] with UID [{}] because of blackhole", + Logging.messageClassName(env.message), association.remoteAddress, env.originUid) + pull(in) // drop message + } else + push(out, env) + } + } + + // OutHandler + override def onPull(): Unit = pull(in) + + setHandlers(in, out, this) + } + } else { + new GraphStageLogic(shape) with InHandler with OutHandler { + override def onPush(): Unit = push(out, grab(in)) + override def onPull(): Unit = pull(in) + setHandlers(in, out, this) + } + } + } + +} + diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionProtocol.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionProtocol.scala new file mode 100644 index 0000000000..2945bf70f8 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionProtocol.scala @@ -0,0 +1,77 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import akka.actor.ActorRef +import akka.remote.UniqueAddress +import akka.remote.artery.ControlMessage + +// FIXME serialization +/** INTERNAL API */ +/** + * INTERNAL API + */ +private[remote] object CompressionProtocol { + + /** INTERNAL API */ + sealed trait CompressionMessage + + /** INTERNAL API */ + sealed trait CompressionAdvertisement[T] extends ControlMessage with CompressionMessage { + def from: UniqueAddress + def table: CompressionTable[T] + } + + /** + * INTERNAL API + * Sent by the "receiving" node after allocating a compression id to a given [[akka.actor.ActorRef]] + */ + private[remote] final case class ActorRefCompressionAdvertisement(from: UniqueAddress, table: CompressionTable[ActorRef]) + extends CompressionAdvertisement[ActorRef] + + /** + * INTERNAL API + * Sent by the "sending" node after receiving [[ActorRefCompressionAdvertisement]] + * The advertisement is also confirmed by the first message using that table version, + * but we need separate ack in case the sender is not using any of the refs in the advertised + * table. + */ + private[remote] final case class ActorRefCompressionAdvertisementAck(from: UniqueAddress, tableVersion: Byte) + extends ControlMessage with CompressionMessage + + /** + * INTERNAL API + * Sent by the "receiving" node after allocating a compression id to a given class manifest + */ + private[remote] final case class ClassManifestCompressionAdvertisement(from: UniqueAddress, table: CompressionTable[String]) + extends CompressionAdvertisement[String] + + /** + * INTERNAL API + * Sent by the "sending" node after receiving [[ClassManifestCompressionAdvertisement]] + * The advertisement is also confirmed by the first message using that table version, + * but we need separate ack in case the sender is not using any of the refs in the advertised + * table. + */ + private[remote] final case class ClassManifestCompressionAdvertisementAck(from: UniqueAddress, tableVersion: Byte) + extends ControlMessage with CompressionMessage + + /** INTERNAL API */ + private[remote] object Events { + /** INTERNAL API */ + private[remote] sealed trait Event + + /** INTERNAL API */ + final case class HeavyHitterDetected(key: Any, id: Int, count: Long) extends Event + + /** INTERNAL API */ + final case class ReceivedActorRefCompressionTable(from: UniqueAddress, table: CompressionTable[ActorRef]) extends Event + + /** INTERNAL API */ + final case class ReceivedClassManifestCompressionTable(from: UniqueAddress, table: CompressionTable[String]) extends Event + + } + +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala new file mode 100644 index 0000000000..88444bb7f5 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala @@ -0,0 +1,66 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import java.util +import java.util.Comparator + +/** + * INTERNAL API: Versioned compression table to be advertised between systems + * + * @param version Either -1 for disabled or a version between 0 and 127 + */ +private[remote] final case class CompressionTable[T](originUid: Long, version: Byte, dictionary: Map[T, Int]) { + import CompressionTable.NotCompressedId + + def compress(value: T): Int = + dictionary.get(value) match { + case Some(id) ⇒ id + case None ⇒ NotCompressedId + } + + def invert: DecompressionTable[T] = + if (dictionary.isEmpty) DecompressionTable.empty[T].copy(originUid = originUid, version = version) + else { + // TODO: these are some expensive sanity checks, about the numbers being consecutive, without gaps + // TODO: we can remove them, make them re-map (not needed I believe though) + val expectedGaplessSum = Integer.valueOf((dictionary.size * (dictionary.size + 1)) / 2) /* Dirichlet */ + require(dictionary.values.min == 0, "Compression table should start allocating from 0, yet lowest allocated id was " + dictionary.values.min) + require(dictionary.values.sum + dictionary.size == expectedGaplessSum, "Given compression map does not seem to be gap-less and starting from zero, " + + "which makes compressing it into an Array difficult, bailing out! Map was: " + dictionary) + + val tups = Array.ofDim[(Object, Int)](dictionary.size).asInstanceOf[Array[(T, Int)]] + val ts = Array.ofDim[Object](dictionary.size).asInstanceOf[Array[T]] + + var i = 0 + val mit = dictionary.iterator + while (i < tups.length) { + tups(i) = mit.next() + i += 1 + } + util.Arrays.sort(tups, CompressionTable.compareBy2ndValue[T]) + + i = 0 + while (i < tups.length) { + ts(i) = tups(i)._1 + i += 1 + } + + DecompressionTable[T](originUid, version, ts) + } +} +/** INTERNAL API */ +private[remote] object CompressionTable { + final val NotCompressedId = -1 + + final val CompareBy2ndValue: Comparator[(Object, Int)] = new Comparator[(Object, Int)] { + override def compare(o1: (Object, Int), o2: (Object, Int)): Int = + o1._2 compare o2._2 + } + def compareBy2ndValue[T]: Comparator[Tuple2[T, Int]] = CompareBy2ndValue.asInstanceOf[Comparator[(T, Int)]] + + private[this] val _empty = new CompressionTable[Any](0, 0, Map.empty) + def empty[T] = _empty.asInstanceOf[CompressionTable[T]] +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala new file mode 100644 index 0000000000..efb02181f1 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala @@ -0,0 +1,40 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +/** + * INTERNAL API + * + * @param version Either -1 for disabled or a version between 0 and 127 + */ +private[remote] final case class DecompressionTable[T](originUid: Long, version: Byte, table: Array[T]) { + + private[this] val length = table.length + + def get(idx: Int): T = { + if (idx >= length) + throw new IllegalArgumentException(s"Attempted decompression of unknown id: [$idx]! " + + s"Only $length ids allocated in table version [$version] for origin [$originUid].") + table(idx) + } + + def invert: CompressionTable[T] = + CompressionTable(originUid, version, Map(table.zipWithIndex: _*)) + + /** Writes complete table as String (heavy operation) */ + override def toString = + s"DecompressionTable($originUid, $version, " + + s"Map(${table.zipWithIndex.map({ case (t, i) ⇒ s"$i -> $t" }).mkString(",")}))" +} + +/** INTERNAL API */ +private[remote] object DecompressionTable { + + val DisabledVersion: Byte = -1 + + private[this] val _empty = DecompressionTable(0, 0, Array.empty) + def empty[T] = _empty.asInstanceOf[DecompressionTable[T]] + def disabled[T] = empty[T].copy(version = DisabledVersion) +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala new file mode 100644 index 0000000000..8e326371de --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala @@ -0,0 +1,500 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import java.util.concurrent.atomic.AtomicReference +import java.util.function.LongFunction + +import scala.concurrent.duration.{ Duration, FiniteDuration } +import akka.actor.{ ActorRef, ActorSystem, Address } +import akka.event.Logging +import akka.remote.artery.{ ArterySettings, InboundContext, OutboundContext } +import akka.util.OptionVal +import org.agrona.collections.Long2ObjectHashMap + +import scala.annotation.tailrec +import akka.actor.Cancellable +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.atomic.AtomicInteger + +/** + * INTERNAL API + * Decompress and cause compression advertisements. + * + * One per inbound message stream thus must demux by originUid to use the right tables. + */ +private[remote] trait InboundCompressions { + def hitActorRef(originUid: Long, remote: Address, ref: ActorRef, n: Int): Unit + def decompressActorRef(originUid: Long, tableVersion: Byte, idx: Int): OptionVal[ActorRef] + def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Byte): Unit + + def hitClassManifest(originUid: Long, remote: Address, manifest: String, n: Int): Unit + def decompressClassManifest(originUid: Long, tableVersion: Byte, idx: Int): OptionVal[String] + def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Byte): Unit + + /** + * Cancel advertisement scheduling + */ + def close(): Unit + + /** + * Remove compression and cancel advertisement scheduling for a specific origin + */ + def close(originUid: Long): Unit + +} + +/** + * INTERNAL API + * + * One per incoming Aeron stream, actual compression tables are kept per-originUid and created on demand. + */ +private[remote] final class InboundCompressionsImpl( + system: ActorSystem, + inboundContext: InboundContext, + settings: ArterySettings.Compression) extends InboundCompressions { + + private val stopped = new AtomicBoolean + + // None is used as tombstone value after closed + // TOOD would be nice if we can cleanup the tombstones + private[this] val _actorRefsIns = new Long2ObjectHashMap[Option[InboundActorRefCompression]]() + private val createInboundActorRefsForOrigin = new LongFunction[Option[InboundActorRefCompression]] { + override def apply(originUid: Long): Option[InboundActorRefCompression] = { + val actorRefHitters = new TopHeavyHitters[ActorRef](settings.ActorRefs.Max) + Some(new InboundActorRefCompression(system, settings, originUid, inboundContext, actorRefHitters, stopped)) + } + } + private def actorRefsIn(originUid: Long): Option[InboundActorRefCompression] = + _actorRefsIns.computeIfAbsent(originUid, createInboundActorRefsForOrigin) + + // None is used as tombstone value after closed + private[this] val _classManifestsIns = new Long2ObjectHashMap[Option[InboundManifestCompression]]() + private val createInboundManifestsForOrigin = new LongFunction[Option[InboundManifestCompression]] { + override def apply(originUid: Long): Option[InboundManifestCompression] = { + val manifestHitters = new TopHeavyHitters[String](settings.Manifests.Max) + Some(new InboundManifestCompression(system, settings, originUid, inboundContext, manifestHitters, stopped)) + } + } + private def classManifestsIn(originUid: Long): Option[InboundManifestCompression] = + _classManifestsIns.computeIfAbsent(originUid, createInboundManifestsForOrigin) + + // actor ref compression --- + + override def decompressActorRef(originUid: Long, tableVersion: Byte, idx: Int): OptionVal[ActorRef] = + actorRefsIn(originUid) match { + case Some(a) ⇒ a.decompress(tableVersion, idx) + case None ⇒ OptionVal.None + } + + override def hitActorRef(originUid: Long, address: Address, ref: ActorRef, n: Int): Unit = { + if (ArterySettings.Compression.Debug) println(s"[compress] hitActorRef($originUid, $address, $ref, $n)") + actorRefsIn(originUid) match { + case Some(a) ⇒ a.increment(address, ref, n) + case None ⇒ // closed + } + } + + override def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Byte): Unit = { + _actorRefsIns.get(originUid) match { + case null ⇒ // ignore + case Some(a) ⇒ a.confirmAdvertisement(tableVersion) + case None ⇒ // closed + } + } + + // class manifest compression --- + + override def decompressClassManifest(originUid: Long, tableVersion: Byte, idx: Int): OptionVal[String] = + classManifestsIn(originUid) match { + case Some(a) ⇒ a.decompress(tableVersion, idx) + case None ⇒ OptionVal.None + } + + override def hitClassManifest(originUid: Long, address: Address, manifest: String, n: Int): Unit = { + if (ArterySettings.Compression.Debug) println(s"[compress] hitClassManifest($originUid, $address, $manifest, $n)") + classManifestsIn(originUid) match { + case Some(a) ⇒ a.increment(address, manifest, n) + case None ⇒ // closed + } + } + override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Byte): Unit = { + _classManifestsIns.get(originUid) match { + case null ⇒ // ignore + case Some(a) ⇒ a.confirmAdvertisement(tableVersion) + case None ⇒ // closed + } + } + + override def close(): Unit = stopped.set(true) + + override def close(originUid: Long): Unit = { + _actorRefsIns.get(originUid) match { + case null ⇒ + if (_actorRefsIns.putIfAbsent(originUid, None) != null) + close(originUid) + case oldValue @ Some(a) ⇒ + if (_actorRefsIns.replace(originUid, oldValue, None)) + a.close() + case None ⇒ // already closed + } + _classManifestsIns.get(originUid) match { + case null ⇒ + if (_classManifestsIns.putIfAbsent(originUid, None) != null) + close(originUid) + case oldValue @ Some(a) ⇒ + if (_classManifestsIns.replace(originUid, oldValue, None)) + a.close() + case None ⇒ // already closed + } + } + + // testing utilities --- + + /** INTERNAL API: for testing only */ + private[remote] def runNextActorRefAdvertisement() = { + import scala.collection.JavaConverters._ + _actorRefsIns.values().asScala.foreach { + case Some(inbound) ⇒ inbound.runNextTableAdvertisement() + case None ⇒ // closed + } + } + + /** INTERNAL API: for testing only */ + private[remote] def runNextClassManifestAdvertisement() = { + import scala.collection.JavaConverters._ + _classManifestsIns.values().asScala.foreach { + case Some(inbound) ⇒ inbound.runNextTableAdvertisement() + case None ⇒ // closed + } + } +} + +/** + * INTERNAL API + * Dedicated per remote system inbound compression table. + * + * The outbound context is available by looking it up in the association. + * It can be used to advertise a compression table. + * If the association is not complete - we simply dont advertise the table, which is fine (handshake not yet complete). + */ +private[remote] final class InboundActorRefCompression( + system: ActorSystem, + settings: ArterySettings.Compression, + originUid: Long, + inboundContext: InboundContext, + heavyHitters: TopHeavyHitters[ActorRef], + stopped: AtomicBoolean) + extends InboundCompression[ActorRef](system, settings, originUid, inboundContext, heavyHitters, stopped) { + + override def decompress(tableVersion: Byte, idx: Int): OptionVal[ActorRef] = + super.decompressInternal(tableVersion, idx, 0) + + override protected def tableAdvertisementInterval = settings.ActorRefs.AdvertisementInterval + + override def advertiseCompressionTable(outboundContext: OutboundContext, table: CompressionTable[ActorRef]): Unit = { + log.debug(s"Advertise {} compression [{}] to [{}#{}]", Logging.simpleName(getClass), table, outboundContext.remoteAddress, + originUid) + outboundContext.sendControl(CompressionProtocol.ActorRefCompressionAdvertisement(inboundContext.localAddress, table)) + } +} + +/** + * INTERNAL API + */ +private[remote] final class InboundManifestCompression( + system: ActorSystem, + settings: ArterySettings.Compression, + originUid: Long, + inboundContext: InboundContext, + heavyHitters: TopHeavyHitters[String], + stopped: AtomicBoolean) + extends InboundCompression[String](system, settings, originUid, inboundContext, heavyHitters, stopped) { + + override protected def tableAdvertisementInterval = settings.Manifests.AdvertisementInterval + + override def advertiseCompressionTable(outboundContext: OutboundContext, table: CompressionTable[String]): Unit = { + log.debug(s"Advertise {} compression [{}] to [{}#{}]", Logging.simpleName(getClass), table, outboundContext.remoteAddress, + originUid) + outboundContext.sendControl(CompressionProtocol.ClassManifestCompressionAdvertisement(inboundContext.localAddress, table)) + } + + override def increment(remoteAddress: Address, value: String, n: Long): Unit = + if (value != "") super.increment(remoteAddress, value, n) + + override def decompress(incomingTableVersion: Byte, idx: Int): OptionVal[String] = + decompressInternal(incomingTableVersion, idx, 0) +} +/** + * INTERNAL API + */ +private[remote] object InboundCompression { + + object State { + def empty[T] = State( + oldTable = DecompressionTable.disabled[T], + activeTable = DecompressionTable.empty[T], + nextTable = DecompressionTable.empty[T].copy(version = 1), + advertisementInProgress = None) + } + + final case class State[T]( + oldTable: DecompressionTable[T], + activeTable: DecompressionTable[T], + nextTable: DecompressionTable[T], + advertisementInProgress: Option[CompressionTable[T]]) { + + def startUsingNextTable(): State[T] = { + def incrementTableVersion(version: Byte): Byte = + if (version == 127) 0 + else (version + 1).toByte + + State( + oldTable = activeTable, + activeTable = nextTable, + nextTable = DecompressionTable.empty[T].copy(version = incrementTableVersion(nextTable.version)), + advertisementInProgress = None) + } + } + +} + +/** + * INTERNAL API + * Handles counting and detecting of heavy-hitters and compressing them via a table lookup. + */ +private[remote] abstract class InboundCompression[T >: Null]( + val system: ActorSystem, + val settings: ArterySettings.Compression, + originUid: Long, + inboundContext: InboundContext, + val heavyHitters: TopHeavyHitters[T], + stopped: AtomicBoolean) { + + val log = Logging(system, getClass) + + // FIXME InboundCompressions should be owned by the Decoder stage, and then doesn't have to be thread-safe + private[this] val state: AtomicReference[InboundCompression.State[T]] = + new AtomicReference(InboundCompression.State.empty) + // We should not continue sending advertisements to an association that might be dead (not quarantined yet) + @volatile private[this] var alive = true + private[this] val resendCount = new AtomicInteger + + private[this] val cms = new CountMinSketch(16, 1024, System.currentTimeMillis().toInt) + + log.debug("Initializing inbound compression for originUid [{}]", originUid) + val schedulerTask: Option[Cancellable] = + tableAdvertisementInterval match { + case d: FiniteDuration ⇒ + Some(system.scheduler.schedule(d, d)(runNextTableAdvertisement)(system.dispatcher)) + case _ ⇒ + None + } + + def close(): Unit = { + schedulerTask.foreach(_.cancel()) + log.debug("Closed inbound compression for originUid [{}]", originUid) + } + + /* ==== COMPRESSION ==== */ + + /** Override and specialize if needed, for default compression logic delegate to 3-param overload */ + def decompress(incomingTableVersion: Byte, idx: Int): OptionVal[T] + + /** + * Decompress given identifier into its original representation. + * Passed in tableIds must only ever be in not-decreasing order (as old tables are dropped), + * tableIds must not have gaps. If an "old" tableId is received the value will fail to be decompressed. + * + * @throws UnknownCompressedIdException if given id is not known, this may indicate a bug – such situation should not happen. + */ + @tailrec final def decompressInternal(incomingTableVersion: Byte, idx: Int, attemptCounter: Int): OptionVal[T] = { + // effectively should never loop more than once, to avoid infinite recursion blow up eagerly + if (attemptCounter > 2) throw new IllegalStateException(s"Unable to decompress $idx from table $incomingTableVersion. Internal state: ${state.get}") + + val current = state.get + val oldVersion = current.oldTable.version + val activeVersion = current.activeTable.version + + if (incomingTableVersion == DecompressionTable.DisabledVersion) OptionVal.None // no compression, bail out early + else if (incomingTableVersion == activeVersion) { + val value: T = current.activeTable.get(idx) + if (value != null) OptionVal.Some[T](value) + else throw new UnknownCompressedIdException(idx) + } else if (incomingTableVersion == oldVersion) { + // must handle one old table due to messages in flight during advertisement + val value: T = current.oldTable.get(idx) + if (value != null) OptionVal.Some[T](value) + else throw new UnknownCompressedIdException(idx) + } else if (current.advertisementInProgress.isDefined && incomingTableVersion == current.advertisementInProgress.get.version) { + log.debug( + "Received first value from originUid [{}] compressed using the advertised compression table, flipping to it (version: {})", + originUid, current.nextTable.version) + confirmAdvertisement(incomingTableVersion) + decompressInternal(incomingTableVersion, idx, attemptCounter + 1) // recurse + } else { + // which means that incoming version was > nextTable.version, which likely that + // it is using a table that was built for previous incarnation of this system + log.warning( + "Inbound message from originUid [{}] is using unknown compression table version. " + + "It may have been sent with compression table built for previous incarnation of this system. " + + "Versions activeTable: {}, nextTable: {}, incomingTable: {}", + originUid, activeVersion, current.nextTable.version, incomingTableVersion) + OptionVal.None + } + } + + @tailrec final def confirmAdvertisement(tableVersion: Byte): Unit = { + val current = state.get + current.advertisementInProgress match { + case Some(inProgress) if tableVersion == inProgress.version ⇒ + if (state.compareAndSet(current, current.startUsingNextTable())) + log.debug("Confirmed compression table version [{}] for originUid [{}]", tableVersion, originUid) + else + confirmAdvertisement(tableVersion) // recur + case Some(inProgress) if tableVersion != inProgress.version ⇒ + log.debug( + "Confirmed compression table version [{}] for originUid [{}] but other version in progress [{}]", + tableVersion, originUid, inProgress.version) + case None ⇒ // already confirmed + } + + } + + /** + * Add `n` occurrence for the given key and call `heavyHittedDetected` if element has become a heavy hitter. + * Empty keys are omitted. + */ + def increment(remoteAddress: Address, value: T, n: Long): Unit = { + val count = cms.addObjectAndEstimateCount(value, n) + addAndCheckIfheavyHitterDetected(value, count) + alive = true + } + + /** Mutates heavy hitters */ + private def addAndCheckIfheavyHitterDetected(value: T, count: Long): Boolean = { + heavyHitters.update(value, count) + } + + /* ==== TABLE ADVERTISEMENT ==== */ + + protected def tableAdvertisementInterval: Duration + + /** + * INTERNAL / TESTING API + * Used for manually triggering when a compression table should be advertised. + * Note that most likely you'd want to set the advertisment-interval to `0` when using this. + */ + private[remote] def triggerNextTableAdvertisement(): Unit = // TODO use this in tests for triggering + runNextTableAdvertisement() + + /** + * Entry point to advertising a new compression table. + * + * [1] First we must *hand the new table over to the Incoming compression side on this system*, + * so it will not be used by someone else before "we" know about it in the Decoder. + * [2] Then the table must be *advertised to the remote system*, and MAY start using it immediately + * + * It must be advertised to the other side so it can start using it in its outgoing compression. + * Triggers compression table advertisement. May be triggered by schedule or manually, i.e. for testing. + */ + private[remote] def runNextTableAdvertisement() = { + if (stopped.get) { + schedulerTask.foreach(_.cancel()) + } else { + val current = state.get + if (ArterySettings.Compression.Debug) println(s"[compress] runNextTableAdvertisement, state = $current") + current.advertisementInProgress match { + case None ⇒ + inboundContext.association(originUid) match { + case OptionVal.Some(association) ⇒ + if (alive) { + val table = prepareCompressionAdvertisement(current.nextTable.version) + // TODO expensive, check if building the other way wouldn't be faster? + val nextState = current.copy(nextTable = table.invert, advertisementInProgress = Some(table)) + if (state.compareAndSet(current, nextState)) { + alive = false // will be set to true on first incoming message + resendCount.set(0) + advertiseCompressionTable(association, table) + } + } else + log.debug("Inbound compression table for originUid [{}] not changed, no need to advertise same.", originUid) + + case OptionVal.None ⇒ + // otherwise it's too early, association not ready yet. + // so we don't build the table since we would not be able to send it anyway. + log.debug("No Association for originUid [{}] yet, unable to advertise compression table.", originUid) + } + + case Some(inProgress) ⇒ + if (resendCount.incrementAndGet() <= 5) { + // The ActorRefCompressionAdvertisement message is resent because it can be lost + log.debug( + "Advertisment in progress for originUid [{}] version {}, resending", + originUid, inProgress.version) + inboundContext.association(originUid) match { + case OptionVal.Some(association) ⇒ + advertiseCompressionTable(association, inProgress) // resend + case OptionVal.None ⇒ + } + } else { + // give up, it might be dead + log.debug( + "Advertisment in progress for originUid [{}] version {} but no confirmation after retries.", + originUid, inProgress.version) + confirmAdvertisement(inProgress.version) + } + } + } + } + + /** + * Must be implemented by extending classes in order to send a `ControlMessage` + * of appropriate type to the remote system in order to advertise the compression table to it. + */ + protected def advertiseCompressionTable(association: OutboundContext, table: CompressionTable[T]): Unit + + private def prepareCompressionAdvertisement(nextTableVersion: Byte): CompressionTable[T] = { + // TODO surely we can do better than that, optimise + CompressionTable(originUid, nextTableVersion, Map(heavyHitters.snapshot.filterNot(_ == null).zipWithIndex: _*)) + } + + override def toString = + s"""${getClass.getSimpleName}(countMinSketch: $cms, heavyHitters: $heavyHitters)""" + +} + +/** + * INTERNAL API + */ +private[akka] final class UnknownCompressedIdException(id: Long) + extends RuntimeException( + s"Attempted de-compress unknown id [$id]! " + + s"This could happen if this node has started a new ActorSystem bound to the same address as previously, " + + s"and previous messages from a remote system were still in flight (using an old compression table). " + + s"The remote system is expected to drop the compression table and this system will advertise a new one.") + +/** + * INTERNAL API + * + * Literarily, no compression! + */ +private[remote] case object NoInboundCompressions extends InboundCompressions { + override def hitActorRef(originUid: Long, remote: Address, ref: ActorRef, n: Int): Unit = () + override def decompressActorRef(originUid: Long, tableVersion: Byte, idx: Int): OptionVal[ActorRef] = + if (idx == -1) throw new IllegalArgumentException("Attemted decompression of illegal compression id: -1") + else OptionVal.None + override def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Byte): Unit = () + + override def hitClassManifest(originUid: Long, remote: Address, manifest: String, n: Int): Unit = () + override def decompressClassManifest(originUid: Long, tableVersion: Byte, idx: Int): OptionVal[String] = + if (idx == -1) throw new IllegalArgumentException("Attemted decompression of illegal compression id: -1") + else OptionVal.None + override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Byte): Unit = () + + override def close(): Unit = () + + override def close(originUid: Long): Unit = () +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/TopHeavyHitters.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/TopHeavyHitters.scala new file mode 100644 index 0000000000..638806bb58 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/TopHeavyHitters.scala @@ -0,0 +1,298 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery.compress + +import java.util.Objects + +import scala.annotation.{ switch, tailrec } +import scala.reflect.ClassTag + +/** + * INTERNAL API + * + * Mutable, open-addressing with linear-probing (though naive one which in theory could get pathological) heavily optimised "top N heavy hitters" data-structure. + * + * Keeps a number of specific heavy hitters around in memory. + * + * See also Section 5.2 of http://dimacs.rutgers.edu/~graham/pubs/papers/cm-full.pdf + * for a discussion about the assumptions made and guarantees about the Heavy Hitters made in this model. + * We assume the Cash Register model in which there are only additions, which simplifies HH detecion significantly. + */ +private[remote] final class TopHeavyHitters[T >: Null](val max: Int)(implicit classTag: ClassTag[T]) { + + require((max & (max - 1)) == 0, "Maximum numbers of heavy hitters should be in form of 2^k for any natural k") + + val capacity = max * 2 + val mask = capacity - 1 + + import TopHeavyHitters._ + + private[this] val hashes: Array[Int] = Array.ofDim(capacity) + private[this] val items: Array[T] = Array.ofDim[T](capacity) + private[this] val weights: Array[Long] = Array.ofDim(capacity) + private[this] val heapIndex: Array[Int] = Array.fill(capacity)(-1) + private[this] val heap: Array[Int] = Array.fill(max)(-1) + + // TODO think if we could get away without copy + /** Returns the current heavy hitters, order is not of significance */ + def snapshot: Array[T] = { + val snap = Array.ofDim(max).asInstanceOf[Array[T]] + var i = 0 + while (i < max) { + val index = heap(i) + val value = + if (index < 0) null + else items(index) + snap(i) = value + i += 1 + } + snap + } + + def toDebugString = + s"""TopHeavyHitters( + | max: $max, + | lowestHitterIdx: $lowestHitterIndex (weight: $lowestHitterWeight) + | + | hashes: ${hashes.toList.mkString("[", ", ", "]")} + | weights: ${weights.toList.mkString("[", ", ", "]")} + | items: ${items.toList.mkString("[", ", ", "]")} + | heapIndex: ${heapIndex.toList.mkString("[", ", ", "]")} + | heap: ${heap.toList.mkString("[", ", ", "]")} + |)""".stripMargin + + /** + * Attempt adding item to heavy hitters set, if it does not fit in the top yet, + * it will be dropped and the method will return `false`. + * + * @return `true` if the added item has become a heavy hitter. + */ + // TODO possibly can be optimised further? (there is a benchmark) + def update(item: T, count: Long): Boolean = + isHeavy(count) && { // O(1) terminate execution ASAP if known to not be a heavy hitter anyway + val hashCode = new HashCodeVal(item.hashCode()) // avoid re-calculating hashCode + val startIndex = hashCode.get & mask + (findHashIdx(startIndex, hashCode): @switch) match { // worst case O(n), common O(1 + alpha), can't really bin search here since indexes are kept in synch with other arrays hmm... + case -1 ⇒ + // not previously heavy hitter + insertKnownNewHeavy(hashCode, item, count) // O(log n + alpha) + true + case potentialIndexGuess ⇒ + // the found index could be one of many which hash to the same value (we're using open-addressing), + // so it is only used as hint for the replace call. If the value matches, we're good, if not we need to search from here onwards. + val actualIdx = findItemIdx(potentialIndexGuess, hashCode, item) + + if (actualIdx == -1) { + insertKnownNewHeavy(hashCode, item, count) // O(1 + log n), we simply replace the current lowest heavy hitter + true + } else replaceExistingHeavyHitter(actualIdx, hashCode, item, count) // usually O(1), worst case O(n) if we need to scan due to hash conflicts + } + } + + def isHeavy(count: Long): Boolean = + count > lowestHitterWeight + + private def findItemIdx(searchFromIndex: Int, hashCode: HashCodeVal, o: T): Int = { + @tailrec def loop(index: Int, start: Int, hashCodeVal: HashCodeVal, o: T): Int = { + if (index == start) -1 + else if (hashCodeVal.get == hashes(index)) { + val item: T = items(index) + if (item == o) { + index + } else { + loop((index + 1) & mask, start, hashCodeVal, o) + } + } else { + loop((index + 1) & mask, start, hashCodeVal, o) + } + } + + if (searchFromIndex == -1) -1 + else if (Objects.equals(items(searchFromIndex), o)) searchFromIndex + else loop((searchFromIndex + 1) & mask, searchFromIndex, hashCode, o) + } + + /** + * Replace existing heavy hitter – give it a new `count` value. + * If it was the lowest heavy hitter we update the `_lowestHitterIdx` as well, otherwise there is no need to. + * + * @return `false` to indicate "no, this insertion did not make this item a new heavy hitter" if update was successful, + * otherwise might throw [[NoSuchElementException]] if the `item` actually was not found + */ + @tailrec private def replaceExistingHeavyHitter(foundHashIndex: Int, hashCode: HashCodeVal, item: T, count: Long): Boolean = + if (foundHashIndex == -1) throw new NoSuchElementException(s"Item $item is not present in HeavyHitters, can not replace it!") + else if (Objects.equals(items(foundHashIndex), item)) { + updateCount(foundHashIndex, count) // we don't need to change `hashCode` or `item`, those remain the same + fixHeap(heapIndex(foundHashIndex)) + false // not a "new" heavy hitter, since we only replaced it (so it was signaled as new once before) + } else replaceExistingHeavyHitter(findHashIdx(foundHashIndex + 1, hashCode), hashCode, item, count) // recurse + + private def findHashIdx(searchFromIndex: Int, hashCode: HashCodeVal): Int = + findEqIndex(hashes, searchFromIndex, hashCode.get) + + /** + * Fix heap property on `heap` array + * @param index place to check and fix + */ + @tailrec + private def fixHeap(index: Int): Unit = { + val leftIndex = index * 2 + 1 + val rightIndex = index * 2 + 2 + val currentWeights: Long = weights(heap(index)) + if (rightIndex < max) { + val leftValueIndex: Int = heap(leftIndex) + val rightValueIndex: Int = heap(rightIndex) + if (leftValueIndex < 0) { + swapHeapNode(index, leftIndex) + fixHeap(leftIndex) + } else if (rightValueIndex < 0) { + swapHeapNode(index, rightIndex) + fixHeap(rightIndex) + } else { + val rightWeights: Long = weights(rightValueIndex) + val leftWeights: Long = weights(leftValueIndex) + if (leftWeights < rightWeights) { + if (currentWeights > leftWeights) { + swapHeapNode(index, leftIndex) + fixHeap(leftIndex) + } + } else { + if (currentWeights > rightWeights) { + swapHeapNode(index, rightIndex) + fixHeap(rightIndex) + } + } + } + } else if (leftIndex < max) { + val leftValueIndex: Int = heap(leftIndex) + if (leftValueIndex < 0) { + swapHeapNode(index, leftIndex) + fixHeap(leftIndex) + } else { + val leftWeights: Long = weights(leftValueIndex) + if (currentWeights > leftWeights) { + swapHeapNode(index, leftIndex) + fixHeap(leftIndex) + } + } + } + } + + /** + * Swaps two elements in `heap` array and maintain correct index in `heapIndex`. + * + * @param a index of first element + * @param b index of second element + */ + private def swapHeapNode(a: Int, b: Int): Unit = { + if (heap(a) >= 0) { + heapIndex(heap(a)) = b + } + if (heap(b) >= 0) { + heapIndex(heap(b)) = a + } + val temp = heap(a) + heap(a) = heap(b) + heap(b) = temp + } + + /** + * Puts the item and additional information into the index of the current lowest hitter. + * + * @return index at which the insertion was performed + */ + private def insertKnownNewHeavy(hashCode: HashCodeVal, item: T, count: Long): Unit = { + removeHash(lowestHitterIndex) + lowestHitterIndex = insert(hashCode, item, count) + } + + /** + * Remove value from hash-table based on position. + * + * @param index position to remove + */ + private def removeHash(index: Int): Unit = { + if (index > 0) { + items(index) = null + hashes(index) = 0 + weights(index) = 0 + } + } + + /** + * Only update the count for a given index, e.g. if value and hashCode remained the same. + */ + private def updateCount(idx: Int, count: Long): Unit = + weights(idx) = count + + /** + * Insert value in hash-table. + * + * Using open addressing for resolving collisions. + * Initial index is reminder in division hashCode and table size. + * + * @param hashCode hashCode of item + * @param item value which should be added to hash-table + * @param count count associated to value + * @return Index in hash-table where was inserted + */ + private def insert(hashCode: HashCodeVal, item: T, count: Long): Int = { + var index: Int = hashCode.get & mask + while (items(index) != null) { + index = (index + 1) & mask + } + hashes(index) = hashCode.get + items(index) = item + weights(index) = count + index + } + + /** Weight of lowest heavy hitter, if a new inserted item has a weight greater than this it is a heavy hitter. */ + def lowestHitterWeight: Long = { + val index: Int = lowestHitterIndex + if (index > 0) { + weights(index) + } else { + 0 + } + + } + + private def lowestHitterIndex: Int = { + heap(0) + } + + private def lowestHitterIndex_=(index: Int): Unit = { + heap(0) = index + heapIndex(index) = 0 + fixHeap(0) + } + + private def findEqIndex(hashes: Array[Int], searchFromIndex: Int, hashCode: Int): Int = { + var i: Int = 0 + while (i < hashes.length) { + val index = (i + searchFromIndex) & mask + if (hashes(index) == hashCode) { + return index + } + i += 1 + } + -1 + } + + override def toString = + s"${getClass.getSimpleName}(max:$max)" +} + +/** + * INTERNAL API + */ +private[remote] object TopHeavyHitters { + + /** Value class to avoid mixing up count and hashCode in APIs. */ + private[compress] final class HashCodeVal(val get: Int) extends AnyVal { + def isEmpty = false + } + +} diff --git a/akka-remote/src/main/scala/akka/remote/serialization/ActorRefResolveCache.scala b/akka-remote/src/main/scala/akka/remote/serialization/ActorRefResolveCache.scala new file mode 100644 index 0000000000..4ced76fe04 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/serialization/ActorRefResolveCache.scala @@ -0,0 +1,64 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.serialization + +import akka.actor.ActorRef +import akka.actor.ActorSystem +import akka.actor.EmptyLocalActorRef +import akka.actor.ExtendedActorSystem +import akka.actor.Extension +import akka.actor.ExtensionId +import akka.actor.ExtensionIdProvider +import akka.remote.RemoteActorRefProvider +import akka.remote.artery.FastHash +import akka.remote.artery.LruBoundedCache + +/** + * INTERNAL API: Thread local cache per actor system + */ +private[akka] object ActorRefResolveThreadLocalCache + extends ExtensionId[ActorRefResolveThreadLocalCache] with ExtensionIdProvider { + + override def get(system: ActorSystem): ActorRefResolveThreadLocalCache = super.get(system) + + override def lookup = ActorRefResolveThreadLocalCache + + override def createExtension(system: ExtendedActorSystem): ActorRefResolveThreadLocalCache = + new ActorRefResolveThreadLocalCache(system) +} + +/** + * INTERNAL API + */ +private[akka] class ActorRefResolveThreadLocalCache(val system: ExtendedActorSystem) extends Extension { + + private val provider = system.provider match { + case r: RemoteActorRefProvider ⇒ r + case _ ⇒ throw new IllegalArgumentException( + "ActorRefResolveThreadLocalCache can only be used with RemoteActorRefProvider, " + + s"not with ${system.provider.getClass}") + } + + private val current = new ThreadLocal[ActorRefResolveCache] { + override def initialValue: ActorRefResolveCache = new ActorRefResolveCache(provider) + } + + def threadLocalCache(provider: RemoteActorRefProvider): ActorRefResolveCache = + current.get + +} + +/** + * INTERNAL API + */ +private[akka] final class ActorRefResolveCache(provider: RemoteActorRefProvider) + extends LruBoundedCache[String, ActorRef](capacity = 1024, evictAgeThreshold = 600) { + + override protected def compute(k: String): ActorRef = + provider.internalResolveActorRef(k) + + override protected def hash(k: String): Int = FastHash.ofString(k) + + override protected def isCacheable(v: ActorRef): Boolean = !v.isInstanceOf[EmptyLocalActorRef] +} diff --git a/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala new file mode 100644 index 0000000000..e11be5f28f --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala @@ -0,0 +1,247 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ +package akka.remote.serialization + +import akka.actor.{ ActorRef, Address, ExtendedActorSystem } +import akka.protobuf.MessageLite +import akka.remote.RemoteWatcher.ArteryHeartbeatRsp +import akka.remote.artery.OutboundHandshake.{ HandshakeReq, HandshakeRsp } +import akka.remote.artery.compress.CompressionProtocol._ +import akka.remote.artery.compress.{ CompressionProtocol, CompressionTable } +import akka.remote.artery.{ ActorSystemTerminating, ActorSystemTerminatingAck, Quarantined, SystemMessageDelivery } +import akka.remote._ +import akka.serialization.{ BaseSerializer, Serialization, SerializationExtension, SerializerWithStringManifest } + +/** INTERNAL API */ +private[akka] object ArteryMessageSerializer { + private val QuarantinedManifest = "a" + private val ActorSystemTerminatingManifest = "b" + private val ActorSystemTerminatingAckManifest = "c" + private val HandshakeReqManifest = "d" + private val HandshakeRspManifest = "e" + private val ActorRefCompressionAdvertisementManifest = "f" + private val ActorRefCompressionAdvertisementAckManifest = "g" + private val ClassManifestCompressionAdvertisementManifest = "h" + private val ClassManifestCompressionAdvertisementAckManifest = "i" + private val SystemMessageEnvelopeManifest = "j" + private val SystemMessageDeliveryAckManifest = "k" + private val SystemMessageDeliveryNackManifest = "l" + + private val ArteryHeartbeatManifest = "m" + private val ArteryHeartbeatRspManifest = "n" + + private final val DeadLettersRepresentation = "" +} + +/** INTERNAL API */ +private[akka] final class ArteryMessageSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest with BaseSerializer { + import ArteryMessageSerializer._ + + private lazy val serialization = SerializationExtension(system) + + override def manifest(o: AnyRef): String = o match { // most frequent ones first + case _: SystemMessageDelivery.SystemMessageEnvelope ⇒ SystemMessageEnvelopeManifest + case _: SystemMessageDelivery.Ack ⇒ SystemMessageDeliveryAckManifest + case _: HandshakeReq ⇒ HandshakeReqManifest + case _: HandshakeRsp ⇒ HandshakeRspManifest + case _: RemoteWatcher.ArteryHeartbeat.type ⇒ ArteryHeartbeatManifest + case _: RemoteWatcher.ArteryHeartbeatRsp ⇒ ArteryHeartbeatRspManifest + case _: SystemMessageDelivery.Nack ⇒ SystemMessageDeliveryNackManifest + case _: Quarantined ⇒ QuarantinedManifest + case _: ActorSystemTerminating ⇒ ActorSystemTerminatingManifest + case _: ActorSystemTerminatingAck ⇒ ActorSystemTerminatingAckManifest + case _: CompressionProtocol.ActorRefCompressionAdvertisement ⇒ ActorRefCompressionAdvertisementManifest + case _: CompressionProtocol.ActorRefCompressionAdvertisementAck ⇒ ActorRefCompressionAdvertisementAckManifest + case _: CompressionProtocol.ClassManifestCompressionAdvertisement ⇒ ClassManifestCompressionAdvertisementManifest + case _: CompressionProtocol.ClassManifestCompressionAdvertisementAck ⇒ ClassManifestCompressionAdvertisementAckManifest + case _ ⇒ + throw new IllegalArgumentException(s"Can't serialize object of type ${o.getClass} in [${getClass.getName}]") + } + + override def toBinary(o: AnyRef): Array[Byte] = o match { // most frequent ones first + case env: SystemMessageDelivery.SystemMessageEnvelope ⇒ serializeSystemMessageEnvelope(env).toByteArray + case SystemMessageDelivery.Ack(seqNo, from) ⇒ serializeSystemMessageDeliveryAck(seqNo, from).toByteArray + case HandshakeReq(from, to) ⇒ serializeHandshakeReq(from, to).toByteArray + case HandshakeRsp(from) ⇒ serializeWithAddress(from).toByteArray + case RemoteWatcher.ArteryHeartbeat ⇒ Array.emptyByteArray + case RemoteWatcher.ArteryHeartbeatRsp(from) ⇒ serializeArteryHeartbeatRsp(from).toByteArray + case SystemMessageDelivery.Nack(seqNo, from) ⇒ serializeSystemMessageDeliveryAck(seqNo, from).toByteArray + case q: Quarantined ⇒ serializeQuarantined(q).toByteArray + case ActorSystemTerminating(from) ⇒ serializeWithAddress(from).toByteArray + case ActorSystemTerminatingAck(from) ⇒ serializeWithAddress(from).toByteArray + case adv: ActorRefCompressionAdvertisement ⇒ serializeActorRefCompressionAdvertisement(adv).toByteArray + case ActorRefCompressionAdvertisementAck(from, id) ⇒ serializeCompressionTableAdvertisementAck(from, id).toByteArray + case adv: ClassManifestCompressionAdvertisement ⇒ serializeCompressionAdvertisement(adv)(identity).toByteArray + case ClassManifestCompressionAdvertisementAck(from, id) ⇒ serializeCompressionTableAdvertisementAck(from, id).toByteArray + } + + override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef = manifest match { // most frequent ones first (could be made a HashMap in the future) + case SystemMessageEnvelopeManifest ⇒ deserializeSystemMessageEnvelope(bytes) + case SystemMessageDeliveryAckManifest ⇒ deserializeSystemMessageDeliveryAck(bytes, SystemMessageDelivery.Ack) + case HandshakeReqManifest ⇒ deserializeHandshakeReq(bytes, HandshakeReq) + case HandshakeRspManifest ⇒ deserializeWithFromAddress(bytes, HandshakeRsp) + case SystemMessageDeliveryNackManifest ⇒ deserializeSystemMessageDeliveryAck(bytes, SystemMessageDelivery.Nack) + case QuarantinedManifest ⇒ deserializeQuarantined(ArteryControlFormats.Quarantined.parseFrom(bytes)) + case ActorSystemTerminatingManifest ⇒ deserializeWithFromAddress(bytes, ActorSystemTerminating) + case ActorSystemTerminatingAckManifest ⇒ deserializeWithFromAddress(bytes, ActorSystemTerminatingAck) + case ActorRefCompressionAdvertisementManifest ⇒ deserializeActorRefCompressionAdvertisement(bytes) + case ActorRefCompressionAdvertisementAckManifest ⇒ deserializeCompressionTableAdvertisementAck(bytes, ActorRefCompressionAdvertisementAck) + case ClassManifestCompressionAdvertisementManifest ⇒ deserializeCompressionAdvertisement(bytes, identity, ClassManifestCompressionAdvertisement) + case ClassManifestCompressionAdvertisementAckManifest ⇒ deserializeCompressionTableAdvertisementAck(bytes, ClassManifestCompressionAdvertisementAck) + case ArteryHeartbeatManifest ⇒ RemoteWatcher.ArteryHeartbeat + case ArteryHeartbeatRspManifest ⇒ deserializeArteryHeartbeatRsp(bytes, ArteryHeartbeatRsp) + case _ ⇒ throw new IllegalArgumentException(s"Manifest '$manifest' not defined for ArteryControlMessageSerializer (serializer id $identifier)") + } + + import scala.collection.JavaConverters._ + + def serializeQuarantined(quarantined: Quarantined): ArteryControlFormats.Quarantined = + ArteryControlFormats.Quarantined.newBuilder() + .setFrom(serializeUniqueAddress(quarantined.from)) + .setTo(serializeUniqueAddress(quarantined.to)) + .build + + def deserializeQuarantined(quarantined: ArteryControlFormats.Quarantined): Quarantined = + Quarantined(deserializeUniqueAddress(quarantined.getFrom), deserializeUniqueAddress(quarantined.getTo)) + + def serializeActorRef(ref: ActorRef): String = + if ((ref eq ActorRef.noSender) || (ref eq system.deadLetters)) DeadLettersRepresentation + else Serialization.serializedActorPath(ref) + + def deserializeActorRef(str: String): ActorRef = + if (str == DeadLettersRepresentation) system.deadLetters + else system.provider.resolveActorRef(str) + + def serializeActorRefCompressionAdvertisement(adv: ActorRefCompressionAdvertisement): ArteryControlFormats.CompressionTableAdvertisement = + serializeCompressionAdvertisement(adv)(serializeActorRef) + + def deserializeActorRefCompressionAdvertisement(bytes: Array[Byte]): ActorRefCompressionAdvertisement = + deserializeCompressionAdvertisement(bytes, deserializeActorRef, ActorRefCompressionAdvertisement) + + def serializeCompressionAdvertisement[T](adv: CompressionAdvertisement[T])(keySerializer: T ⇒ String): ArteryControlFormats.CompressionTableAdvertisement = { + val builder = + ArteryControlFormats.CompressionTableAdvertisement.newBuilder + .setFrom(serializeUniqueAddress(adv.from)) + .setOriginUid(adv.table.originUid) + .setTableVersion(adv.table.version) + + adv.table.dictionary.foreach { + case (key, value) ⇒ + builder + .addKeys(keySerializer(key)) + .addValues(value) + } + + builder.build + } + + def deserializeCompressionAdvertisement[T, U](bytes: Array[Byte], keyDeserializer: String ⇒ T, create: (UniqueAddress, CompressionTable[T]) ⇒ U): U = { + val protoAdv = ArteryControlFormats.CompressionTableAdvertisement.parseFrom(bytes) + + val kvs = + protoAdv.getKeysList.asScala.map(keyDeserializer).zip( + protoAdv.getValuesList.asScala.asInstanceOf[Iterable[Int]] /* to avoid having to call toInt explicitly */ ) + + val table = CompressionTable(protoAdv.getOriginUid, protoAdv.getTableVersion.byteValue, kvs.toMap) + create(deserializeUniqueAddress(protoAdv.getFrom), table) + } + + def serializeCompressionTableAdvertisementAck(from: UniqueAddress, version: Int): MessageLite = + ArteryControlFormats.CompressionTableAdvertisementAck.newBuilder + .setFrom(serializeUniqueAddress(from)) + .setVersion(version) + .build() + + def deserializeCompressionTableAdvertisementAck(bytes: Array[Byte], create: (UniqueAddress, Byte) ⇒ AnyRef): AnyRef = { + val msg = ArteryControlFormats.CompressionTableAdvertisementAck.parseFrom(bytes) + create(deserializeUniqueAddress(msg.getFrom), msg.getVersion.toByte) + } + + def serializeSystemMessageEnvelope(env: SystemMessageDelivery.SystemMessageEnvelope): ArteryControlFormats.SystemMessageEnvelope = { + val msg = MessageSerializer.serialize(system, env.message) + + val builder = + ArteryControlFormats.SystemMessageEnvelope.newBuilder + .setMessage(msg.getMessage) + .setSerializerId(msg.getSerializerId) + .setSeqNo(env.seqNo) + .setAckReplyTo(serializeUniqueAddress(env.ackReplyTo)) + + if (msg.hasMessageManifest) builder.setMessageManifest(msg.getMessageManifest) + + builder.build + } + def deserializeSystemMessageEnvelope(bytes: Array[Byte]): SystemMessageDelivery.SystemMessageEnvelope = { + val protoEnv = ArteryControlFormats.SystemMessageEnvelope.parseFrom(bytes) + + SystemMessageDelivery.SystemMessageEnvelope( + serialization.deserialize( + protoEnv.getMessage.toByteArray, + protoEnv.getSerializerId, + if (protoEnv.hasMessageManifest) protoEnv.getMessageManifest.toStringUtf8 else "").get, + protoEnv.getSeqNo, + deserializeUniqueAddress(protoEnv.getAckReplyTo)) + } + + def serializeSystemMessageDeliveryAck(seqNo: Long, from: UniqueAddress): ArteryControlFormats.SystemMessageDeliveryAck = + ArteryControlFormats.SystemMessageDeliveryAck.newBuilder + .setSeqNo(seqNo) + .setFrom(serializeUniqueAddress(from)) + .build + + def deserializeSystemMessageDeliveryAck(bytes: Array[Byte], create: (Long, UniqueAddress) ⇒ AnyRef): AnyRef = { + val protoAck = ArteryControlFormats.SystemMessageDeliveryAck.parseFrom(bytes) + + create(protoAck.getSeqNo, deserializeUniqueAddress(protoAck.getFrom)) + } + + def serializeWithAddress(from: UniqueAddress): MessageLite = + ArteryControlFormats.MessageWithAddress.newBuilder.setAddress(serializeUniqueAddress(from)).build() + + def deserializeWithFromAddress(bytes: Array[Byte], create: UniqueAddress ⇒ AnyRef): AnyRef = + create(deserializeUniqueAddress(ArteryControlFormats.MessageWithAddress.parseFrom(bytes).getAddress)) + + def serializeHandshakeReq(from: UniqueAddress, to: Address): MessageLite = + ArteryControlFormats.HandshakeReq.newBuilder + .setFrom(serializeUniqueAddress(from)) + .setTo(serializeAddress(to)) + .build() + + def deserializeHandshakeReq(bytes: Array[Byte], create: (UniqueAddress, Address) ⇒ HandshakeReq): HandshakeReq = { + val protoEnv = ArteryControlFormats.HandshakeReq.parseFrom(bytes) + create(deserializeUniqueAddress(protoEnv.getFrom), deserializeAddress(protoEnv.getTo)) + } + + def serializeUniqueAddress(address: UniqueAddress): ArteryControlFormats.UniqueAddress = + ArteryControlFormats.UniqueAddress.newBuilder() + .setAddress(serializeAddress(address.address)) + .setUid(address.uid) + .build() + + def deserializeUniqueAddress(address: ArteryControlFormats.UniqueAddress): UniqueAddress = + UniqueAddress(deserializeAddress(address.getAddress), address.getUid) + + def serializeAddress(address: Address): ArteryControlFormats.Address = + address match { + case Address(protocol, system, Some(host), Some(port)) ⇒ + ArteryControlFormats.Address.newBuilder() + .setProtocol(protocol) + .setSystem(system) + .setHostname(host) + .setPort(port) + .build() + case _ ⇒ throw new IllegalArgumentException(s"Address [$address] could not be serialized: host or port missing.") + } + + def deserializeAddress(address: ArteryControlFormats.Address): Address = + Address(address.getProtocol, address.getSystem, address.getHostname, address.getPort) + + def serializeArteryHeartbeatRsp(uid: Long): ArteryControlFormats.ArteryHeartbeatRsp = + ArteryControlFormats.ArteryHeartbeatRsp.newBuilder().setUid(uid).build() + + def deserializeArteryHeartbeatRsp(bytes: Array[Byte], create: Long ⇒ ArteryHeartbeatRsp): ArteryHeartbeatRsp = { + val msg = ArteryControlFormats.ArteryHeartbeatRsp.parseFrom(bytes) + create(msg.getUid) + } +} diff --git a/akka-remote/src/main/scala/akka/remote/serialization/MiscMessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/serialization/MiscMessageSerializer.scala index 025511de9c..b29c5a2f2d 100644 --- a/akka-remote/src/main/scala/akka/remote/serialization/MiscMessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/serialization/MiscMessageSerializer.scala @@ -5,33 +5,45 @@ package akka.remote.serialization import akka.actor._ import akka.protobuf.ByteString -import akka.remote.ContainerFormats -import akka.serialization.{ Serialization, BaseSerializer, SerializationExtension, SerializerWithStringManifest } +import akka.remote.{ ContainerFormats, RemoteWatcher } +import akka.serialization.{ BaseSerializer, Serialization, SerializationExtension, SerializerWithStringManifest } class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest with BaseSerializer { + // WARNING! This must lazy otherwise it will deadlock the ActorSystem creation private lazy val serialization = SerializationExtension(system) + private val payloadSupport = new WrappedPayloadSupport(system) + private val throwableSupport = new ThrowableSupport(system) - private val NoneSerialized = Array.empty[Byte] + private val ParameterlessSerializedMessage = Array.empty[Byte] def toBinary(obj: AnyRef): Array[Byte] = obj match { - case identify: Identify ⇒ serializeIdentify(identify) - case identity: ActorIdentity ⇒ serializeActorIdentity(identity) - case Some(value) ⇒ serializeSome(value) - case None ⇒ NoneSerialized - case _ ⇒ throw new IllegalArgumentException(s"Cannot serialize object of type [${obj.getClass.getName}]") + case identify: Identify ⇒ serializeIdentify(identify) + case identity: ActorIdentity ⇒ serializeActorIdentity(identity) + case Some(value) ⇒ serializeSome(value) + case None ⇒ ParameterlessSerializedMessage + case r: ActorRef ⇒ serializeActorRef(r) + case s: Status.Success ⇒ serializeStatusSuccess(s) + case f: Status.Failure ⇒ serializeStatusFailure(f) + case ex: ActorInitializationException ⇒ serializeActorInitializationException(ex) + case t: Throwable ⇒ throwableSupport.serializeThrowable(t) + case PoisonPill ⇒ ParameterlessSerializedMessage + case Kill ⇒ ParameterlessSerializedMessage + case RemoteWatcher.Heartbeat ⇒ ParameterlessSerializedMessage + case hbrsp: RemoteWatcher.HeartbeatRsp ⇒ serializeHeartbeatRsp(hbrsp) + case _ ⇒ throw new IllegalArgumentException(s"Cannot serialize object of type [${obj.getClass.getName}]") } private def serializeIdentify(identify: Identify): Array[Byte] = ContainerFormats.Identify.newBuilder() - .setMessageId(payloadBuilder(identify.messageId)) + .setMessageId(payloadSupport.payloadBuilder(identify.messageId)) .build() .toByteArray private def serializeActorIdentity(actorIdentity: ActorIdentity): Array[Byte] = { val builder = ContainerFormats.ActorIdentity.newBuilder() - .setCorrelationId(payloadBuilder(actorIdentity.correlationId)) + .setCorrelationId(payloadSupport.payloadBuilder(actorIdentity.correlationId)) actorIdentity.ref.foreach { actorRef ⇒ builder.setRef(actorRefBuilder(actorRef)) @@ -44,51 +56,81 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW private def serializeSome(someValue: Any): Array[Byte] = ContainerFormats.Option.newBuilder() - .setValue(payloadBuilder(someValue)) + .setValue(payloadSupport.payloadBuilder(someValue)) .build() .toByteArray + private def serializeActorRef(ref: ActorRef): Array[Byte] = + actorRefBuilder(ref).build().toByteArray + + private def serializeHeartbeatRsp(hbrsp: RemoteWatcher.HeartbeatRsp): Array[Byte] = { + ContainerFormats.WatcherHeartbeatResponse.newBuilder().setUid(hbrsp.addressUid).build().toByteArray + } + private def actorRefBuilder(actorRef: ActorRef): ContainerFormats.ActorRef.Builder = ContainerFormats.ActorRef.newBuilder() .setPath(Serialization.serializedActorPath(actorRef)) - private def payloadBuilder(input: Any): ContainerFormats.Payload.Builder = { - val payload = input.asInstanceOf[AnyRef] - val builder = ContainerFormats.Payload.newBuilder() - val serializer = serialization.findSerializerFor(payload) - - builder - .setEnclosedMessage(ByteString.copyFrom(serializer.toBinary(payload))) - .setSerializerId(serializer.identifier) - - serializer match { - case ser2: SerializerWithStringManifest ⇒ - val manifest = ser2.manifest(payload) - if (manifest != "") - builder.setMessageManifest(ByteString.copyFromUtf8(manifest)) - case _ ⇒ - if (serializer.includeManifest) - builder.setMessageManifest(ByteString.copyFromUtf8(payload.getClass.getName)) - } + private def serializeStatusSuccess(success: Status.Success): Array[Byte] = + payloadSupport.payloadBuilder(success.status).build().toByteArray + + private def serializeStatusFailure(failure: Status.Failure): Array[Byte] = + payloadSupport.payloadBuilder(failure.cause).build().toByteArray + + private def serializeActorInitializationException(ex: ActorInitializationException): Array[Byte] = { + val builder = ContainerFormats.ActorInitializationException.newBuilder() + if (ex.getActor ne null) + builder.setActor(actorRefBuilder(ex.getActor)) builder + .setMessage(ex.getMessage) + .setCause(payloadSupport.payloadBuilder(ex.getCause)) + .build().toByteArray } private val IdentifyManifest = "A" - private val ActorIdentifyManifest = "B" + private val ActorIdentityManifest = "B" private val OptionManifest = "C" + private val StatusSuccessManifest = "D" + private val StatusFailureManifest = "E" + private val ThrowableManifest = "F" + private val ActorRefManifest = "G" + private val PoisonPillManifest = "P" + private val KillManifest = "K" + private val RemoteWatcherHBManifest = "RWHB" + private val RemoteWatcherHBRespManifest = "RWHR" + private val ActorInitializationExceptionManifest = "AIEX" private val fromBinaryMap = Map[String, Array[Byte] ⇒ AnyRef]( IdentifyManifest → deserializeIdentify, - ActorIdentifyManifest → deserializeActorIdentity, - OptionManifest → deserializeOption + ActorIdentityManifest → deserializeActorIdentity, + OptionManifest → deserializeOption, + StatusSuccessManifest → deserializeStatusSuccess, + StatusFailureManifest → deserializeStatusFailure, + ThrowableManifest → throwableSupport.deserializeThrowable, + ActorRefManifest → deserializeActorRefBytes, + OptionManifest → deserializeOption, + PoisonPillManifest → ((_) ⇒ PoisonPill), + KillManifest → ((_) ⇒ Kill), + RemoteWatcherHBManifest → ((_) ⇒ RemoteWatcher.Heartbeat), + RemoteWatcherHBRespManifest → deserializeHeartbeatRsp, + ActorInitializationExceptionManifest → deserializeActorInitializationException ) override def manifest(o: AnyRef): String = o match { - case _: Identify ⇒ IdentifyManifest - case _: ActorIdentity ⇒ ActorIdentifyManifest - case _: Option[Any] ⇒ OptionManifest + case _: Identify ⇒ IdentifyManifest + case _: ActorIdentity ⇒ ActorIdentityManifest + case _: Option[Any] ⇒ OptionManifest + case _: ActorRef ⇒ ActorRefManifest + case _: Status.Success ⇒ StatusSuccessManifest + case _: Status.Failure ⇒ StatusFailureManifest + case _: ActorInitializationException ⇒ ActorInitializationExceptionManifest + case _: Throwable ⇒ ThrowableManifest + case PoisonPill ⇒ PoisonPillManifest + case Kill ⇒ KillManifest + case RemoteWatcher.Heartbeat ⇒ RemoteWatcherHBManifest + case _: RemoteWatcher.HeartbeatRsp ⇒ RemoteWatcherHBRespManifest case _ ⇒ throw new IllegalArgumentException(s"Can't serialize object of type ${o.getClass} in [${getClass.getName}]") } @@ -102,13 +144,13 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW private def deserializeIdentify(bytes: Array[Byte]): Identify = { val identifyProto = ContainerFormats.Identify.parseFrom(bytes) - val messageId = deserializePayload(identifyProto.getMessageId) + val messageId = payloadSupport.deserializePayload(identifyProto.getMessageId) Identify(messageId) } private def deserializeActorIdentity(bytes: Array[Byte]): ActorIdentity = { val actorIdentityProto = ContainerFormats.ActorIdentity.parseFrom(bytes) - val correlationId = deserializePayload(actorIdentityProto.getCorrelationId) + val correlationId = payloadSupport.deserializePayload(actorIdentityProto.getCorrelationId) val actorRef = if (actorIdentityProto.hasRef) Some(deserializeActorRef(actorIdentityProto.getRef)) @@ -117,6 +159,9 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW ActorIdentity(correlationId, actorRef) } + private def deserializeActorRefBytes(bytes: Array[Byte]): ActorRef = + deserializeActorRef(ContainerFormats.ActorRef.parseFrom(bytes)) + private def deserializeActorRef(actorRef: ContainerFormats.ActorRef): ActorRef = serialization.system.provider.resolveActorRef(actorRef.getPath) @@ -125,16 +170,35 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW None else { val optionProto = ContainerFormats.Option.parseFrom(bytes) - Some(deserializePayload(optionProto.getValue)) + Some(payloadSupport.deserializePayload(optionProto.getValue)) } } - private def deserializePayload(payload: ContainerFormats.Payload): Any = { - val manifest = if (payload.hasMessageManifest) payload.getMessageManifest.toStringUtf8 else "" - serialization.deserialize( - payload.getEnclosedMessage.toByteArray, - payload.getSerializerId, - manifest).get + private def deserializeStatusSuccess(bytes: Array[Byte]): Status.Success = + Status.Success(payloadSupport.deserializePayload(ContainerFormats.Payload.parseFrom(bytes))) + + private def deserializeStatusFailure(bytes: Array[Byte]): Status.Failure = + Status.Failure(payloadSupport.deserializePayload(ContainerFormats.Payload.parseFrom(bytes)).asInstanceOf[Throwable]) + + private def deserializeHeartbeatRsp(bytes: Array[Byte]): RemoteWatcher.HeartbeatRsp = { + RemoteWatcher.HeartbeatRsp(ContainerFormats.WatcherHeartbeatResponse.parseFrom(bytes).getUid.toInt) + } + + private def deserializeActorInitializationException(bytes: Array[Byte]): ActorInitializationException = { + val serializedEx = ContainerFormats.ActorInitializationException.parseFrom(bytes) + val ref = deserializeActorRef(serializedEx.getActor) + val refString = ref.path.toString + val message = serializedEx.getMessage + + val reconstructedMessage = + if (message.startsWith(refString)) message.drop(refString.length + 2) + else message + + ActorInitializationException( + if (serializedEx.hasActor) ref else null, + reconstructedMessage, + payloadSupport.deserializePayload(serializedEx.getCause).asInstanceOf[Throwable] + ) } } diff --git a/akka-remote/src/main/scala/akka/remote/serialization/PrimitiveSerializers.scala b/akka-remote/src/main/scala/akka/remote/serialization/PrimitiveSerializers.scala new file mode 100644 index 0000000000..4b71fd4d12 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/serialization/PrimitiveSerializers.scala @@ -0,0 +1,118 @@ +package akka.remote.serialization + +import java.nio.{ BufferOverflowException, ByteBuffer } + +import akka.actor.{ ExtendedActorSystem, Kill, PoisonPill } +import akka.remote.OversizedPayloadException +import akka.serialization.{ BaseSerializer, ByteBufferSerializer } +import akka.util.ByteString + +class LongSerializer(val system: ExtendedActorSystem) extends BaseSerializer with ByteBufferSerializer { + override def includeManifest: Boolean = false + + override def toBinary(o: AnyRef, buf: ByteBuffer): Unit = { + buf.putLong(Long.unbox(o)) + } + + override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = { + Long.box(buf.getLong) + } + + override def toBinary(o: AnyRef): Array[Byte] = { + val result = Array.ofDim[Byte](8) + var long = Long.unbox(o) + var i = 0 + while (long != 0) { + result(i) = (long & 0xFF).toByte + i += 1 + long >>>= 8 + } + result + } + + override def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef = { + var result = 0L + var i = 7 + while (i >= 0) { + result <<= 8 + result |= (bytes(i).toLong & 0xFF) + i -= 1 + } + Long.box(result) + } +} + +class IntSerializer(val system: ExtendedActorSystem) extends BaseSerializer with ByteBufferSerializer { + override def includeManifest: Boolean = false + + override def toBinary(o: AnyRef, buf: ByteBuffer): Unit = buf.putInt(Int.unbox(o)) + + override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = Int.box(buf.getInt) + + override def toBinary(o: AnyRef): Array[Byte] = { + val result = Array.ofDim[Byte](4) + var int = Int.unbox(o) + var i = 0 + while (int != 0) { + result(i) = (int & 0xFF).toByte + i += 1 + int >>>= 8 + } + result + } + + override def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef = { + var result = 0 + var i = 3 + while (i >= 0) { + result <<= 8 + result |= (bytes(i).toInt & 0xFF) + i -= 1 + } + Int.box(result) + } +} + +class StringSerializer(val system: ExtendedActorSystem) extends BaseSerializer with ByteBufferSerializer { + override def includeManifest: Boolean = false + + override def toBinary(o: AnyRef, buf: ByteBuffer): Unit = buf.put(toBinary(o)) + + override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = { + val bytes = Array.ofDim[Byte](buf.remaining()) + buf.get(bytes) + new String(bytes, "UTF-8") + } + + override def toBinary(o: AnyRef): Array[Byte] = o.asInstanceOf[String].getBytes("UTF-8") + + override def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef = new String(bytes, "UTF-8") + +} + +class ByteStringSerializer(val system: ExtendedActorSystem) extends BaseSerializer with ByteBufferSerializer { + override def includeManifest: Boolean = false + + override def toBinary(o: AnyRef, buf: ByteBuffer): Unit = { + val bs = o.asInstanceOf[ByteString] + + // ByteString.copyToBuffer does not throw BufferOverflowException + if (bs.copyToBuffer(buf) < bs.length) + throw new BufferOverflowException() + } + + override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = + ByteString.fromByteBuffer(buf) + + override def toBinary(o: AnyRef): Array[Byte] = { + val bs = o.asInstanceOf[ByteString] + val result = Array.ofDim[Byte](bs.length) + bs.copyToArray(result, 0, bs.length) + result + } + + override def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef = { + ByteString(bytes) + } + +} diff --git a/akka-remote/src/main/scala/akka/remote/serialization/SystemMessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/serialization/SystemMessageSerializer.scala new file mode 100644 index 0000000000..9ba4d423cc --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/serialization/SystemMessageSerializer.scala @@ -0,0 +1,161 @@ +package akka.remote.serialization + +import akka.actor.{ ActorInitializationException, ActorRef, ExtendedActorSystem, InternalActorRef } +import akka.dispatch.sysmsg._ +import akka.remote.{ ContainerFormats, SystemMessageFormats } +import akka.serialization.{ BaseSerializer, Serialization, SerializationExtension } + +class SystemMessageSerializer(val system: ExtendedActorSystem) extends BaseSerializer { + import SystemMessageFormats.SystemMessage.Type._ + + // WARNING! This must lazy otherwise it will deadlock the ActorSystem creation + private lazy val serialization = SerializationExtension(system) + private val payloadSupport = new WrappedPayloadSupport(system) + + override def includeManifest: Boolean = false + + override def toBinary(o: AnyRef): Array[Byte] = { + val builder = SystemMessageFormats.SystemMessage.newBuilder() + + o.asInstanceOf[SystemMessage] match { + case Create(failure) ⇒ + builder.setType(CREATE) + failure match { + case Some(throwable) ⇒ builder.setCauseData(serializeThrowable(throwable)) + case None ⇒ // Nothing to set + } + + case Recreate(cause) ⇒ + builder.setType(RECREATE) + builder.setCauseData(serializeThrowable(cause)) + + case Suspend() ⇒ + builder.setType(SUSPEND) + + case Resume(cause) ⇒ + builder.setType(RESUME) + builder.setCauseData(serializeThrowable(cause)) + + case Terminate() ⇒ + builder.setType(TERMINATE) + + case Supervise(child, async) ⇒ + builder.setType(SUPERVISE) + val superviseData = SystemMessageFormats.SuperviseData.newBuilder() + .setChild(serializeActorRef(child)) + .setAsync(async) + builder.setSuperviseData(superviseData) + + case Watch(watchee, watcher) ⇒ + builder.setType(WATCH) + val watchData = SystemMessageFormats.WatchData.newBuilder() + .setWatchee(serializeActorRef(watchee)) + .setWatcher(serializeActorRef(watcher)) + builder.setWatchData(watchData) + + case Unwatch(watchee, watcher) ⇒ + builder.setType(UNWATCH) + val watchData = SystemMessageFormats.WatchData.newBuilder() + .setWatchee(serializeActorRef(watchee)) + .setWatcher(serializeActorRef(watcher)) + builder.setWatchData(watchData) + + case Failed(child, cause, uid) ⇒ + builder.setType(FAILED) + val failedData = SystemMessageFormats.FailedData.newBuilder() + .setChild(serializeActorRef(child)) + .setUid(uid) + builder.setCauseData(serializeThrowable(cause)) + builder.setFailedData(failedData) + + case DeathWatchNotification(actor, existenceConfirmed, addressTerminated) ⇒ + builder.setType(DEATHWATCH_NOTIFICATION) + val deathWatchNotificationData = SystemMessageFormats.DeathWatchNotificationData.newBuilder() + .setActor(serializeActorRef(actor)) + .setExistenceConfirmed(existenceConfirmed) + .setAddressTerminated(addressTerminated) + builder.setDwNotificationData(deathWatchNotificationData) + + case NoMessage ⇒ + throw new IllegalArgumentException("NoMessage should never be serialized or deserialized") + } + + builder.build().toByteArray + } + + override def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef = { + deserializeSystemMessage(SystemMessageFormats.SystemMessage.parseFrom(bytes)) + } + + private def deserializeSystemMessage(sysmsg: SystemMessageFormats.SystemMessage): SystemMessage = + sysmsg.getType match { + case CREATE ⇒ + val cause = + if (sysmsg.hasCauseData) + Some(getCauseThrowable(sysmsg).asInstanceOf[ActorInitializationException]) + else + None + + Create(cause) + + case RECREATE ⇒ + Recreate(getCauseThrowable(sysmsg)) + + case SUSPEND ⇒ + // WARNING!! Must always create a new instance! + Suspend() + + case RESUME ⇒ + Resume(getCauseThrowable(sysmsg)) + + case TERMINATE ⇒ + // WARNING!! Must always create a new instance! + Terminate() + + case SUPERVISE ⇒ + Supervise(deserializeActorRef(sysmsg.getSuperviseData.getChild), sysmsg.getSuperviseData.getAsync) + + case WATCH ⇒ + Watch( + deserializeActorRef(sysmsg.getWatchData.getWatchee).asInstanceOf[InternalActorRef], + deserializeActorRef(sysmsg.getWatchData.getWatcher).asInstanceOf[InternalActorRef] + ) + + case UNWATCH ⇒ + Unwatch( + deserializeActorRef(sysmsg.getWatchData.getWatchee).asInstanceOf[InternalActorRef], + deserializeActorRef(sysmsg.getWatchData.getWatcher).asInstanceOf[InternalActorRef] + ) + + case FAILED ⇒ + Failed( + deserializeActorRef(sysmsg.getFailedData.getChild), + getCauseThrowable(sysmsg), + sysmsg.getFailedData.getUid.toInt) + + case DEATHWATCH_NOTIFICATION ⇒ + DeathWatchNotification( + deserializeActorRef(sysmsg.getDwNotificationData.getActor), + sysmsg.getDwNotificationData.getExistenceConfirmed, + sysmsg.getDwNotificationData.getAddressTerminated + ) + } + + private def serializeThrowable(throwable: Throwable): ContainerFormats.Payload.Builder = { + payloadSupport.payloadBuilder(throwable) + } + + private def getCauseThrowable(msg: SystemMessageFormats.SystemMessage): Throwable = { + payloadSupport.deserializePayload(msg.getCauseData).asInstanceOf[Throwable] + } + + private def serializeActorRef(actorRef: ActorRef): ContainerFormats.ActorRef.Builder = { + ContainerFormats.ActorRef.newBuilder() + .setPath(Serialization.serializedActorPath(actorRef)) + } + + private def deserializeActorRef(serializedRef: ContainerFormats.ActorRef): ActorRef = { + serialization.system.provider.resolveActorRef(serializedRef.getPath) + } + +} diff --git a/akka-remote/src/main/scala/akka/remote/serialization/ThrowableSupport.scala b/akka-remote/src/main/scala/akka/remote/serialization/ThrowableSupport.scala new file mode 100644 index 0000000000..6d8a276af2 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/serialization/ThrowableSupport.scala @@ -0,0 +1,80 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.serialization + +import akka.actor.ExtendedActorSystem +import akka.remote.ContainerFormats +import akka.serialization.SerializationExtension + +/** + * INTERNAL API + */ +private[akka] class ThrowableSupport(system: ExtendedActorSystem) { + + private lazy val serialization = SerializationExtension(system) + private val payloadSupport = new WrappedPayloadSupport(system) + + def serializeThrowable(t: Throwable): Array[Byte] = { + toProtobufThrowable(t).build().toByteArray + } + + def toProtobufThrowable(t: Throwable): ContainerFormats.Throwable.Builder = { + val b = ContainerFormats.Throwable.newBuilder() + .setClassName(t.getClass.getName) + if (t.getMessage != null) + b.setMessage(t.getMessage) + if (t.getCause != null) + b.setCause(payloadSupport.payloadBuilder(t.getCause)) + val stackTrace = t.getStackTrace + if (stackTrace != null) { + var i = 0 + while (i < stackTrace.length) { + b.addStackTrace(stackTraceElementBuilder(stackTrace(i))) + i += 1 + } + } + + b + } + + def stackTraceElementBuilder(elem: StackTraceElement): ContainerFormats.StackTraceElement.Builder = { + ContainerFormats.StackTraceElement.newBuilder() + .setClassName(elem.getClassName) + .setMethodName(elem.getMethodName) + .setFileName(elem.getFileName) + .setLineNumber(elem.getLineNumber) + } + + def deserializeThrowable(bytes: Array[Byte]): Throwable = { + fromProtobufThrowable(ContainerFormats.Throwable.parseFrom(bytes)) + } + + def fromProtobufThrowable(protoT: ContainerFormats.Throwable): Throwable = { + val t: Throwable = + if (protoT.hasCause) { + val cause = payloadSupport.deserializePayload(protoT.getCause).asInstanceOf[Throwable] + system.dynamicAccess.createInstanceFor[Throwable]( + protoT.getClassName, + List(classOf[String] → protoT.getMessage, classOf[Throwable] → cause)).get + } else { + // Important security note: before creating an instance of from the class name we + // check that the class is a Throwable and that it has a configured serializer. + val clazz = system.dynamicAccess.getClassFor[Throwable](protoT.getClassName).get + serialization.serializerFor(clazz) // this will throw NotSerializableException if no serializer configured + + system.dynamicAccess.createInstanceFor[Throwable]( + clazz, + List(classOf[String] → protoT.getMessage)).get + } + + import scala.collection.JavaConverters._ + val stackTrace = + protoT.getStackTraceList.asScala.map { elem ⇒ + new StackTraceElement(elem.getClassName, elem.getMethodName, elem.getFileName, elem.getLineNumber) + }.toArray + t.setStackTrace(stackTrace) + t + } + +} diff --git a/akka-remote/src/main/scala/akka/remote/serialization/WrappedPayloadSupport.scala b/akka-remote/src/main/scala/akka/remote/serialization/WrappedPayloadSupport.scala new file mode 100644 index 0000000000..738c4eb6eb --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/serialization/WrappedPayloadSupport.scala @@ -0,0 +1,49 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.serialization + +import akka.actor.ExtendedActorSystem +import akka.remote.ContainerFormats +import akka.serialization.SerializationExtension +import akka.serialization.SerializerWithStringManifest +import akka.protobuf.ByteString + +/** + * INTERNAL API + */ +private[akka] class WrappedPayloadSupport(system: ExtendedActorSystem) { + + private lazy val serialization = SerializationExtension(system) + + def payloadBuilder(input: Any): ContainerFormats.Payload.Builder = { + val payload = input.asInstanceOf[AnyRef] + val builder = ContainerFormats.Payload.newBuilder() + val serializer = serialization.findSerializerFor(payload) + + builder + .setEnclosedMessage(ByteString.copyFrom(serializer.toBinary(payload))) + .setSerializerId(serializer.identifier) + + serializer match { + case ser2: SerializerWithStringManifest ⇒ + val manifest = ser2.manifest(payload) + if (manifest != "") + builder.setMessageManifest(ByteString.copyFromUtf8(manifest)) + case _ ⇒ + if (serializer.includeManifest) + builder.setMessageManifest(ByteString.copyFromUtf8(payload.getClass.getName)) + } + + builder + } + + def deserializePayload(payload: ContainerFormats.Payload): Any = { + val manifest = if (payload.hasMessageManifest) payload.getMessageManifest.toStringUtf8 else "" + serialization.deserialize( + payload.getEnclosedMessage.toByteArray, + payload.getSerializerId, + manifest).get + } + +} diff --git a/akka-remote/src/main/scala/akka/remote/transport/AkkaPduCodec.scala b/akka-remote/src/main/scala/akka/remote/transport/AkkaPduCodec.scala index ed8e3d5ad7..6a209512f8 100644 --- a/akka-remote/src/main/scala/akka/remote/transport/AkkaPduCodec.scala +++ b/akka-remote/src/main/scala/akka/remote/transport/AkkaPduCodec.scala @@ -10,6 +10,7 @@ import akka.remote._ import akka.util.ByteString import akka.protobuf.InvalidProtocolBufferException import akka.protobuf.{ ByteString ⇒ PByteString } +import akka.util.OptionVal /** * INTERNAL API @@ -38,7 +39,7 @@ private[remote] object AkkaPduCodec { recipient: InternalActorRef, recipientAddress: Address, serializedMessage: SerializedMessage, - senderOption: Option[ActorRef], + senderOption: OptionVal[ActorRef], seqOption: Option[SeqNo]) extends HasSequenceNumber { def reliableDeliveryEnabled = seqOption.isDefined @@ -97,9 +98,9 @@ private[remote] trait AkkaPduCodec { localAddress: Address, recipient: ActorRef, serializedMessage: SerializedMessage, - senderOption: Option[ActorRef], - seqOption: Option[SeqNo] = None, - ackOption: Option[Ack] = None): ByteString + senderOption: OptionVal[ActorRef], + seqOption: Option[SeqNo] = None, + ackOption: Option[Ack] = None): ByteString def constructPureAck(ack: Ack): ByteString } @@ -121,16 +122,20 @@ private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec { localAddress: Address, recipient: ActorRef, serializedMessage: SerializedMessage, - senderOption: Option[ActorRef], - seqOption: Option[SeqNo] = None, - ackOption: Option[Ack] = None): ByteString = { + senderOption: OptionVal[ActorRef], + seqOption: Option[SeqNo] = None, + ackOption: Option[Ack] = None): ByteString = { val ackAndEnvelopeBuilder = AckAndEnvelopeContainer.newBuilder val envelopeBuilder = RemoteEnvelope.newBuilder envelopeBuilder.setRecipient(serializeActorRef(recipient.path.address, recipient)) - senderOption foreach { ref ⇒ envelopeBuilder.setSender(serializeActorRef(localAddress, ref)) } + senderOption match { + case OptionVal.Some(sender) ⇒ envelopeBuilder.setSender(serializeActorRef(localAddress, sender)) + case OptionVal.None ⇒ + } + seqOption foreach { seq ⇒ envelopeBuilder.setSeq(seq.rawValue) } ackOption foreach { ack ⇒ ackAndEnvelopeBuilder.setAck(ackBuilder(ack)) } envelopeBuilder.setMessage(serializedMessage) @@ -146,7 +151,7 @@ private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec { ByteString.ByteString1C(AkkaProtocolMessage.newBuilder().setPayload(PByteString.copyFrom(payload.asByteBuffer)).build.toByteArray) //Reuse Byte Array (naughty!) override def constructAssociate(info: HandshakeInfo): ByteString = { - val handshakeInfo = AkkaHandshakeInfo.newBuilder.setOrigin(serializeAddress(info.origin)).setUid(info.uid) + val handshakeInfo = AkkaHandshakeInfo.newBuilder.setOrigin(serializeAddress(info.origin)).setUid(info.uid.toLong) info.cookie foreach handshakeInfo.setCookie constructControlMessagePdu(WireFormats.CommandType.ASSOCIATE, Some(handshakeInfo)) } @@ -193,8 +198,8 @@ private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec { recipientAddress = AddressFromURIString(msgPdu.getRecipient.getPath), serializedMessage = msgPdu.getMessage, senderOption = - if (msgPdu.hasSender) Some(provider.resolveActorRefWithLocalAddress(msgPdu.getSender.getPath, localAddress)) - else None, + if (msgPdu.hasSender) OptionVal(provider.resolveActorRefWithLocalAddress(msgPdu.getSender.getPath, localAddress)) + else OptionVal.None, seqOption = if (msgPdu.hasSeq) Some(SeqNo(msgPdu.getSeq)) else None)) } else None diff --git a/akka-remote/src/test/java/akka/remote/artery/AeronStat.java b/akka-remote/src/test/java/akka/remote/artery/AeronStat.java new file mode 100644 index 0000000000..3b63943f12 --- /dev/null +++ b/akka-remote/src/test/java/akka/remote/artery/AeronStat.java @@ -0,0 +1,288 @@ +/* + * Copyright 2014 - 2016 Real Logic Ltd. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package akka.remote.artery; + +import java.io.File; +import java.io.PrintStream; +import java.nio.MappedByteBuffer; +import java.util.Date; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; +import java.util.regex.Pattern; + +import io.aeron.CncFileDescriptor; +import io.aeron.CommonContext; +import org.agrona.DirectBuffer; +import org.agrona.IoUtil; +import org.agrona.concurrent.status.CountersReader; +import org.agrona.concurrent.SigInt; + +import static io.aeron.CncFileDescriptor.*; +import static io.aeron.driver.status.StreamPositionCounter.*; +import static io.aeron.driver.status.PublisherLimit.PUBLISHER_LIMIT_TYPE_ID; +import static io.aeron.driver.status.SubscriberPos.SUBSCRIBER_POSITION_TYPE_ID; +import static io.aeron.driver.status.SystemCounterDescriptor.SYSTEM_COUNTER_TYPE_ID; + +/** + * Tool for printing out Aeron counters. A command-and-control (cnc) file is maintained by media driver + * in shared memory. This application reads the the cnc file and prints the counters. Layout of the cnc file is + * described in {@link CncFileDescriptor}. + * + * This tool accepts filters on the command line, e.g. for connections only see example below: + * + * + * java -cp aeron-samples/build/libs/samples.jar io.aeron.samples.AeronStat type=[1-4] identity=12345 + * + */ +public class AeronStat +{ + /** + * Types of the counters. + *
    + *
  • 0: System Counters
  • + *
  • 1 - 4: Stream Positions
  • + *
+ */ + private static final String COUNTER_TYPE_ID = "type"; + + /** + * The identity of each counter that can either be the system counter id or registration id for positions. + */ + private static final String COUNTER_IDENTITY = "identity"; + + /** + * Session id filter to be used for position counters. + */ + private static final String COUNTER_SESSION_ID = "session"; + + /** + * Stream id filter to be used for position counters. + */ + private static final String COUNTER_STREAM_ID = "stream"; + + /** + * Channel filter to be used for position counters. + */ + private static final String COUNTER_CHANNEL = "channel"; + + private static final int ONE_SECOND = 1_000; + + private final CountersReader counters; + private final Pattern typeFilter; + private final Pattern identityFilter; + private final Pattern sessionFilter; + private final Pattern streamFilter; + private final Pattern channelFilter; + + public AeronStat( + final CountersReader counters, + final Pattern typeFilter, + final Pattern identityFilter, + final Pattern sessionFilter, + final Pattern streamFilter, + final Pattern channelFilter) + { + this.counters = counters; + this.typeFilter = typeFilter; + this.identityFilter = identityFilter; + this.sessionFilter = sessionFilter; + this.streamFilter = streamFilter; + this.channelFilter = channelFilter; + } + + public AeronStat(final CountersReader counters) + { + this.counters = counters; + this.typeFilter = null; + this.identityFilter = null; + this.sessionFilter = null; + this.streamFilter = null; + this.channelFilter = null; + } + + public static CountersReader mapCounters() + { + return mapCounters(CommonContext.newDefaultCncFile()); + } + + public static CountersReader mapCounters(final MappedByteBuffer cncByteBuffer) + { + final DirectBuffer cncMetaData = createMetaDataBuffer(cncByteBuffer); + final int cncVersion = cncMetaData.getInt(cncVersionOffset(0)); + + if (CncFileDescriptor.CNC_VERSION != cncVersion) + { + throw new IllegalStateException("CnC version not supported: file version=" + cncVersion); + } + + return new CountersReader( + createCountersMetaDataBuffer(cncByteBuffer, cncMetaData), + createCountersValuesBuffer(cncByteBuffer, cncMetaData)); + } + + public static CountersReader mapCounters(final File cncFile) + { + System.out.println("Command `n Control file " + cncFile); + + final MappedByteBuffer cncByteBuffer = IoUtil.mapExistingFile(cncFile, "cnc"); + final DirectBuffer cncMetaData = createMetaDataBuffer(cncByteBuffer); + final int cncVersion = cncMetaData.getInt(cncVersionOffset(0)); + + if (CncFileDescriptor.CNC_VERSION != cncVersion) + { + throw new IllegalStateException("CnC version not supported: file version=" + cncVersion); + } + + return new CountersReader( + createCountersMetaDataBuffer(cncByteBuffer, cncMetaData), + createCountersValuesBuffer(cncByteBuffer, cncMetaData)); + } + + public static void main(final String[] args) throws Exception + { + Pattern typeFilter = null; + Pattern identityFilter = null; + Pattern sessionFilter = null; + Pattern streamFilter = null; + Pattern channelFilter = null; + + if (0 != args.length) + { + checkForHelp(args); + + for (final String arg : args) + { + final int equalsIndex = arg.indexOf('='); + if (-1 == equalsIndex) + { + System.out.println("Arguments must be in name=pattern format: Invalid '" + arg + "'"); + return; + } + + final String argName = arg.substring(0, equalsIndex); + final String argValue = arg.substring(equalsIndex + 1); + + switch (argName) + { + case COUNTER_TYPE_ID: + typeFilter = Pattern.compile(argValue); + break; + + case COUNTER_IDENTITY: + identityFilter = Pattern.compile(argValue); + break; + + case COUNTER_SESSION_ID: + sessionFilter = Pattern.compile(argValue); + break; + + case COUNTER_STREAM_ID: + streamFilter = Pattern.compile(argValue); + break; + + case COUNTER_CHANNEL: + channelFilter = Pattern.compile(argValue); + break; + + default: + System.out.println("Unrecognised argument: '" + arg + "'"); + return; + } + } + } + + final AeronStat aeronStat = new AeronStat( + mapCounters(), typeFilter, identityFilter, sessionFilter, streamFilter, channelFilter); + final AtomicBoolean running = new AtomicBoolean(true); + SigInt.register(() -> running.set(false)); + + while (running.get()) + { + System.out.print("\033[H\033[2J"); + + System.out.format("%1$tH:%1$tM:%1$tS - Aeron Stat%n", new Date()); + System.out.println("========================="); + + aeronStat.print(System.out); + System.out.println("--"); + + Thread.sleep(ONE_SECOND); + } + } + + public void print(final PrintStream out) + { + counters.forEach( + (counterId, typeId, keyBuffer, label) -> + { + if (filter(typeId, keyBuffer)) + { + final long value = counters.getCounterValue(counterId); + out.format("%3d: %,20d - %s%n", counterId, value, label); + } + }); + } + + private static void checkForHelp(final String[] args) + { + for (final String arg : args) + { + if ("-?".equals(arg) || "-h".equals(arg) || "-help".equals(arg)) + { + System.out.println( + "Usage: [-Daeron.dir=] AeronStat%n" + + "\tfilter by optional regex patterns:%n" + + "\t[type=]%n" + + "\t[identity=]%n" + + "\t[sessionId=]%n" + + "\t[streamId=]%n" + + "\t[channel=]%n"); + + System.exit(0); + } + } + } + + private boolean filter(final int typeId, final DirectBuffer keyBuffer) + { + if (!match(typeFilter, () -> Integer.toString(typeId))) + { + return false; + } + + if (SYSTEM_COUNTER_TYPE_ID == typeId && !match(identityFilter, () -> Integer.toString(keyBuffer.getInt(0)))) + { + return false; + } + else if (typeId >= PUBLISHER_LIMIT_TYPE_ID && typeId <= SUBSCRIBER_POSITION_TYPE_ID) + { + if (!match(identityFilter, () -> Long.toString(keyBuffer.getLong(REGISTRATION_ID_OFFSET))) || + !match(sessionFilter, () -> Integer.toString(keyBuffer.getInt(SESSION_ID_OFFSET))) || + !match(streamFilter, () -> Integer.toString(keyBuffer.getInt(STREAM_ID_OFFSET))) || + !match(channelFilter, () -> keyBuffer.getStringUtf8(CHANNEL_OFFSET))) + { + return false; + } + } + + return true; + } + + private static boolean match(final Pattern pattern, final Supplier supplier) + { + return null == pattern || pattern.matcher(supplier.get()).find(); + } +} diff --git a/akka-remote/src/test/java/akka/remote/artery/RateReporter.java b/akka-remote/src/test/java/akka/remote/artery/RateReporter.java new file mode 100644 index 0000000000..0e455fc0ab --- /dev/null +++ b/akka-remote/src/test/java/akka/remote/artery/RateReporter.java @@ -0,0 +1,115 @@ +/* + * Copyright 2014 - 2016 Real Logic Ltd. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package akka.remote.artery; + +import java.util.concurrent.locks.LockSupport; + +/** + * Tracker and reporter of rates. + * + * Uses volatile semantics for counters. + */ +public class RateReporter implements Runnable +{ + /** + * Interface for reporting of rate information + */ + @FunctionalInterface + public interface Reporter + { + /** + * Called for a rate report. + * + * @param messagesPerSec since last report + * @param bytesPerSec since last report + * @param totalMessages since beginning of reporting + * @param totalBytes since beginning of reporting + */ + void onReport(double messagesPerSec, double bytesPerSec, long totalMessages, long totalBytes); + } + + private final long reportIntervalNs; + private final long parkNs; + private final Reporter reportingFunc; + + private volatile boolean halt = false; + private volatile long totalBytes; + private volatile long totalMessages; + private long lastTotalBytes; + private long lastTotalMessages; + private long lastTimestamp; + + /** + * Create a rate reporter with the given report interval in nanoseconds and the reporting function. + * + * @param reportInterval in nanoseconds + * @param reportingFunc to call for reporting rates + */ + public RateReporter(final long reportInterval, final Reporter reportingFunc) + { + this.reportIntervalNs = reportInterval; + this.parkNs = reportInterval; + this.reportingFunc = reportingFunc; + lastTimestamp = System.nanoTime(); + } + + /** + * Run loop for the rate reporter + */ + @Override + public void run() + { + do + { + LockSupport.parkNanos(parkNs); + + final long currentTotalMessages = totalMessages; + final long currentTotalBytes = totalBytes; + final long currentTimestamp = System.nanoTime(); + + final long timeSpanNs = currentTimestamp - lastTimestamp; + final double messagesPerSec = ((currentTotalMessages - lastTotalMessages) * reportIntervalNs) / (double)timeSpanNs; + final double bytesPerSec = ((currentTotalBytes - lastTotalBytes) * reportIntervalNs) / (double)timeSpanNs; + + reportingFunc.onReport(messagesPerSec, bytesPerSec, currentTotalMessages, currentTotalBytes); + + lastTotalBytes = currentTotalBytes; + lastTotalMessages = currentTotalMessages; + lastTimestamp = currentTimestamp; + } + while (!halt); + } + + /** + * Signal the run loop to exit. Does not block. + */ + public void halt() + { + halt = true; + } + + /** + * Tell rate reporter of number of messages and bytes received, sent, etc. + * + * @param messages received, sent, etc. + * @param bytes received, sent, etc. + */ + public void onMessage(final long messages, final long bytes) + { + totalBytes += bytes; + totalMessages += messages; + } +} \ No newline at end of file diff --git a/akka-remote/src/test/resources/aeron.properties b/akka-remote/src/test/resources/aeron.properties new file mode 100644 index 0000000000..007050a287 --- /dev/null +++ b/akka-remote/src/test/resources/aeron.properties @@ -0,0 +1,21 @@ +# External Aeron Media Driver using this properties file (loaded as classpath resource) +# can be run with: +# sbt "akka-remote/test:runMain io.aeron.driver.MediaDriver aeron.properties" + +aeron.mtu.length=16384 +aeron.socket.so_sndbuf=2097152 +aeron.socket.so_rcvbuf=2097152 +aeron.rcv.buffer.length=16384 +aeron.rcv.initial.window.length=2097152 +agrona.disable.bounds.checks=true + +aeron.threading.mode=SHARED_NETWORK + +# low latency settings +#aeron.threading.mode=DEDICATED +#aeron.sender.idle.strategy=org.agrona.concurrent.BusySpinIdleStrategy +#aeron.receiver.idle.strategy=org.agrona.concurrent.BusySpinIdleStrategy + +# use same director in akka.remote.artery.advanced.aeron-dir config +# of the Akka application +aeron.dir=target/aeron diff --git a/akka-remote/src/test/scala/akka/remote/ActorsLeakSpec.scala b/akka-remote/src/test/scala/akka/remote/ActorsLeakSpec.scala index ba38dd29ab..7012986e2d 100644 --- a/akka-remote/src/test/scala/akka/remote/ActorsLeakSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/ActorsLeakSpec.scala @@ -20,7 +20,7 @@ object ActorsLeakSpec { val config = ConfigFactory.parseString( """ - | akka.actor.provider = "akka.remote.RemoteActorRefProvider" + | akka.actor.provider = remote | #akka.loglevel = DEBUG | akka.remote.netty.tcp.applied-adapters = ["trttl"] | #akka.remote.log-lifecycle-events = on @@ -119,7 +119,7 @@ class ActorsLeakSpec extends AkkaSpec(ActorsLeakSpec.config) with ImplicitSender val beforeQuarantineActors = targets.flatMap(collectLiveActors).toSet // it must not quarantine the current connection - RARP(system).provider.transport.quarantine(remoteAddress, Some(AddressUidExtension(remoteSystem).addressUid + 1)) + RARP(system).provider.transport.quarantine(remoteAddress, Some(AddressUidExtension(remoteSystem).addressUid + 1), "test") // the message from local to remote should reuse passive inbound connection system.actorSelection(RootActorPath(remoteAddress) / "user" / "stoppable") ! Identify(1) diff --git a/akka-remote/src/test/scala/akka/remote/DaemonicSpec.scala b/akka-remote/src/test/scala/akka/remote/DaemonicSpec.scala index 70085256bb..5a8abcebf3 100644 --- a/akka-remote/src/test/scala/akka/remote/DaemonicSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/DaemonicSpec.scala @@ -32,7 +32,7 @@ class DaemonicSpec extends AkkaSpec { // create a separate actor system that we can check the threads for val daemonicSystem = ActorSystem("daemonic", ConfigFactory.parseString(""" akka.daemonic = on - akka.actor.provider = "akka.remote.RemoteActorRefProvider" + akka.actor.provider = remote akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" akka.remote.netty.tcp.port = 0 akka.log-dead-letters-during-shutdown = off diff --git a/akka-remote/src/test/scala/akka/remote/LogSourceSpec.scala b/akka-remote/src/test/scala/akka/remote/LogSourceSpec.scala index 22757a42fa..24bb7c9e5b 100644 --- a/akka-remote/src/test/scala/akka/remote/LogSourceSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/LogSourceSpec.scala @@ -25,7 +25,7 @@ object LogSourceSpec { class LogSourceSpec extends AkkaSpec( """ akka.loglevel = INFO - akka.actor.provider = "akka.remote.RemoteActorRefProvider" + akka.actor.provider = remote akka.remote.netty.tcp.port = 0 """) { diff --git a/akka-remote/src/test/scala/akka/remote/RemoteActorMailboxSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteActorMailboxSpec.scala index beba7dbfd5..9c98860c44 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteActorMailboxSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteActorMailboxSpec.scala @@ -4,7 +4,7 @@ import akka.actor.ActorMailboxSpec import com.typesafe.config.ConfigFactory class RemoteActorMailboxSpec extends ActorMailboxSpec( - ConfigFactory.parseString("""akka.actor.provider = "akka.remote.RemoteActorRefProvider""""). + ConfigFactory.parseString("""akka.actor.provider = remote"""). withFallback(ActorMailboxSpec.mailboxConf)) { } \ No newline at end of file diff --git a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala index 75f6781108..626dbf7bc2 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala @@ -13,7 +13,7 @@ import akka.remote.transport.netty.{ NettyTransportSettings, SSLSettings } class RemoteConfigSpec extends AkkaSpec( """ - akka.actor.provider = "akka.remote.RemoteActorRefProvider" + akka.actor.provider = remote akka.remote.netty.tcp.port = 0 """) { diff --git a/akka-remote/src/test/scala/akka/remote/RemoteConsistentHashingRouterSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteConsistentHashingRouterSpec.scala index a3f49a32c3..7611e850db 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteConsistentHashingRouterSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteConsistentHashingRouterSpec.scala @@ -10,7 +10,7 @@ import akka.routing.ConsistentRoutee import akka.routing.ConsistentHash class RemoteConsistentHashingRouterSpec extends AkkaSpec(""" - akka.actor.provider = "akka.remote.RemoteActorRefProvider" """) { + akka.actor.provider = remote """) { "ConsistentHashingGroup" must { diff --git a/akka-remote/src/test/scala/akka/remote/RemoteDeathWatchSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteDeathWatchSpec.scala index f3a76782b3..fcdeab7b30 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteDeathWatchSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteDeathWatchSpec.scala @@ -14,7 +14,7 @@ import akka.event.Logging.Warning class RemoteDeathWatchSpec extends AkkaSpec(ConfigFactory.parseString(""" akka { actor { - provider = "akka.remote.RemoteActorRefProvider" + provider = remote deployment { /watchers.remote = "akka.tcp://other@localhost:2666" } @@ -28,6 +28,10 @@ akka { } """)) with ImplicitSender with DefaultTimeout with DeathWatchSpec { + val protocol = + if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka" + else "akka.tcp" + val other = ActorSystem("other", ConfigFactory.parseString("akka.remote.netty.tcp.port=2666") .withFallback(system.settings.config)) @@ -49,7 +53,7 @@ akka { // pick an unused port val port = SocketUtil.temporaryServerAddress().getPort // simulate de-serialized ActorRef - val ref = rarp.resolveActorRef(s"akka.tcp://OtherSystem@localhost:$port/user/foo/bar#1752527294") + val ref = rarp.resolveActorRef(s"$protocol://OtherSystem@localhost:$port/user/foo/bar#1752527294") system.actorOf(Props(new Actor { context.watch(ref) def receive = { @@ -67,7 +71,7 @@ akka { } "receive Terminated when watched node is unknown host" in { - val path = RootActorPath(Address("akka.tcp", system.name, "unknownhost", 2552)) / "user" / "subject" + val path = RootActorPath(Address(protocol, system.name, "unknownhost", 2552)) / "user" / "subject" system.actorOf(Props(new Actor { context.watch(context.actorFor(path)) def receive = { @@ -79,7 +83,7 @@ akka { } "receive ActorIdentity(None) when identified node is unknown host" in { - val path = RootActorPath(Address("akka.tcp", system.name, "unknownhost2", 2552)) / "user" / "subject" + val path = RootActorPath(Address(protocol, system.name, "unknownhost2", 2552)) / "user" / "subject" system.actorSelection(path) ! Identify(path) expectMsg(60.seconds, ActorIdentity(path, None)) } @@ -87,7 +91,7 @@ akka { "quarantine systems after unsuccessful system message delivery if have not communicated before" in { // Synthesize an ActorRef to a remote system this one has never talked to before. // This forces ReliableDeliverySupervisor to start with unknown remote system UID. - val extinctPath = RootActorPath(Address("akka.tcp", "extinct-system", "localhost", SocketUtil.temporaryServerAddress().getPort)) / "user" / "noone" + val extinctPath = RootActorPath(Address(protocol, "extinct-system", "localhost", SocketUtil.temporaryServerAddress().getPort)) / "user" / "noone" val transport = RARP(system).provider.transport val extinctRef = new RemoteActorRef(transport, transport.localAddressForRemote(extinctPath.address), extinctPath, Nobody, props = None, deploy = None) diff --git a/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala index 8265261856..2eed9b809f 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala @@ -11,7 +11,7 @@ import akka.ConfigurationException object RemoteDeployerSpec { val deployerConf = ConfigFactory.parseString(""" - akka.actor.provider = "akka.remote.RemoteActorRefProvider" + akka.actor.provider = remote akka.actor.deployment { /service2 { router = round-robin-pool diff --git a/akka-remote/src/test/scala/akka/remote/RemoteInitErrorSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteInitErrorSpec.scala index fe86653f6b..98886e6407 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteInitErrorSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteInitErrorSpec.scala @@ -24,7 +24,7 @@ class RemoteInitErrorSpec extends FlatSpec with Matchers { """ akka { actor { - provider = "akka.remote.RemoteActorRefProvider" + provider = remote } remote { enabled-transports = ["akka.remote.netty.tcp"] diff --git a/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala index c574e94c21..43976d8aa0 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala @@ -21,7 +21,7 @@ object RemoteRouterSpec { } class RemoteRouterSpec extends AkkaSpec(""" - akka.actor.provider = "akka.remote.RemoteActorRefProvider" + akka.actor.provider = remote akka.remote.netty.tcp { hostname = localhost port = 0 @@ -45,6 +45,9 @@ class RemoteRouterSpec extends AkkaSpec(""" val port = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress.port.get val sysName = system.name + val protocol = + if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka" + else "akka.tcp" val conf = ConfigFactory.parseString( s""" akka { @@ -52,7 +55,7 @@ class RemoteRouterSpec extends AkkaSpec(""" /blub { router = round-robin-pool nr-of-instances = 2 - target.nodes = ["akka.tcp://${sysName}@localhost:${port}"] + target.nodes = ["$protocol://${sysName}@localhost:${port}"] } /elastic-blub { router = round-robin-pool @@ -60,10 +63,10 @@ class RemoteRouterSpec extends AkkaSpec(""" lower-bound = 2 upper-bound = 3 } - target.nodes = ["akka.tcp://${sysName}@localhost:${port}"] + target.nodes = ["$protocol://${sysName}@localhost:${port}"] } /remote-blub { - remote = "akka.tcp://${sysName}@localhost:${port}" + remote = "$protocol://${sysName}@localhost:${port}" router = round-robin-pool nr-of-instances = 2 } @@ -71,12 +74,12 @@ class RemoteRouterSpec extends AkkaSpec(""" remote = "akka://MasterRemoteRouterSpec" router = round-robin-pool nr-of-instances = 2 - target.nodes = ["akka.tcp://${sysName}@localhost:${port}"] + target.nodes = ["$protocol://${sysName}@localhost:${port}"] } /local-blub2 { router = round-robin-pool nr-of-instances = 4 - target.nodes = ["akka.tcp://${sysName}@localhost:${port}"] + target.nodes = ["$protocol://${sysName}@localhost:${port}"] } } }""").withFallback(system.settings.config) @@ -104,7 +107,7 @@ class RemoteRouterSpec extends AkkaSpec(""" val children = replies.toSet children should have size 2 children.map(_.parent) should have size 1 - children foreach (_.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}")) + children foreach (_.address.toString should ===(s"$protocol://${sysName}@localhost:${port}")) masterSystem.stop(router) } @@ -112,12 +115,12 @@ class RemoteRouterSpec extends AkkaSpec(""" val probe = TestProbe()(masterSystem) val router = masterSystem.actorOf(new RemoteRouterConfig( RoundRobinPool(2), - Seq(Address("akka.tcp", sysName, "localhost", port))).props(echoActorProps), "blub2") + Seq(Address(protocol, sysName, "localhost", port))).props(echoActorProps), "blub2") val replies = collectRouteePaths(probe, router, 5) val children = replies.toSet children should have size 2 children.map(_.parent) should have size 1 - children foreach (_.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}")) + children foreach (_.address.toString should ===(s"$protocol://${sysName}@localhost:${port}")) masterSystem.stop(router) } @@ -128,81 +131,81 @@ class RemoteRouterSpec extends AkkaSpec(""" val children = replies.toSet children.size should be >= 2 children.map(_.parent) should have size 1 - children foreach (_.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}")) + children foreach (_.address.toString should ===(s"$protocol://${sysName}@localhost:${port}")) masterSystem.stop(router) } "deploy remote routers based on configuration" in { val probe = TestProbe()(masterSystem) val router = masterSystem.actorOf(FromConfig.props(echoActorProps), "remote-blub") - router.path.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}") + router.path.address.toString should ===(s"$protocol://${sysName}@localhost:${port}") val replies = collectRouteePaths(probe, router, 5) val children = replies.toSet children should have size 2 val parents = children.map(_.parent) parents should have size 1 parents.head should ===(router.path) - children foreach (_.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}")) + children foreach (_.address.toString should ===(s"$protocol://${sysName}@localhost:${port}")) masterSystem.stop(router) } "deploy remote routers based on explicit deployment" in { val probe = TestProbe()(masterSystem) val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps) - .withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka.tcp://${sysName}@localhost:${port}")))), "remote-blub2") - router.path.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}") + .withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"$protocol://${sysName}@localhost:${port}")))), "remote-blub2") + router.path.address.toString should ===(s"$protocol://${sysName}@localhost:${port}") val replies = collectRouteePaths(probe, router, 5) val children = replies.toSet children should have size 2 val parents = children.map(_.parent) parents should have size 1 parents.head should ===(router.path) - children foreach (_.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}")) + children foreach (_.address.toString should ===(s"$protocol://${sysName}@localhost:${port}")) masterSystem.stop(router) } "let remote deployment be overridden by local configuration" in { val probe = TestProbe()(masterSystem) val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps) - .withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka.tcp://${sysName}@localhost:${port}")))), "local-blub") + .withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"$protocol://${sysName}@localhost:${port}")))), "local-blub") router.path.address.toString should ===("akka://MasterRemoteRouterSpec") val replies = collectRouteePaths(probe, router, 5) val children = replies.toSet children should have size 2 val parents = children.map(_.parent) parents should have size 1 - parents.head.address should ===(Address("akka.tcp", sysName, "localhost", port)) - children foreach (_.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}")) + parents.head.address should ===(Address(protocol, sysName, "localhost", port)) + children foreach (_.address.toString should ===(s"$protocol://${sysName}@localhost:${port}")) masterSystem.stop(router) } "let remote deployment router be overridden by local configuration" in { val probe = TestProbe()(masterSystem) val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps) - .withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka.tcp://${sysName}@localhost:${port}")))), "local-blub2") - router.path.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}") + .withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"$protocol://${sysName}@localhost:${port}")))), "local-blub2") + router.path.address.toString should ===(s"$protocol://${sysName}@localhost:${port}") val replies = collectRouteePaths(probe, router, 5) val children = replies.toSet children should have size 4 val parents = children.map(_.parent) parents should have size 1 parents.head should ===(router.path) - children foreach (_.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}")) + children foreach (_.address.toString should ===(s"$protocol://${sysName}@localhost:${port}")) masterSystem.stop(router) } "let remote deployment be overridden by remote configuration" in { val probe = TestProbe()(masterSystem) val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps) - .withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka.tcp://${sysName}@localhost:${port}")))), "remote-override") - router.path.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}") + .withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"$protocol://${sysName}@localhost:${port}")))), "remote-override") + router.path.address.toString should ===(s"$protocol://${sysName}@localhost:${port}") val replies = collectRouteePaths(probe, router, 5) val children = replies.toSet children should have size 4 val parents = children.map(_.parent) parents should have size 1 parents.head should ===(router.path) - children foreach (_.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}")) + children foreach (_.address.toString should ===(s"$protocol://${sysName}@localhost:${port}")) masterSystem.stop(router) } @@ -213,7 +216,7 @@ class RemoteRouterSpec extends AkkaSpec(""" } val router = masterSystem.actorOf(new RemoteRouterConfig( RoundRobinPool(1, supervisorStrategy = escalator), - Seq(Address("akka.tcp", sysName, "localhost", port))).props(Props.empty), "blub3") + Seq(Address(protocol, sysName, "localhost", port))).props(Props.empty), "blub3") router.tell(GetRoutees, probe.ref) EventFilter[ActorKilledException](occurrences = 1).intercept { diff --git a/akka-remote/src/test/scala/akka/remote/RemoteWatcherSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteWatcherSpec.scala index 23d1072c51..6be790e272 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteWatcherSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteWatcherSpec.scala @@ -37,7 +37,7 @@ object RemoteWatcherSpec { object TestRemoteWatcher { final case class AddressTerm(address: Address) - final case class Quarantined(address: Address, uid: Option[Int]) + final case class Quarantined(address: Address, uid: Option[Long]) } class TestRemoteWatcher(heartbeatExpectedResponseAfter: FiniteDuration) extends RemoteWatcher( @@ -53,7 +53,7 @@ object RemoteWatcherSpec { // that doesn't interfere with the real watch that is going on in the background context.system.eventStream.publish(TestRemoteWatcher.AddressTerm(address)) - override def quarantine(address: Address, uid: Option[Int]): Unit = { + override def quarantine(address: Address, uid: Option[Long], reason: String): Unit = { // don't quarantine in remoting, but publish a testable message context.system.eventStream.publish(TestRemoteWatcher.Quarantined(address, uid)) } @@ -66,7 +66,7 @@ class RemoteWatcherSpec extends AkkaSpec( """akka { loglevel = INFO log-dead-letters-during-shutdown = false - actor.provider = "akka.remote.RemoteActorRefProvider" + actor.provider = remote remote.netty.tcp { hostname = localhost port = 0 diff --git a/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala b/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala index 23feeac38c..252a76299e 100644 --- a/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala @@ -79,7 +79,7 @@ object RemotingSpec { } akka { - actor.provider = "akka.remote.RemoteActorRefProvider" + actor.provider = remote remote { retry-gate-closed-for = 1 s @@ -557,7 +557,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D val otherGuyRemoteTest = otherGuy.path.toSerializationFormatWithAddress(addr(otherSystem, "test")) val remoteEchoHereSsl = system.actorFor(s"akka.ssl.tcp://remote-sys@localhost:${port(remoteSystem, "ssl.tcp")}/user/echo") val proxySsl = system.actorOf(Props(classOf[Proxy], remoteEchoHereSsl, testActor), "proxy-ssl") - EventFilter.warning(start = "Error while resolving address", occurrences = 1).intercept { + EventFilter.warning(start = "Error while resolving ActorRef", occurrences = 1).intercept { proxySsl ! otherGuy expectMsg(3.seconds, ("pong", otherGuyRemoteTest)) }(otherSystem) @@ -769,11 +769,11 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D inboundHandleProbe.expectNoMsg(1.second) // Quarantine unrelated connection - RARP(thisSystem).provider.quarantine(remoteAddress, Some(-1)) + RARP(thisSystem).provider.quarantine(remoteAddress, Some(-1), "test") inboundHandleProbe.expectNoMsg(1.second) // Quarantine the connection - RARP(thisSystem).provider.quarantine(remoteAddress, Some(remoteUID)) + RARP(thisSystem).provider.quarantine(remoteAddress, Some(remoteUID.toLong), "test") // Even though the connection is stashed it will be disassociated inboundHandleProbe.expectMsgType[AssociationHandle.Disassociated] diff --git a/akka-remote/src/test/scala/akka/remote/Ticket1978CommunicationSpec.scala b/akka-remote/src/test/scala/akka/remote/Ticket1978CommunicationSpec.scala index eb25421863..78a9d1389b 100644 --- a/akka-remote/src/test/scala/akka/remote/Ticket1978CommunicationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/Ticket1978CommunicationSpec.scala @@ -26,7 +26,7 @@ object Configuration { private val keyStore = getClass.getClassLoader.getResource("keystore").getPath private val conf = """ akka { - actor.provider = "akka.remote.RemoteActorRefProvider" + actor.provider = remote test { single-expect-default = 10s filter-leeway = 10s diff --git a/akka-remote/src/test/scala/akka/remote/TypedActorRemoteDeploySpec.scala b/akka-remote/src/test/scala/akka/remote/TypedActorRemoteDeploySpec.scala index 0f080db7b6..0bbf76b43a 100644 --- a/akka-remote/src/test/scala/akka/remote/TypedActorRemoteDeploySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/TypedActorRemoteDeploySpec.scala @@ -12,7 +12,7 @@ import scala.concurrent.duration._ object TypedActorRemoteDeploySpec { val conf = ConfigFactory.parseString(""" - akka.actor.provider = "akka.remote.RemoteActorRefProvider" + akka.actor.provider = remote akka.remote.netty.tcp.port = 0 """) diff --git a/akka-remote/src/test/scala/akka/remote/UntrustedSpec.scala b/akka-remote/src/test/scala/akka/remote/UntrustedSpec.scala index d893cd93a6..b278d062ec 100644 --- a/akka-remote/src/test/scala/akka/remote/UntrustedSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/UntrustedSpec.scala @@ -60,7 +60,7 @@ object UntrustedSpec { } class UntrustedSpec extends AkkaSpec(""" -akka.actor.provider = akka.remote.RemoteActorRefProvider +akka.actor.provider = remote akka.remote.untrusted-mode = on akka.remote.trusted-selection-paths = ["/user/receptionist", ] akka.remote.netty.tcp.port = 0 @@ -70,7 +70,7 @@ akka.loglevel = DEBUG import UntrustedSpec._ val client = ActorSystem("UntrustedSpec-client", ConfigFactory.parseString(""" - akka.actor.provider = akka.remote.RemoteActorRefProvider + akka.actor.provider = remote akka.remote.netty.tcp.port = 0 """)) val addr = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress diff --git a/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala new file mode 100644 index 0000000000..feeb58b2c4 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala @@ -0,0 +1,85 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.io.File + +import scala.concurrent.Await +import scala.concurrent.duration._ +import scala.util.control.NoStackTrace + +import akka.actor.ExtendedActorSystem +import akka.remote.artery.AeronSink.GaveUpMessageException +import akka.stream.ActorMaterializer +import akka.stream.ActorMaterializerSettings +import akka.stream.scaladsl.Sink +import akka.stream.scaladsl.Source +import akka.testkit.AkkaSpec +import akka.testkit.ImplicitSender +import akka.testkit.SocketUtil +import io.aeron.Aeron +import io.aeron.driver.MediaDriver +import org.agrona.IoUtil + +class AeronSinkSpec extends AkkaSpec with ImplicitSender { + + val driver = MediaDriver.launchEmbedded() + + val aeron = { + val ctx = new Aeron.Context + ctx.aeronDirectoryName(driver.aeronDirectoryName) + Aeron.connect(ctx) + } + + val idleCpuLevel = 5 + val taskRunner = { + val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem], idleCpuLevel) + r.start() + r + } + + val pool = new EnvelopeBufferPool(1034 * 1024, 128) + + val matSettings = ActorMaterializerSettings(system).withFuzzing(true) + implicit val mat = ActorMaterializer(matSettings)(system) + + override def afterTermination(): Unit = { + taskRunner.stop() + aeron.close() + driver.close() + IoUtil.delete(new File(driver.aeronDirectoryName), true) + super.afterTermination() + } + + "AeronSink" must { + + "give up sending after given duration" in { + val port = SocketUtil.temporaryServerAddress("localhost", udp = true).getPort + val channel = s"aeron:udp?endpoint=localhost:$port" + + Source.fromGraph(new AeronSource(channel, 1, aeron, taskRunner, pool, IgnoreEventSink)) + // fail receiver stream on first message + .map(_ ⇒ throw new RuntimeException("stop") with NoStackTrace) + .runWith(Sink.ignore) + + // use large enough messages to fill up buffers + val payload = Array.ofDim[Byte](100000) + val done = Source(1 to 1000).map(_ ⇒ payload) + .map { n ⇒ + val envelope = pool.acquire() + envelope.byteBuffer.put(payload) + envelope.byteBuffer.flip() + envelope + } + .runWith(new AeronSink(channel, 1, aeron, taskRunner, pool, 500.millis, IgnoreEventSink)) + + // without the give up timeout the stream would not complete/fail + intercept[GaveUpMessageException] { + Await.result(done, 5.seconds) + } + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala new file mode 100644 index 0000000000..32afc155a7 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala @@ -0,0 +1,101 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.nio.file.{ FileSystems, Files, Path } +import java.util.UUID + +import akka.actor.{ ActorSystem, RootActorPath } +import akka.remote.RARP +import akka.testkit.AkkaSpec +import com.typesafe.config.{ Config, ConfigFactory } +import org.scalatest.Outcome + +object ArteryMultiNodeSpec { + + def defaultConfig = + ConfigFactory.parseString(s""" + akka { + actor.provider = remote + actor.warn-about-java-serializer-usage = off + remote.artery { + enabled = on + canonical { + hostname = localhost + port = 0 + } + advanced.flight-recorder { + enabled=on + destination=target/flight-recorder-${UUID.randomUUID().toString}.afr + } + } + } + """) +} + +/** + * Base class for remoting tests what needs to test interaction between a "local" actor system + * which is always created (the usual AkkaSpec system), and multiple additional actor systems over artery + */ +abstract class ArteryMultiNodeSpec(config: Config) extends AkkaSpec(config.withFallback(ArteryMultiNodeSpec.defaultConfig)) { + + def this() = this(ConfigFactory.empty()) + def this(extraConfig: String) = this(ConfigFactory.parseString(extraConfig)) + + /** just an alias to make tests more readable */ + def localSystem = system + def localPort = port(localSystem) + def port(system: ActorSystem): Int = RARP(system).provider.getDefaultAddress.port.get + def address(sys: ActorSystem) = RARP(sys).provider.getDefaultAddress + def rootActorPath(sys: ActorSystem) = RootActorPath(address(sys)) + def nextGeneratedSystemName = s"${localSystem.name}-remote-${remoteSystems.size}" + private val flightRecorderFile: Path = + FileSystems.getDefault.getPath(RARP(system).provider.remoteSettings.Artery.Advanced.FlightRecorderDestination) + + private var remoteSystems: Vector[ActorSystem] = Vector.empty + + /** + * @return A new actor system configured with artery enabled. The system will + * automatically be terminated after test is completed to avoid leaks. + */ + def newRemoteSystem(extraConfig: Option[String] = None, name: Option[String] = None): ActorSystem = { + val config = + extraConfig.fold( + localSystem.settings.config + )( + str ⇒ ConfigFactory.parseString(str).withFallback(localSystem.settings.config) + ) + + val remoteSystem = ActorSystem(name.getOrElse(nextGeneratedSystemName), config) + remoteSystems = remoteSystems :+ remoteSystem + + remoteSystem + } + + // keep track of failure so that we can print flight recorder output on failures + private var failed = false + override protected def withFixture(test: NoArgTest): Outcome = { + val out = super.withFixture(test) + if (!out.isSucceeded) failed = true + out + } + + override def afterTermination(): Unit = { + remoteSystems.foreach(sys ⇒ shutdown(sys)) + remoteSystems = Vector.empty + handleFlightRecorderFile() + } + + private def handleFlightRecorderFile(): Unit = { + if (Files.exists(flightRecorderFile)) { + if (failed) { + // logger may not be alive anymore so we have to use stdout here + println("Flight recorder dump:") + FlightRecorderReader.dumpToStdout(flightRecorderFile) + } + Files.delete(flightRecorderFile) + } + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/BindCanonicalAddressSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/BindCanonicalAddressSpec.scala new file mode 100644 index 0000000000..709959d4b3 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/BindCanonicalAddressSpec.scala @@ -0,0 +1,83 @@ +package akka.remote.artery + +import akka.testkit.AkkaSpec +import com.typesafe.config.ConfigFactory +import akka.actor.ActorSystem +import akka.remote.transport.netty.NettyTransportSpec._ +import scala.concurrent.Await +import org.scalatest.WordSpec +import org.scalatest.Matchers +import scala.concurrent.duration.Duration +import akka.testkit.SocketUtil +import java.net.InetAddress + +class BindCanonicalAddressSpec extends WordSpec with Matchers { + import BindCanonicalAddressSpec._ + + "artery" should { + + "bind to a random port" in { + val config = ConfigFactory.parseString(s""" + akka.remote.artery.canonical.port = 0 + """) + + implicit val sys = ActorSystem("sys", config.withFallback(commonConfig)) + + getInternal should contain(getExternal) + Await.result(sys.terminate(), Duration.Inf) + } + + "bind to a random port but remoting accepts from a specified port" in { + val address = SocketUtil.temporaryServerAddress(InetAddress.getLocalHost.getHostAddress, udp = true) + + val config = ConfigFactory.parseString(s""" + akka.remote.artery.canonical.port = ${address.getPort} + akka.remote.artery.bind.port = 0 + """) + + implicit val sys = ActorSystem("sys", config.withFallback(commonConfig)) + + getExternal should ===(address.toAkkaAddress("akka")) + getInternal should not contain (address.toAkkaAddress("akka")) + + Await.result(sys.terminate(), Duration.Inf) + } + + "bind to a specified port and remoting accepts from a bound port" in { + val address = SocketUtil.temporaryServerAddress(InetAddress.getLocalHost.getHostAddress, udp = true) + + val config = ConfigFactory.parseString(s""" + akka.remote.artery.canonical.port = 0 + akka.remote.artery.bind.port = ${address.getPort} + """) + + implicit val sys = ActorSystem("sys", config.withFallback(commonConfig)) + + getExternal should ===(address.toAkkaAddress("akka")) + getInternal should contain(address.toAkkaAddress("akka")) + } + + "bind to all interfaces" in { + val config = ConfigFactory.parseString(s""" + akka.remote.artery.bind.hostname = "0.0.0.0" + """) + + implicit val sys = ActorSystem("sys", config.withFallback(commonConfig)) + + getInternal.flatMap(_.port) should contain(getExternal.port.get) + getInternal.map(_.host.get should include regex "0.0.0.0".r) // regexp dot is intentional to match IPv4 and 6 addresses + + Await.result(sys.terminate(), Duration.Inf) + } + } + +} + +object BindCanonicalAddressSpec { + val commonConfig = ConfigFactory.parseString(""" + akka { + actor.provider = remote + remote.artery.enabled = true + } + """) +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala new file mode 100644 index 0000000000..bb1865762a --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala @@ -0,0 +1,211 @@ +/* + * Copyright (C) 2009-2016 Lightbend Inc. + */ + +package akka.remote.artery + +import java.nio.{ ByteBuffer, ByteOrder } + +import akka.actor._ +import akka.remote.artery.compress.{ CompressionTable, CompressionTestUtils, InboundCompressions } +import akka.serialization.Serialization +import akka.testkit.AkkaSpec +import akka.util.{ ByteString, OptionVal } + +class EnvelopeBufferSpec extends AkkaSpec { + import CompressionTestUtils._ + + object TestCompressor extends InboundCompressions { + val refToIdx: Map[ActorRef, Int] = Map( + minimalRef("compressable0") → 0, + minimalRef("compressable1") → 1, + minimalRef("reallylongcompressablestring") → 2) + val idxToRef: Map[Int, ActorRef] = refToIdx.map(_.swap) + + val serializerToIdx = Map( + "serializer0" → 0, + "serializer1" → 1) + val idxToSer = serializerToIdx.map(_.swap) + + val manifestToIdx = Map( + "manifest0" → 0, + "manifest1" → 1) + val idxToManifest = manifestToIdx.map(_.swap) + + val outboundActorRefTable: CompressionTable[ActorRef] = + CompressionTable(17L, version = 28.toByte, refToIdx) + + val outboundClassManifestTable: CompressionTable[String] = + CompressionTable(17L, version = 35.toByte, manifestToIdx) + + override def hitActorRef(originUid: Long, remote: Address, ref: ActorRef, n: Int): Unit = () + override def decompressActorRef(originUid: Long, tableVersion: Byte, idx: Int): OptionVal[ActorRef] = OptionVal(idxToRef(idx)) + override def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Byte): Unit = () + + override def hitClassManifest(originUid: Long, remote: Address, manifest: String, n: Int): Unit = () + override def decompressClassManifest(originUid: Long, tableVersion: Byte, idx: Int): OptionVal[String] = OptionVal(idxToManifest(idx)) + override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Byte): Unit = () + override def close(): Unit = () + override def close(originUid: Long): Unit = () + } + + "EnvelopeBuffer" must { + val headerOut = HeaderBuilder.in(TestCompressor) + val headerIn = HeaderBuilder.out() + + headerIn.setOutboundActorRefCompression(TestCompressor.outboundActorRefTable) + headerIn.setOutboundClassManifestCompression(TestCompressor.outboundClassManifestTable) + + val byteBuffer = ByteBuffer.allocate(1024).order(ByteOrder.LITTLE_ENDIAN) + val envelope = new EnvelopeBuffer(byteBuffer) + + val originUid = 1L + + "be able to encode and decode headers with compressed literals" in { + headerIn setVersion 1 + headerIn setUid 42 + headerIn setSerializer 4 + headerIn setRecipientActorRef minimalRef("compressable1") + headerIn setSenderActorRef minimalRef("compressable0") + + headerIn setManifest "manifest1" + + envelope.writeHeader(headerIn) + envelope.byteBuffer.position() should ===(EnvelopeBuffer.MetadataContainerAndLiteralSectionOffset) // Fully compressed header + + envelope.byteBuffer.flip() + envelope.parseHeader(headerOut) + + headerOut.version should ===(1) + headerOut.uid should ===(42) + headerOut.inboundActorRefCompressionTableVersion should ===(28.toByte) + headerOut.inboundClassManifestCompressionTableVersion should ===(35.toByte) + headerOut.serializer should ===(4) + headerOut.senderActorRef(originUid).get.path.toSerializationFormat should ===("akka://EnvelopeBufferSpec/compressable0") + headerOut.senderActorRefPath should ===(OptionVal.None) + headerOut.recipientActorRef(originUid).get.path.toSerializationFormat should ===("akka://EnvelopeBufferSpec/compressable1") + headerOut.recipientActorRefPath should ===(OptionVal.None) + headerOut.manifest(originUid).get should ===("manifest1") + } + + "be able to encode and decode headers with uncompressed literals" in { + val senderRef = minimalRef("uncompressable0") + val recipientRef = minimalRef("uncompressable11") + + headerIn setVersion 1 + headerIn setUid 42 + headerIn setSerializer 4 + headerIn setSenderActorRef senderRef + headerIn setRecipientActorRef recipientRef + headerIn setManifest "uncompressable3333" + + val expectedHeaderLength = + EnvelopeBuffer.MetadataContainerAndLiteralSectionOffset + // Constant header part + 2 + lengthOfSerializedActorRefPath(senderRef) + // Length field + literal + 2 + lengthOfSerializedActorRefPath(recipientRef) + // Length field + literal + 2 + "uncompressable3333".length // Length field + literal + + envelope.writeHeader(headerIn) + envelope.byteBuffer.position() should ===(expectedHeaderLength) + + envelope.byteBuffer.flip() + envelope.parseHeader(headerOut) + + headerOut.version should ===(1) + headerOut.uid should ===(42) + headerOut.serializer should ===(4) + headerOut.senderActorRefPath should ===(OptionVal.Some("akka://EnvelopeBufferSpec/uncompressable0")) + headerOut.senderActorRef(originUid) should ===(OptionVal.None) + headerOut.recipientActorRefPath should ===(OptionVal.Some("akka://EnvelopeBufferSpec/uncompressable11")) + headerOut.recipientActorRef(originUid) should ===(OptionVal.None) + headerOut.manifest(originUid).get should ===("uncompressable3333") + } + + "be able to encode and decode headers with mixed literals" in { + val recipientRef = minimalRef("uncompressable1") + + headerIn setVersion 1 + headerIn setUid 42 + headerIn setSerializer 4 + headerIn setSenderActorRef minimalRef("reallylongcompressablestring") + headerIn setRecipientActorRef recipientRef + headerIn setManifest "manifest1" + + envelope.writeHeader(headerIn) + envelope.byteBuffer.position() should ===( + EnvelopeBuffer.MetadataContainerAndLiteralSectionOffset + + 2 + lengthOfSerializedActorRefPath(recipientRef)) + + envelope.byteBuffer.flip() + envelope.parseHeader(headerOut) + + headerOut.version should ===(1) + headerOut.uid should ===(42) + headerOut.serializer should ===(4) + headerOut.senderActorRef(originUid).get.path.toSerializationFormat should ===("akka://EnvelopeBufferSpec/reallylongcompressablestring") + headerOut.senderActorRefPath should ===(OptionVal.None) + headerOut.recipientActorRefPath should ===(OptionVal.Some("akka://EnvelopeBufferSpec/uncompressable1")) + headerOut.recipientActorRef(originUid) should ===(OptionVal.None) + headerOut.manifest(originUid).get should ===("manifest1") + + val senderRef = minimalRef("uncompressable0") + + headerIn setVersion 3 + headerIn setUid Long.MinValue + headerIn setSerializer -1 + headerIn setSenderActorRef senderRef + headerIn setRecipientActorRef minimalRef("reallylongcompressablestring") + headerIn setManifest "longlonglongliteralmanifest" + + envelope.writeHeader(headerIn) + envelope.byteBuffer.position() should ===( + EnvelopeBuffer.MetadataContainerAndLiteralSectionOffset + + 2 + lengthOfSerializedActorRefPath(senderRef) + + 2 + "longlonglongliteralmanifest".length) + + envelope.byteBuffer.flip() + envelope.parseHeader(headerOut) + + headerOut.version should ===(3) + headerOut.uid should ===(Long.MinValue) + headerOut.serializer should ===(-1) + headerOut.senderActorRefPath should ===(OptionVal.Some("akka://EnvelopeBufferSpec/uncompressable0")) + headerOut.senderActorRef(originUid) should ===(OptionVal.None) + headerOut.recipientActorRef(originUid).get.path.toSerializationFormat should ===("akka://EnvelopeBufferSpec/reallylongcompressablestring") + headerOut.recipientActorRefPath should ===(OptionVal.None) + headerOut.manifest(originUid).get should ===("longlonglongliteralmanifest") + } + + "be able to encode and decode headers with mixed literals and payload" in { + val payload = ByteString("Hello Artery!") + + headerIn setVersion 1 + headerIn setUid 42 + headerIn setSerializer 4 + headerIn setSenderActorRef minimalRef("reallylongcompressablestring") + headerIn setRecipientActorRef minimalRef("uncompressable1") + headerIn setManifest "manifest1" + + envelope.writeHeader(headerIn) + envelope.byteBuffer.put(payload.toByteBuffer) + envelope.byteBuffer.flip() + + envelope.parseHeader(headerOut) + + headerOut.version should ===(1) + headerOut.uid should ===(42) + headerOut.serializer should ===(4) + headerOut.senderActorRef(originUid).get.path.toSerializationFormat should ===("akka://EnvelopeBufferSpec/reallylongcompressablestring") + headerOut.senderActorRefPath should ===(OptionVal.None) + headerOut.recipientActorRefPath should ===(OptionVal.Some("akka://EnvelopeBufferSpec/uncompressable1")) + headerOut.recipientActorRef(originUid) should ===(OptionVal.None) + headerOut.manifest(originUid).get should ===("manifest1") + + ByteString.fromByteBuffer(envelope.byteBuffer) should ===(payload) + } + + } + + def lengthOfSerializedActorRefPath(ref: ActorRef): Int = + Serialization.serializedActorPath(ref).length +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala new file mode 100644 index 0000000000..9f082877d5 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala @@ -0,0 +1,436 @@ +/* + * Copyright (C) 2009-2016 Lightbend Inc. + */ + +package akka.remote.artery + +import java.io.{ File, IOException, RandomAccessFile } +import java.nio.channels.FileChannel +import java.nio.file.{ Files, Path, StandardOpenOption } +import java.time.Instant +import java.util.Arrays +import java.util.concurrent.{ CountDownLatch, TimeUnit } + +import akka.testkit.AkkaSpec +import com.google.common.jimfs.{ Configuration, Jimfs } + +class FlightRecorderSpec extends AkkaSpec { + import FlightRecorderReader._ + + "Flight Recorder" must { + + "properly initialize AFR file when created" in withFlightRecorder { (recorder, reader, channel) ⇒ + channel.force(false) + val currentTime = Instant.now() + reader.rereadStructure() + + currentTime.isAfter(reader.structure.startTime) should be(true) + (currentTime.toEpochMilli - reader.structure.startTime.toEpochMilli < 3000) should be(true) + + reader.structure.alertLog.logs.size should ===(FlightRecorder.SnapshotCount) + reader.structure.loFreqLog.logs.size should ===(FlightRecorder.SnapshotCount) + reader.structure.hiFreqLog.logs.size should ===(FlightRecorder.SnapshotCount) + + def checkLogInitialized(log: reader.RollingLog): Unit = { + log.logs(0).state should ===(Live) + log.logs(0).head should ===(0) + log.logs(0).richEntries.toSeq should ===(Nil) + + log.logs(1).state should ===(Empty) + log.logs(1).head should ===(0) + log.logs(1).richEntries.toSeq should ===(Nil) + + log.logs(2).state should ===(Empty) + log.logs(2).head should ===(0) + log.logs(2).richEntries.toSeq should ===(Nil) + + log.logs(3).state should ===(Empty) + log.logs(3).head should ===(0) + log.logs(3).richEntries.toSeq should ===(Nil) + } + + checkLogInitialized(reader.structure.alertLog) + checkLogInitialized(reader.structure.loFreqLog) + checkLogInitialized(reader.structure.hiFreqLog) + } + + "properly rotate logs when snapshotting" in withFlightRecorder { (recorder, reader, channel) ⇒ + recorder.snapshot() + channel.force(false) + reader.rereadStructure() + + def checkLogRotated(log: reader.RollingLog, states: Seq[LogState]): Unit = + log.logs.zip(states).foreach { case (log, state) ⇒ log.state should ===(state) } + + checkLogRotated(reader.structure.alertLog, List(Snapshot, Live, Empty, Empty)) + checkLogRotated(reader.structure.loFreqLog, List(Snapshot, Live, Empty, Empty)) + checkLogRotated(reader.structure.hiFreqLog, List(Snapshot, Live, Empty, Empty)) + + recorder.snapshot() + reader.rereadStructure() + + checkLogRotated(reader.structure.alertLog, List(Snapshot, Snapshot, Live, Empty)) + checkLogRotated(reader.structure.loFreqLog, List(Snapshot, Snapshot, Live, Empty)) + checkLogRotated(reader.structure.hiFreqLog, List(Snapshot, Snapshot, Live, Empty)) + + recorder.snapshot() + recorder.snapshot() + reader.rereadStructure() + + checkLogRotated(reader.structure.alertLog, List(Live, Snapshot, Snapshot, Snapshot)) + checkLogRotated(reader.structure.loFreqLog, List(Live, Snapshot, Snapshot, Snapshot)) + checkLogRotated(reader.structure.hiFreqLog, List(Live, Snapshot, Snapshot, Snapshot)) + } + + "properly report zero low frequency events" in withFlightRecorder { (recorder, reader, channel) ⇒ + channel.force(false) + reader.rereadStructure() + + val entries = reader.structure.loFreqLog.logs(0).richEntries.toSeq + + entries.isEmpty should be(true) + } + + "properly report zero high frequency events" in withFlightRecorder { (recorder, reader, channel) ⇒ + channel.force(false) + reader.rereadStructure() + + val entries = reader.structure.hiFreqLog.logs(0).compactEntries.toSeq + + entries.isEmpty should be(true) + } + + "properly store one low frequency event" in withFlightRecorder { (recorder, reader, channel) ⇒ + val sink = recorder.createEventSink() + val helloBytes = "Hello".getBytes("US-ASCII") + + sink.loFreq(42, helloBytes) + channel.force(false) + + reader.rereadStructure() + val entries = reader.structure.loFreqLog.logs(0).richEntries.toSeq + + entries.exists(_.dirty) should be(false) + entries.map(_.code.toInt) should ===(List(42)) + } + + "properly store one high frequency event" in withFlightRecorder { (recorder, reader, channel) ⇒ + val sink = recorder.createEventSink() + + sink.hiFreq(42, 64) + sink.flushHiFreqBatch() + channel.force(false) + + reader.rereadStructure() + val entries = reader.structure.hiFreqLog.logs(0).compactEntries.toSeq + + entries.exists(_.dirty) should be(false) + entries.map(_.code.toInt) should ===(List(42)) + entries.map(_.param.toInt) should ===(List(64)) + } + + "properly store low frequency events" in withFlightRecorder { (recorder, reader, channel) ⇒ + val sink = recorder.createEventSink() + val helloBytes = "Hello".getBytes("US-ASCII") + + for (i ← 0 until FlightRecorder.LoFreqWindow) + sink.loFreq(i, helloBytes) + + channel.force(false) + + reader.rereadStructure() + val entries = reader.structure.loFreqLog.logs(0).richEntries.toSeq + + entries.exists(_.dirty) should be(false) + entries.map(_.code.toInt) should ===(0 until FlightRecorder.LoFreqWindow) + entries.forall(entry ⇒ Arrays.equals(entry.metadata, helloBytes)) should be(true) + + // Timestamps are monotonic + entries.sortBy(_.code) should ===(entries.sortBy(_.timeStamp)) + } + + "properly truncate low frequency event metadata if necessary" in withFlightRecorder { (recorder, reader, channel) ⇒ + val sink = recorder.createEventSink() + val longMetadata = Array.ofDim[Byte](1024) + + sink.loFreq(0, longMetadata) + channel.force(false) + + reader.rereadStructure() + val entries = reader.structure.loFreqLog.logs(0).richEntries.toSeq + + entries.size should ===(1) + entries.head.metadata should ===(Array.ofDim[Byte](FlightRecorder.LoFreqRecordSize - 32)) + + } + + "properly store high frequency events" in withFlightRecorder { (recorder, reader, channel) ⇒ + val EffectiveHighFreqWindow = FlightRecorder.HiFreqWindow * FlightRecorder.HiFreqBatchSize + val sink = recorder.createEventSink() + + for (i ← 0 until EffectiveHighFreqWindow) + sink.hiFreq(i, 42) + + sink.flushHiFreqBatch() + channel.force(false) + + reader.rereadStructure() + val entries = reader.structure.hiFreqLog.logs(0).compactEntries.toSeq + + entries.exists(_.dirty) should be(false) + entries.map(_.code.toInt) should ===(0 until EffectiveHighFreqWindow) + entries.forall(entry ⇒ entry.param == 42) should be(true) + + // Timestamps are monotonic + entries.sortBy(_.code) should ===(entries.sortBy(_.timeStamp)) + } + + "properly store and rotate low frequency events" in withFlightRecorder { (recorder, reader, channel) ⇒ + val sink = recorder.createEventSink() + val helloBytes = "Hello".getBytes("US-ASCII") + + for (i ← 0 until FlightRecorder.LoFreqWindow + 100) + sink.loFreq(i, helloBytes) + + channel.force(false) + + reader.rereadStructure() + val entries = reader.structure.loFreqLog.logs(0).richEntries.toSeq + + entries.exists(_.dirty) should be(false) + entries.map(_.code.toInt).sorted should ===(100 until (FlightRecorder.LoFreqWindow + 100)) + entries.forall(entry ⇒ Arrays.equals(entry.metadata, helloBytes)) should be(true) + + // Timestamps are monotonic + entries.sortBy(_.code) should ===(entries.sortBy(_.timeStamp)) + } + + "properly store and rotate high frequency events" in withFlightRecorder { (recorder, reader, channel) ⇒ + val EffectiveHighFreqWindow = FlightRecorder.HiFreqWindow * FlightRecorder.HiFreqBatchSize + val sink = recorder.createEventSink() + + for (i ← 0 until EffectiveHighFreqWindow + 100) + sink.hiFreq(i, 42) + + sink.flushHiFreqBatch() + channel.force(false) + + reader.rereadStructure() + val entries = reader.structure.hiFreqLog.logs(0).compactEntries.toSeq + + entries.exists(_.dirty) should be(false) + // Note the (2 * FlightRecorder.HiFreqBatchSize) initial sequence number. + // This is because the overflow by 100 events rotates out two records, not just 100. + entries.map(_.code.toInt).sorted should ===((2 * FlightRecorder.HiFreqBatchSize) until (EffectiveHighFreqWindow + 100)) + entries.forall(entry ⇒ entry.param == 42) should be(true) + + // Timestamps are monotonic + entries.sortBy(_.code) should ===(entries.sortBy(_.timeStamp)) + } + + "properly store low frequency events after snapshot" in withFlightRecorder { (recorder, reader, channel) ⇒ + val sink = recorder.createEventSink() + val helloBytes = "Hello".getBytes("US-ASCII") + val hello2Bytes = "Hello2".getBytes("US-ASCII") + + for (i ← 0 until 100) + sink.loFreq(i, helloBytes) + + recorder.snapshot() + + for (i ← 0 until 50) + sink.loFreq(i, hello2Bytes) + + reader.rereadStructure() + + reader.structure.loFreqLog.logs(0).state should ===(Snapshot) + reader.structure.loFreqLog.logs(1).state should ===(Live) + + val snapshotEntries = reader.structure.loFreqLog.logs(0).richEntries.toSeq + val liveEntries = reader.structure.loFreqLog.logs(1).richEntries.toSeq + + snapshotEntries.exists(_.dirty) should be(false) + snapshotEntries.map(_.code.toInt) should ===(0 until 100) + snapshotEntries.forall(entry ⇒ Arrays.equals(entry.metadata, helloBytes)) should be(true) + + // Timestamps are monotonic + snapshotEntries.sortBy(_.code) should ===(snapshotEntries.sortBy(_.timeStamp)) + + liveEntries.exists(_.dirty) should be(false) + liveEntries.map(_.code.toInt) should ===(0 until 50) + liveEntries.forall(entry ⇒ Arrays.equals(entry.metadata, hello2Bytes)) should be(true) + + // Timestamps are monotonic + liveEntries.sortBy(_.code) should ===(liveEntries.sortBy(_.timeStamp)) + } + + "properly store high frequency events after snapshot" in withFlightRecorder { (recorder, reader, channel) ⇒ + val sink = recorder.createEventSink() + + for (i ← 0 until 100) + sink.hiFreq(i, 0) + + sink.flushHiFreqBatch() + recorder.snapshot() + + for (i ← 0 until 50) + sink.hiFreq(i, 1) + + sink.flushHiFreqBatch() + channel.force(false) + reader.rereadStructure() + + reader.structure.hiFreqLog.logs(0).state should ===(Snapshot) + reader.structure.hiFreqLog.logs(1).state should ===(Live) + + val snapshotEntries = reader.structure.hiFreqLog.logs(0).compactEntries.toSeq + val liveEntries = reader.structure.hiFreqLog.logs(1).compactEntries.toSeq + + snapshotEntries.exists(_.dirty) should be(false) + snapshotEntries.map(_.code.toInt) should ===(0 until 100) + snapshotEntries.forall(_.param == 0) should be(true) + + // Timestamps are monotonic + snapshotEntries.sortBy(_.code) should ===(snapshotEntries.sortBy(_.timeStamp)) + + liveEntries.exists(_.dirty) should be(false) + liveEntries.map(_.code.toInt) should ===(0 until 50) + liveEntries.forall(_.param == 1) should be(true) + + // Timestamps are monotonic + liveEntries.sortBy(_.code) should ===(liveEntries.sortBy(_.timeStamp)) + } + + "properly store alerts and make a snapshot" in withFlightRecorder { (recorder, reader, channel) ⇒ + val sink = recorder.createEventSink() + val helloBytes = "Hello".getBytes("US-ASCII") + val alertBytes = "An alert".getBytes("US-ASCII") + + for (i ← 0 until 100) { + sink.hiFreq(i, 1) + sink.loFreq(i, helloBytes) + } + + sink.alert(42, alertBytes) + reader.rereadStructure() + + // Snapshot is automatically taken + reader.structure.alertLog.logs(0).state should ===(Snapshot) + reader.structure.loFreqLog.logs(0).state should ===(Snapshot) + reader.structure.hiFreqLog.logs(0).state should ===(Snapshot) + reader.structure.alertLog.logs(1).state should ===(Live) + reader.structure.loFreqLog.logs(1).state should ===(Live) + reader.structure.hiFreqLog.logs(1).state should ===(Live) + + val hiFreqEntries = reader.structure.hiFreqLog.logs(0).compactEntries.toSeq + val loFreqEntries = reader.structure.loFreqLog.logs(0).richEntries.toSeq + val alertEntries = reader.structure.alertLog.logs(0).richEntries.toSeq + + // High frequency events are flushed (100 leaves an uncomplete batch if not flushed, + // i.e. only the first batch visible if alert did not flush) + hiFreqEntries.map(_.code.toInt) should ===(0 until 100) + hiFreqEntries.forall(_.param == 1) should be(true) + loFreqEntries.map(_.code.toInt) should ===(0 until 100) + loFreqEntries.forall(entry ⇒ Arrays.equals(entry.metadata, helloBytes)) should be(true) + alertEntries.map(_.code.toInt) should ===(List(42)) + Arrays.equals(alertEntries.head.metadata, alertBytes) should be(true) + } + + "properly store events from multiple threads" in withFlightRecorder { (recorder, reader, channel) ⇒ + val Threads = 4 + val startLatch = new CountDownLatch(1) + val finishLatch = new CountDownLatch(Threads) + + for (i ← 1 to Threads) { + new Thread { + override def run(): Unit = { + val sink = recorder.createEventSink() + startLatch.await(3, TimeUnit.SECONDS) + + for (j ← 0 until 100) sink.loFreq(code = i, Array(j.toByte)) + finishLatch.countDown() + } + }.start() + } + + startLatch.countDown() + finishLatch.await(3, TimeUnit.SECONDS) + channel.force(false) + reader.rereadStructure() + + reader.structure.loFreqLog.logs(0).richEntries.size should ===(FlightRecorder.LoFreqWindow) + + for (i ← 1 to Threads) { + val entries = reader.structure.loFreqLog.logs(0).richEntries.filter(_.code == i).toSeq + + entries.exists(_.dirty) should be(false) + // Entries are consecutive for any given writer + entries.map(_.metadata(0).toInt).sorted should ===((100 - entries.size) until 100) + entries.forall(_.code == i) should be(true) + + // Timestamps are monotonic + entries.sortBy(_.metadata(0).toInt) should ===(entries.sortBy(_.timeStamp)) + } + } + + "create flight recorder file" in { + def assertFileIsSound(path: Path) = { + Files.exists(path) should ===(true) + Files.isRegularFile(path) should ===(true) + Files.isWritable(path) should ===(true) + Files.isReadable(path) should ===(true) + } + val fs = Jimfs.newFileSystem(Configuration.unix()) + + try { + val tmpPath = FlightRecorder.createFlightRecorderFile("", fs) + assertFileIsSound(tmpPath) + // this is likely in the actual file system, so lets delete it + Files.delete(tmpPath) + + Files.createDirectory(fs.getPath("/directory")) + val tmpFileInGivenPath = FlightRecorder.createFlightRecorderFile("/directory", fs) + assertFileIsSound(tmpFileInGivenPath) + + val specificFile = FlightRecorder.createFlightRecorderFile("/directory/flight-recorder.afr", fs) + assertFileIsSound(specificFile) + + } finally { + fs.close() + } + + } + + } + + private def withFlightRecorder(body: (FlightRecorder, FlightRecorderReader, FileChannel) ⇒ Unit): Unit = { + val file = File.createTempFile("artery", ".afr") + file.deleteOnExit() + + var randomAccessFile: RandomAccessFile = null + var recorder: FlightRecorder = null + var reader: FlightRecorderReader = null + var channel: FileChannel = null + + try { + randomAccessFile = new RandomAccessFile(file, "rwd") + randomAccessFile.setLength(FlightRecorder.TotalSize) + randomAccessFile.close() + + channel = FileChannel.open(file.toPath, StandardOpenOption.CREATE, StandardOpenOption.WRITE, StandardOpenOption.READ) + recorder = new FlightRecorder(channel) + reader = new FlightRecorderReader(channel) + body(recorder, reader, channel) + } finally { + // Try to delete anyway + try { + if (randomAccessFile ne null) randomAccessFile.close() + if (recorder ne null) recorder.close() + if (reader ne null) reader.close() + if (channel ne null) channel.close() + file.delete() + } catch { case e: IOException ⇒ e.printStackTrace() } + } + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/FlushOnShutdownSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/FlushOnShutdownSpec.scala new file mode 100644 index 0000000000..918241ed77 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/FlushOnShutdownSpec.scala @@ -0,0 +1,69 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.actor.{ Actor, ActorIdentity, ActorSystem, Identify, Props, RootActorPath } +import akka.remote.RARP +import akka.testkit.{ AkkaSpec, ImplicitSender, TestProbe } +import com.typesafe.config.ConfigFactory + +import scala.concurrent.Await +import scala.concurrent.duration._ + +object FlushOnShutdownSpec { + + val config = ConfigFactory.parseString(s""" + akka { + actor.provider = remote + actor.serialize-creators = off + remote.artery.enabled = on + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 + } + """) + +} + +class FlushOnShutdownSpec extends ArteryMultiNodeSpec(FlushOnShutdownSpec.config) { + + val remoteSystem = newRemoteSystem() + + "Artery" must { + + "flush messages enqueued before shutdown" in { + + val probe = TestProbe() + val probeRef = probe.ref + + localSystem.actorOf(Props(new Actor { + def receive = { + case msg ⇒ probeRef ! msg + } + }), "receiver") + + val actorOnSystemB = remoteSystem.actorOf(Props(new Actor { + def receive = { + case "start" ⇒ + context.actorSelection(rootActorPath(localSystem) / "user" / "receiver") ! Identify(None) + + case ActorIdentity(_, Some(receiverRef)) ⇒ + receiverRef ! "msg1" + receiverRef ! "msg2" + receiverRef ! "msg3" + context.system.terminate() + } + }), "sender") + + actorOnSystemB ! "start" + + probe.expectMsg("msg1") + probe.expectMsg("msg2") + probe.expectMsg("msg3") + + Await.result(remoteSystem.whenTerminated, 6.seconds) + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeDenySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeDenySpec.scala new file mode 100644 index 0000000000..fe6e9993ca --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeDenySpec.scala @@ -0,0 +1,49 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.duration._ + +import akka.actor.{ ActorIdentity, ActorSystem, Identify } +import akka.testkit._ +import com.typesafe.config.ConfigFactory +import akka.actor.RootActorPath + +object HandshakeDenySpec { + + val commonConfig = ConfigFactory.parseString(s""" + akka.loglevel = WARNING + akka { + actor.provider = remote + remote.artery.enabled = on + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 + remote.artery.advanced.handshake-timeout = 2s + remote.artery.advanced.image-liveness-timeout = 1.9s + } + """) + +} + +class HandshakeDenySpec extends ArteryMultiNodeSpec(HandshakeDenySpec.commonConfig) with ImplicitSender { + import HandshakeDenySpec._ + + var systemB = newRemoteSystem(name = Some("systemB")) + + "Artery handshake" must { + + "be denied when originating address is unknown" in { + val sel = system.actorSelection(RootActorPath(address(systemB).copy(host = Some("127.0.0.1"))) / "user" / "echo") + + systemB.actorOf(TestActors.echoActorProps, "echo") + + EventFilter.warning(start = "Dropping Handshake Request from").intercept { + sel ! Identify(None) + expectNoMsg(3.seconds) + }(systemB) + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala new file mode 100644 index 0000000000..ec72c0bd4a --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala @@ -0,0 +1,71 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.duration._ + +import akka.actor.{ ActorIdentity, ActorSystem, Identify } +import akka.testkit.{ AkkaSpec, ImplicitSender } +import akka.testkit.SocketUtil +import akka.testkit.TestActors +import com.typesafe.config.ConfigFactory +import akka.testkit.TestProbe + +object HandshakeFailureSpec { + + // need the port before systemB is started + val portB = SocketUtil.temporaryServerAddress("localhost", udp = true).getPort + + val commonConfig = ConfigFactory.parseString(s""" + akka { + actor.provider = remote + remote.artery.enabled = on + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 + remote.artery.advanced.handshake-timeout = 2s + remote.artery.advanced.image-liveness-timeout = 1.9s + } + """) + + val configB = ConfigFactory.parseString(s"akka.remote.artery.canonical.port = $portB") + .withFallback(commonConfig) + +} + +class HandshakeFailureSpec extends AkkaSpec(HandshakeFailureSpec.commonConfig) with ImplicitSender { + import HandshakeFailureSpec._ + + var systemB: ActorSystem = null + + "Artery handshake" must { + + "allow for timeout and later connect" in { + def sel = system.actorSelection(s"akka://systemB@localhost:$portB/user/echo") + sel ! "hello" + expectNoMsg(3.seconds) // longer than handshake-timeout + + systemB = ActorSystem("systemB", HandshakeFailureSpec.configB) + systemB.actorOf(TestActors.echoActorProps, "echo") + + within(10.seconds) { + awaitAssert { + val probe = TestProbe() + sel.tell("hello2", probe.ref) + probe.expectMsg(1.second, "hello2") + } + } + + sel ! Identify(None) + val remoteRef = expectMsgType[ActorIdentity].ref.get + + remoteRef ! "ping" + expectMsg("ping") + } + + } + + override def afterTermination(): Unit = + if (systemB != null) shutdown(systemB) + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala new file mode 100644 index 0000000000..8573ad34f1 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala @@ -0,0 +1,64 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.duration._ + +import akka.actor._ +import akka.testkit.{ AkkaSpec, ImplicitSender } +import akka.testkit.SocketUtil +import akka.testkit.TestActors +import com.typesafe.config.ConfigFactory + +object HandshakeRetrySpec { + + // need the port before systemB is started + val portB = SocketUtil.temporaryServerAddress("localhost", udp = true).getPort + + val commonConfig = ConfigFactory.parseString(s""" + akka { + actor.provider = remote + remote.artery.enabled = on + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 + remote.artery.advanced.handshake-timeout = 10s + remote.artery.advanced.image-liveness-timeout = 7s + } + """) + + val configB = ConfigFactory.parseString(s"akka.remote.artery.canonical.port = $portB") + .withFallback(commonConfig) + +} + +class HandshakeRetrySpec extends AkkaSpec(HandshakeRetrySpec.commonConfig) with ImplicitSender { + import HandshakeRetrySpec._ + + var systemB: ActorSystem = null + + "Artery handshake" must { + + "be retried during handshake-timeout (no message loss)" in { + def sel = system.actorSelection(s"akka://systemB@localhost:$portB/user/echo") + sel ! "hello" + expectNoMsg(1.second) + + systemB = ActorSystem("systemB", HandshakeRetrySpec.configB) + systemB.actorOf(TestActors.echoActorProps, "echo") + + expectMsg("hello") + + sel ! Identify(None) + val remoteRef = expectMsgType[ActorIdentity].ref.get + + remoteRef ! "ping" + expectMsg("ping") + } + + } + + override def afterTermination(): Unit = + if (systemB != null) shutdown(systemB) + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/ImmutableLongMapSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/ImmutableLongMapSpec.scala new file mode 100644 index 0000000000..48919f9d00 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/ImmutableLongMapSpec.scala @@ -0,0 +1,148 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.duration._ +import org.scalatest.Matchers +import org.scalatest.WordSpec +import akka.util.OptionVal +import scala.util.Random + +class ImmutableLongMapSpec extends WordSpec with Matchers { + + "ImmutableLongMap" must { + + "have no entries when empty" in { + val empty = ImmutableLongMap.empty[String] + empty.size should be(0) + empty.keysIterator.toList should be(Nil) + } + + "add and get entries" in { + val m1 = ImmutableLongMap.empty[String].updated(10L, "10") + m1.keysIterator.toList should be(List(10L)) + m1.keysIterator.map(m1.get).toList should be(List(OptionVal("10"))) + + val m2 = m1.updated(20L, "20") + m2.keysIterator.toList should be(List(10L, 20L)) + m2.keysIterator.map(m2.get).toList should be(List(OptionVal("10"), OptionVal("20"))) + + val m3 = m1.updated(5L, "5") + m3.keysIterator.toList should be(List(5L, 10L)) + m3.keysIterator.map(m3.get).toList should be(List(OptionVal("5"), OptionVal("10"))) + + val m4 = m2.updated(5L, "5") + m4.keysIterator.toList should be(List(5L, 10L, 20L)) + m4.keysIterator.map(m4.get).toList should be(List(OptionVal("5"), OptionVal("10"), OptionVal("20"))) + + val m5 = m4.updated(15L, "15") + m5.keysIterator.toList should be(List(5L, 10L, 15L, 20L)) + m5.keysIterator.map(m5.get).toList should be(List(OptionVal("5"), OptionVal("10"), OptionVal("15"), + OptionVal("20"))) + } + + "replace entries" in { + val m1 = ImmutableLongMap.empty[String].updated(10L, "10a").updated(10, "10b") + m1.keysIterator.map(m1.get).toList should be(List(OptionVal("10b"))) + + val m2 = m1.updated(20L, "20a").updated(30L, "30a") + .updated(20L, "20b").updated(30L, "30b") + m2.keysIterator.map(m2.get).toList should be(List(OptionVal("10b"), OptionVal("20b"), OptionVal("30b"))) + } + + "have toString" in { + ImmutableLongMap.empty[String].toString should be("ImmutableLongMap()") + ImmutableLongMap.empty[String].updated(10L, "a").toString should be("ImmutableLongMap(10 -> a)") + ImmutableLongMap.empty[String].updated(10L, "a").updated(20, "b").toString should be( + "ImmutableLongMap(10 -> a, 20 -> b)") + } + + "have equals and hashCode" in { + ImmutableLongMap.empty[String].updated(10L, "10") should be(ImmutableLongMap.empty[String].updated(10L, "10")) + ImmutableLongMap.empty[String].updated(10L, "10").hashCode should be( + ImmutableLongMap.empty[String].updated(10L, "10").hashCode) + + ImmutableLongMap.empty[String].updated(10L, "10").updated(20, "20").updated(30, "30") should be( + ImmutableLongMap.empty[String].updated(10L, "10").updated(20, "20").updated(30, "30")) + ImmutableLongMap.empty[String].updated(10L, "10").updated(20, "20").updated(30, "30").hashCode should be( + ImmutableLongMap.empty[String].updated(10L, "10").updated(20, "20").updated(30, "30").hashCode) + + ImmutableLongMap.empty[String].updated(10L, "10").updated(20, "20") should not be (ImmutableLongMap.empty[String].updated(10L, "10")) + + ImmutableLongMap.empty[String].updated(10L, "10").updated(20, "20").updated(30, "30") should not be ( + ImmutableLongMap.empty[String].updated(10L, "10").updated(20, "20b").updated(30, "30")) + + ImmutableLongMap.empty[String].updated(10L, "10").updated(20, "20").updated(30, "30") should not be ( + ImmutableLongMap.empty[String].updated(10L, "10").updated(20, "20b").updated(31, "30")) + + ImmutableLongMap.empty[String] should be(ImmutableLongMap.empty[String]) + ImmutableLongMap.empty[String].hashCode should be(ImmutableLongMap.empty[String].hashCode) + } + + "remove entries" in { + val m1 = ImmutableLongMap.empty[String].updated(10L, "10").updated(20, "20").updated(30, "30") + + val m2 = m1.remove(10L) + m2.keysIterator.map(m2.get).toList should be(List(OptionVal("20"), OptionVal("30"))) + + val m3 = m1.remove(20L) + m3.keysIterator.map(m3.get).toList should be(List(OptionVal("10"), OptionVal("30"))) + + val m4 = m1.remove(30L) + m4.keysIterator.map(m4.get).toList should be(List(OptionVal("10"), OptionVal("20"))) + + m1.remove(5L) should be(m1) + + m1.remove(10L).remove(20L).remove(30L) should be(ImmutableLongMap.empty) + } + + "get None when entry doesn't exist" in { + val m1 = ImmutableLongMap.empty[String].updated(10L, "10").updated(20L, "20").updated(30L, "30") + m1.get(5L) should be(OptionVal.None) + m1.get(15L) should be(OptionVal.None) + m1.get(25L) should be(OptionVal.None) + m1.get(35L) should be(OptionVal.None) + } + + "contain keys" in { + val m1 = ImmutableLongMap.empty[String].updated(10L, "10").updated(20L, "20").updated(30L, "30") + m1.contains(10L) should be(true) + m1.contains(20L) should be(true) + m1.contains(30L) should be(true) + m1.contains(5L) should be(false) + m1.contains(25L) should be(false) + } + + "have correct behavior for random operations" in { + val seed = System.nanoTime() + val rnd = new Random(seed) + + var longMap = ImmutableLongMap.empty[String] + var reference = Map.empty[Long, String] + + def verify(): Unit = { + val m = longMap.keysIterator.map(key ⇒ key → longMap.get(key).get).toMap + + m should be(reference) + } + + (1 to 1000).foreach { i ⇒ + withClue(s"seed=$seed, iteration=$i") { + val key = rnd.nextInt(100) + val value = String.valueOf(rnd.nextPrintableChar()) + rnd.nextInt(3) match { + case 0 | 1 ⇒ + longMap = longMap.updated(key, value) + reference = reference.updated(key, value) + case 2 ⇒ + longMap = longMap.remove(key) + reference = reference - key + } + verify() + } + } + } + + } +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala new file mode 100644 index 0000000000..a6030bdffc --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala @@ -0,0 +1,77 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.actor.Address +import akka.remote.UniqueAddress +import akka.remote.artery.InboundControlJunction.ControlMessageObserver +import akka.stream.ActorMaterializer +import akka.stream.ActorMaterializerSettings +import akka.stream.scaladsl.Keep +import akka.stream.testkit.scaladsl.TestSink +import akka.stream.testkit.scaladsl.TestSource +import akka.testkit.AkkaSpec +import akka.testkit.ImplicitSender +import akka.testkit.TestProbe +import akka.util.OptionVal + +object InboundControlJunctionSpec { + trait TestControlMessage extends ControlMessage + + case object Control1 extends TestControlMessage + case object Control2 extends TestControlMessage + case object Control3 extends TestControlMessage +} + +class InboundControlJunctionSpec + extends AkkaSpec(""" + akka.actor.serialization-bindings { + "akka.remote.artery.InboundControlJunctionSpec$TestControlMessage" = java + } + """) with ImplicitSender { + import InboundControlJunctionSpec._ + + val matSettings = ActorMaterializerSettings(system).withFuzzing(true) + implicit val mat = ActorMaterializer(matSettings)(system) + + val addressA = UniqueAddress(Address("akka", "sysA", "hostA", 1001), 1) + val addressB = UniqueAddress(Address("akka", "sysB", "hostB", 1002), 2) + + "Control messages" must { + + "be emitted via side channel" in { + val observerProbe = TestProbe() + val inboundContext = new TestInboundContext(localAddress = addressB) + val recipient = OptionVal.None // not used + + val ((upstream, controlSubject), downstream) = TestSource.probe[AnyRef] + .map(msg ⇒ InboundEnvelope(recipient, msg, OptionVal.None, addressA.uid, OptionVal.None)) + .viaMat(new InboundControlJunction)(Keep.both) + .map { case env: InboundEnvelope ⇒ env.message } + .toMat(TestSink.probe[Any])(Keep.both) + .run() + + controlSubject.attach(new ControlMessageObserver { + override def notify(env: InboundEnvelope) = { + observerProbe.ref ! env.message + } + }) + + downstream.request(10) + upstream.sendNext("msg1") + downstream.expectNext("msg1") + upstream.sendNext(Control1) + upstream.sendNext(Control2) + observerProbe.expectMsg(Control1) + observerProbe.expectMsg(Control2) + upstream.sendNext("msg2") + downstream.expectNext("msg2") + upstream.sendNext(Control3) + observerProbe.expectMsg(Control3) + downstream.cancel() + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala new file mode 100644 index 0000000000..96f1b7d648 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala @@ -0,0 +1,101 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.Await +import scala.concurrent.duration._ +import akka.actor.Address +import akka.remote.UniqueAddress +import akka.remote.artery.OutboundHandshake.HandshakeReq +import akka.remote.artery.OutboundHandshake.HandshakeRsp +import akka.stream.ActorMaterializer +import akka.stream.ActorMaterializerSettings +import akka.stream.scaladsl.Keep +import akka.stream.testkit.TestPublisher +import akka.stream.testkit.TestSubscriber +import akka.stream.testkit.scaladsl.TestSink +import akka.stream.testkit.scaladsl.TestSource +import akka.testkit.AkkaSpec +import akka.testkit.ImplicitSender +import akka.testkit.TestProbe +import akka.util.OptionVal + +object InboundHandshakeSpec { + case object Control1 extends ControlMessage + case object Control2 extends ControlMessage + case object Control3 extends ControlMessage +} + +class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { + + val matSettings = ActorMaterializerSettings(system).withFuzzing(true) + implicit val mat = ActorMaterializer(matSettings)(system) + + val addressA = UniqueAddress(Address("akka", "sysA", "hostA", 1001), 1) + val addressB = UniqueAddress(Address("akka", "sysB", "hostB", 1002), 2) + + private def setupStream(inboundContext: InboundContext, timeout: FiniteDuration = 5.seconds): (TestPublisher.Probe[AnyRef], TestSubscriber.Probe[Any]) = { + val recipient = OptionVal.None // not used + TestSource.probe[AnyRef] + .map(msg ⇒ InboundEnvelope(recipient, msg, OptionVal.None, addressA.uid, + inboundContext.association(addressA.uid))) + .via(new InboundHandshake(inboundContext, inControlStream = true)) + .map { case env: InboundEnvelope ⇒ env.message } + .toMat(TestSink.probe[Any])(Keep.both) + .run() + } + + "InboundHandshake stage" must { + + "send HandshakeRsp as reply to HandshakeReq" in { + val replyProbe = TestProbe() + val inboundContext = new TestInboundContext(addressB, controlProbe = Some(replyProbe.ref)) + val (upstream, downstream) = setupStream(inboundContext) + + downstream.request(10) + upstream.sendNext(HandshakeReq(addressA, addressB.address)) + upstream.sendNext("msg1") + replyProbe.expectMsg(HandshakeRsp(addressB)) + downstream.expectNext("msg1") + downstream.cancel() + } + + "complete remoteUniqueAddress when receiving HandshakeReq" in { + val inboundContext = new TestInboundContext(addressB) + val (upstream, downstream) = setupStream(inboundContext) + + downstream.request(10) + upstream.sendNext(HandshakeReq(addressA, addressB.address)) + upstream.sendNext("msg1") + downstream.expectNext("msg1") + val uniqueRemoteAddress = Await.result( + inboundContext.association(addressA.address).associationState.uniqueRemoteAddress, remainingOrDefault) + uniqueRemoteAddress should ===(addressA) + downstream.cancel() + } + + "drop message from unknown (receiving system restarted)" in { + val replyProbe = TestProbe() + val inboundContext = new TestInboundContext(addressB, controlProbe = Some(replyProbe.ref)) + val (upstream, downstream) = setupStream(inboundContext) + + downstream.request(10) + // no HandshakeReq + upstream.sendNext("msg17") + downstream.expectNoMsg(200.millis) // messages from unknown are dropped + + // and accept messages after handshake + upstream.sendNext(HandshakeReq(addressA, addressB.address)) + upstream.sendNext("msg18") + replyProbe.expectMsg(HandshakeRsp(addressB)) + downstream.expectNext("msg18") + upstream.sendNext("msg19") + downstream.expectNext("msg19") + + downstream.cancel() + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala new file mode 100644 index 0000000000..378260c321 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala @@ -0,0 +1,123 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.actor.{ Actor, ActorRef, ActorSelection, Props, RootActorPath } +import akka.remote.{ RARP, RemoteActorRef } +import akka.testkit.TestProbe +import akka.util.ByteString + +import scala.concurrent.Await +import scala.concurrent.duration._ +import akka.testkit.JavaSerializable + +object LargeMessagesStreamSpec { + case class Ping(payload: ByteString = ByteString.empty) extends JavaSerializable + case class Pong(bytesReceived: Long) extends JavaSerializable + + class EchoSize extends Actor { + def receive = { + case Ping(bytes) ⇒ sender() ! Pong(bytes.size) + } + } +} + +class LargeMessagesStreamSpec extends ArteryMultiNodeSpec( + """ + akka { + loglevel = ERROR + remote.artery.large-message-destinations = [ "/user/large" ] + } + """.stripMargin) { + + import LargeMessagesStreamSpec._ + + "The large message support" should { + + "not affect regular communication" in { + val systemA = localSystem + val systemB = newRemoteSystem() + + val senderProbeA = TestProbe()(systemA) + val senderProbeB = TestProbe()(systemB) + + // start actor and make sure it is up and running + val regular = systemB.actorOf(Props(new EchoSize), "regular") + regular.tell(Ping(), senderProbeB.ref) + senderProbeB.expectMsg(Pong(0)) + + // communicate with it from the other system + val regularRemote = awaitResolve(systemA.actorSelection(rootActorPath(systemB) / "user" / "regular")) + regularRemote.tell(Ping(), senderProbeA.ref) + senderProbeA.expectMsg(Pong(0)) + + // flag should be cached now + regularRemote.asInstanceOf[RemoteActorRef].cachedSendQueueIndex should be >= (Association.OrdinaryQueueIndex) + + } + + "pass small regular messages over the large-message stream" in { + val systemA = localSystem + val systemB = newRemoteSystem() + + val senderProbeA = TestProbe()(systemA) + val senderProbeB = TestProbe()(systemB) + + // start actor and make sure it is up and running + val large = systemB.actorOf(Props(new EchoSize), "large") + large.tell(Ping(), senderProbeB.ref) + senderProbeB.expectMsg(Pong(0)) + + // communicate with it from the other system + val addressB = RARP(systemB).provider.getDefaultAddress + val rootB = RootActorPath(addressB) + val largeRemote = awaitResolve(systemA.actorSelection(rootB / "user" / "large")) + largeRemote.tell(Ping(), senderProbeA.ref) + senderProbeA.expectMsg(Pong(0)) + + // flag should be cached now + largeRemote.asInstanceOf[RemoteActorRef].cachedSendQueueIndex should ===(Association.LargeQueueIndex) + + } + + "allow for normal communication while simultaneously sending large messages" in { + val systemA = localSystem + val systemB = newRemoteSystem() + + val senderProbeB = TestProbe()(systemB) + + // setup two actors, one with the large flag and one regular + val large = systemB.actorOf(Props(new EchoSize), "large") + large.tell(Ping(), senderProbeB.ref) + senderProbeB.expectMsg(Pong(0)) + + val regular = systemB.actorOf(Props(new EchoSize), "regular") + regular.tell(Ping(), senderProbeB.ref) + senderProbeB.expectMsg(Pong(0)) + + // both up and running, resolve remote refs + val addressB = RARP(systemB).provider.getDefaultAddress + val rootB = RootActorPath(addressB) + val largeRemote = awaitResolve(systemA.actorSelection(rootB / "user" / "large")) + val regularRemote = awaitResolve(systemA.actorSelection(rootB / "user" / "regular")) + + // send a large message, as well as regular one + val remoteProbe = TestProbe()(systemA) + + val largeBytes = 2000000 + largeRemote.tell(Ping(ByteString.fromArray(Array.ofDim[Byte](largeBytes))), remoteProbe.ref) + regularRemote.tell(Ping(), remoteProbe.ref) + + // should be no problems sending regular small messages while large messages are being sent + remoteProbe.expectMsg(Pong(0)) + remoteProbe.expectMsg(10.seconds, Pong(largeBytes)) + + // cached flags should be set now + largeRemote.asInstanceOf[RemoteActorRef].cachedSendQueueIndex should ===(Association.LargeQueueIndex) + regularRemote.asInstanceOf[RemoteActorRef].cachedSendQueueIndex should be >= (Association.OrdinaryQueueIndex) + } + } + + def awaitResolve(selection: ActorSelection): ActorRef = Await.result(selection.resolveOne(3.seconds), 3.seconds) +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala new file mode 100644 index 0000000000..e50b71d91a --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala @@ -0,0 +1,66 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.duration._ + +import akka.actor.ActorSystem +import akka.actor.RootActorPath +import akka.remote.RARP +import akka.testkit.AkkaSpec +import akka.testkit.ImplicitSender +import akka.testkit.SocketUtil +import akka.testkit.TestActors +import akka.testkit.TestProbe +import com.typesafe.config.ConfigFactory + +object LateConnectSpec { + + val config = ConfigFactory.parseString(s""" + akka { + actor.provider = remote + remote.artery.enabled = on + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 + remote.artery.advanced.handshake-timeout = 3s + remote.artery.advanced.image-liveness-timeout = 2.9s + } + """) + +} + +class LateConnectSpec extends AkkaSpec(LateConnectSpec.config) with ImplicitSender { + + val portB = SocketUtil.temporaryServerAddress("localhost", udp = true).getPort + val configB = ConfigFactory.parseString(s"akka.remote.artery.canonical.port = $portB") + .withFallback(system.settings.config) + lazy val systemB = ActorSystem("systemB", configB) + + "Connection" must { + + "be established after initial lazy restart" in { + system.actorOf(TestActors.echoActorProps, "echoA") + + val echoB = system.actorSelection(s"akka://systemB@localhost:$portB/user/echoB") + echoB ! "ping1" + + // let the outbound streams be restarted (lazy), systemB is not started yet + Thread.sleep((RARP(system).provider.remoteSettings.Artery.Advanced.HandshakeTimeout + 1.second).toMillis) + + // start systemB + systemB.actorOf(TestActors.echoActorProps, "echoB") + + val probeB = TestProbe()(systemB) + val echoA = systemB.actorSelection(RootActorPath(RARP(system).provider.getDefaultAddress) / "user" / "echoA") + echoA.tell("ping2", probeB.ref) + probeB.expectMsg(10.seconds, "ping2") + + echoB ! "ping3" + expectMsg("ping3") + } + } + + override def afterTermination(): Unit = shutdown(systemB) + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/LruBoundedCacheSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/LruBoundedCacheSpec.scala new file mode 100644 index 0000000000..1ae4329585 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/LruBoundedCacheSpec.scala @@ -0,0 +1,225 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.testkit.AkkaSpec + +import scala.util.Random + +class LruBoundedCacheSpec extends AkkaSpec { + + class TestCache(_capacity: Int, threshold: Int, hashSeed: String = "") extends LruBoundedCache[String, String](_capacity, threshold) { + private var cntr = 0 + + override protected def compute(k: String): String = { + val id = cntr + cntr += 1 + k + ":" + id + } + override protected def hash(k: String): Int = FastHash.ofString(hashSeed + k + hashSeed) + + override protected def isCacheable(v: String): Boolean = !v.startsWith("#") + + def internalProbeDistanceOf(idealSlot: Int, actualSlot: Int): Int = probeDistanceOf(idealSlot, actualSlot) + + def expectComputed(key: String, value: String): Unit = { + get(key) should ===(None) + getOrCompute(key) should ===(value) + get(key) should ===(Some(value)) + } + + def expectCached(key: String, value: String): Unit = { + get(key) should ===(Some(value)) + getOrCompute(key) should ===(value) + get(key) should ===(Some(value)) + } + + def expectComputedOnly(key: String, value: String): Unit = { + get(key) should ===(None) + getOrCompute(key) should ===(value) + get(key) should ===(None) + } + } + + final class BrokenHashFunctionTestCache(_capacity: Int, _threshold: Int) extends TestCache(_capacity, _threshold) { + override protected def hash(k: String): Int = 0 + } + + "LruBoundedCache" must { + + "work in the happy case" in { + val cache = new TestCache(4, 4) + + cache.expectComputed("A", "A:0") + cache.expectComputed("B", "B:1") + cache.expectComputed("C", "C:2") + cache.expectComputed("D", "D:3") + + cache.expectCached("A", "A:0") + cache.expectCached("B", "B:1") + cache.expectCached("C", "C:2") + cache.expectCached("D", "D:3") + } + + "evict oldest when full" in { + for (_ ← 1 to 10) { + val seed = Random.nextInt(1024) + info(s"Variant $seed") + val cache = new TestCache(4, 4, seed.toString) + + cache.expectComputed("A", "A:0") + cache.expectComputed("B", "B:1") + cache.expectComputed("C", "C:2") + cache.expectComputed("D", "D:3") + cache.expectComputed("E", "E:4") + + cache.expectCached("B", "B:1") + cache.expectCached("C", "C:2") + cache.expectCached("D", "D:3") + cache.expectCached("E", "E:4") + + cache.expectComputed("A", "A:5") + cache.expectComputed("B", "B:6") + cache.expectComputed("C", "C:7") + cache.expectComputed("D", "D:8") + cache.expectComputed("E", "E:9") + + cache.expectCached("B", "B:6") + cache.expectCached("C", "C:7") + cache.expectCached("D", "D:8") + cache.expectCached("E", "E:9") + } + } + + "work with low quality hash function" in { + val cache = new BrokenHashFunctionTestCache(4, 4) + + cache.expectComputed("A", "A:0") + cache.expectComputed("B", "B:1") + cache.expectComputed("C", "C:2") + cache.expectComputed("D", "D:3") + cache.expectComputed("E", "E:4") + + cache.expectCached("B", "B:1") + cache.expectCached("C", "C:2") + cache.expectCached("D", "D:3") + cache.expectCached("E", "E:4") + + cache.expectComputed("A", "A:5") + cache.expectComputed("B", "B:6") + cache.expectComputed("C", "C:7") + cache.expectComputed("D", "D:8") + cache.expectComputed("E", "E:9") + + cache.expectCached("B", "B:6") + cache.expectCached("C", "C:7") + cache.expectCached("D", "D:8") + cache.expectCached("E", "E:9") + } + + "calculate probe distance correctly" in { + val cache = new TestCache(4, 4) + + cache.internalProbeDistanceOf(0, 0) should ===(0) + cache.internalProbeDistanceOf(0, 1) should ===(1) + cache.internalProbeDistanceOf(0, 2) should ===(2) + cache.internalProbeDistanceOf(0, 3) should ===(3) + + cache.internalProbeDistanceOf(1, 1) should ===(0) + cache.internalProbeDistanceOf(1, 2) should ===(1) + cache.internalProbeDistanceOf(1, 3) should ===(2) + cache.internalProbeDistanceOf(1, 0) should ===(3) + + cache.internalProbeDistanceOf(2, 2) should ===(0) + cache.internalProbeDistanceOf(2, 3) should ===(1) + cache.internalProbeDistanceOf(2, 0) should ===(2) + cache.internalProbeDistanceOf(2, 1) should ===(3) + + cache.internalProbeDistanceOf(3, 3) should ===(0) + cache.internalProbeDistanceOf(3, 0) should ===(1) + cache.internalProbeDistanceOf(3, 1) should ===(2) + cache.internalProbeDistanceOf(3, 2) should ===(3) + } + + "work with a lower age threshold" in { + for (_ ← 1 to 10) { + val seed = Random.nextInt(1024) + info(s"Variant $seed") + val cache = new TestCache(4, 2, seed.toString) + + cache.expectComputed("A", "A:0") + cache.expectComputed("B", "B:1") + cache.expectComputed("C", "C:2") + cache.expectComputed("D", "D:3") + cache.expectComputed("E", "E:4") + + cache.expectCached("D", "D:3") + cache.expectCached("E", "E:4") + + cache.expectComputed("F", "F:5") + cache.expectComputed("G", "G:6") + cache.expectComputed("H", "H:7") + cache.expectComputed("I", "I:8") + cache.expectComputed("J", "J:9") + + cache.expectCached("I", "I:8") + cache.expectCached("J", "J:9") + } + } + + "must not cache noncacheable values" in { + val cache = new TestCache(4, 4) + + cache.expectComputedOnly("#A", "#A:0") + cache.expectComputedOnly("#A", "#A:1") + cache.expectComputedOnly("#A", "#A:2") + cache.expectComputedOnly("#A", "#A:3") + + cache.expectComputed("A", "A:4") + cache.expectComputed("B", "B:5") + cache.expectComputed("C", "C:6") + cache.expectComputed("D", "D:7") + cache.expectComputed("E", "E:8") + + cache.expectCached("B", "B:5") + cache.expectCached("C", "C:6") + cache.expectCached("D", "D:7") + cache.expectCached("E", "E:8") + + cache.expectComputedOnly("#A", "#A:9") + cache.expectComputedOnly("#A", "#A:10") + cache.expectComputedOnly("#A", "#A:11") + cache.expectComputedOnly("#A", "#A:12") + + // Cacheable values are not affected + cache.expectCached("B", "B:5") + cache.expectCached("C", "C:6") + cache.expectCached("D", "D:7") + cache.expectCached("E", "E:8") + } + + "maintain a good average probe distance" in { + for (_ ← 1 to 10) { + val seed = Random.nextInt(1024) + info(s"Variant $seed") + // Cache emulating 60% fill rate + val cache = new TestCache(1024, 600, seed.toString) + + // Fill up cache + for (_ ← 1 to 10000) cache.getOrCompute(Random.nextString(32)) + + val stats = cache.stats + // Have not seen lower than 890 + stats.entries should be > 750 + // Have not seen higher than 1.8 + stats.averageProbeDistance should be < 2.5 + // Have not seen higher than 15 + stats.maxProbeDistance should be < 25 + } + + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/MetaMetadataSerializerSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/MetaMetadataSerializerSpec.scala new file mode 100644 index 0000000000..2652282994 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/MetaMetadataSerializerSpec.scala @@ -0,0 +1,73 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import org.scalacheck.{ Arbitrary, Gen } +import org.scalatest.prop.Checkers +import org.scalatest.{ Matchers, WordSpec } + +class MetaMetadataSerializerSpec extends WordSpec with Matchers with Checkers { + + case class KeyLen(k: Key, l: Len) { + override def toString = s" key = ${k}, len = ${l}" + } + type Key = Byte + type Len = Int + + implicit val arbitraryKeyLength: Arbitrary[KeyLen] = Arbitrary { + for { + key ← Gen.chooseNum(0.toByte, 31.toByte) + len ← Gen.chooseNum(1, 1024) + } yield KeyLen(key, len) + } + + "MetaMetadataSerializer" must { + + "perform roundtrip masking/unmasking of entry key+length" in { + val key: Byte = 17 + val len = 812 + val kl = MetadataEnvelopeSerializer.muxEntryKeyLength(key, len) + + val key2 = MetadataEnvelopeSerializer.unmaskEntryKey(kl) + key2 should ===(key) + val len2 = MetadataEnvelopeSerializer.unmaskEntryLength(kl) + len2 should ===(len) + } + + "perform key roundtrip using mask/unmask" in { + check { (kl: KeyLen) ⇒ + val k = kl.k + + val masked = MetadataEnvelopeSerializer.maskEntryKey(k) + val uk = MetadataEnvelopeSerializer.unmaskEntryKey(masked) + uk should ===(k) + uk == k + } + } + "perform length roundtrip using mask/unmask" in { + check { (kl: KeyLen) ⇒ + val l = kl.l + + val masked = MetadataEnvelopeSerializer.maskEntryLength(l) + val ul = MetadataEnvelopeSerializer.unmaskEntryLength(masked) + ul should ===(l) + ul == l + } + } + "perform muxed roundtrip using mask/unmask" in { + check { (kl: KeyLen) ⇒ + val k = kl.k + val l = kl.l + + val masked = MetadataEnvelopeSerializer.muxEntryKeyLength(k, l) + val uk = MetadataEnvelopeSerializer.unmaskEntryKey(masked) + uk should ===(k) + val ul = MetadataEnvelopeSerializer.unmaskEntryLength(masked) + ul should ===(l) + ul == l && uk == k + } + } + + } +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/MetadataCarryingSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/MetadataCarryingSpec.scala new file mode 100644 index 0000000000..7d953213d8 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/MetadataCarryingSpec.scala @@ -0,0 +1,102 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.concurrent.atomic.AtomicReference + +import scala.concurrent.duration._ + +import akka.actor.ActorRef +import akka.actor.ActorSelectionMessage +import akka.actor.ActorSystem +import akka.actor.ExtendedActorSystem +import akka.actor.Extension +import akka.actor.ExtensionId +import akka.actor.ExtensionIdProvider +import akka.remote.artery.MetadataCarryingSpy.{ RemoteMessageReceived, RemoteMessageSent } +import akka.testkit.ImplicitSender +import akka.testkit.SocketUtil._ +import akka.testkit.TestActors +import akka.testkit.TestProbe +import akka.util.ByteString + +object MetadataCarryingSpy extends ExtensionId[MetadataCarryingSpy] with ExtensionIdProvider { + override def get(system: ActorSystem): MetadataCarryingSpy = super.get(system) + override def lookup = MetadataCarryingSpy + override def createExtension(system: ExtendedActorSystem): MetadataCarryingSpy = new MetadataCarryingSpy + + final case class RemoteMessageSent(recipient: ActorRef, message: Object, sender: ActorRef) + final case class RemoteMessageReceived(recipient: ActorRef, message: Object, sender: ActorRef, metadata: ByteString) +} + +class MetadataCarryingSpy extends Extension { + def ref: Option[ActorRef] = Option(_ref.get()) + def setProbe(bs: ActorRef): Unit = _ref.set(bs) + private[this] val _ref = new AtomicReference[ActorRef]() +} + +class TestInstrument(system: ExtendedActorSystem) extends RemoteInstrument { + + override val identifier: Byte = 1 + + override def remoteMessageSent(recipient: ActorRef, message: Object, sender: ActorRef): ByteString = + message match { + case _: MetadataCarryingSpec.Ping | ActorSelectionMessage(_: MetadataCarryingSpec.Ping, _, _) ⇒ + val metadata = ByteString("!!!") + MetadataCarryingSpy(system).ref.foreach(_ ! RemoteMessageSent(recipient, message, sender)) + metadata // this data will be attached to the remote message + case _ ⇒ + null + } + + override def remoteMessageReceived(recipient: ActorRef, message: Object, sender: ActorRef, metadata: ByteString): Unit = + message match { + case _: MetadataCarryingSpec.Ping | ActorSelectionMessage(_: MetadataCarryingSpec.Ping, _, _) ⇒ + MetadataCarryingSpy(system).ref.foreach(_ ! RemoteMessageReceived(recipient, message, sender, metadata)) + case _ ⇒ + } +} + +object MetadataCarryingSpec { + final case class Ping(payload: ByteString = ByteString.empty) +} + +class MetadataCarryingSpec extends ArteryMultiNodeSpec( + """ + akka { + remote.artery.advanced { + instruments = [ "akka.remote.artery.TestInstrument" ] + } + } + """) with ImplicitSender { + + import MetadataCarryingSpec._ + + "Metadata" should { + + "be included in remote messages" in { + val systemA = localSystem + val systemB = newRemoteSystem(name = Some("systemB")) + + val instrumentProbeA = TestProbe()(systemA) + MetadataCarryingSpy(systemA).setProbe(instrumentProbeA.ref) + val instrumentProbeB = TestProbe()(systemB) + MetadataCarryingSpy(systemB).setProbe(instrumentProbeB.ref) + + systemB.actorOf(TestActors.echoActorProps, "reply") + systemA.actorSelection(rootActorPath(systemB) / "user" / "reply") ! Ping() + expectMsgType[Ping] + + val sentA = instrumentProbeA.expectMsgType[RemoteMessageSent] + val recvdB = instrumentProbeB.expectMsgType[RemoteMessageReceived] + recvdB.metadata should ===(ByteString("!!!")) + + // for the reply + val sentB = instrumentProbeB.expectMsgType[RemoteMessageSent] + val recvdA = instrumentProbeA.expectMsgType[RemoteMessageReceived] + recvdA.metadata should ===(ByteString("!!!")) + } + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/MetadataContainerSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/MetadataContainerSpec.scala new file mode 100644 index 0000000000..a1a4907100 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/MetadataContainerSpec.scala @@ -0,0 +1,65 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.actor.ActorRef +import akka.testkit.{ AkkaSpec, TestProbe } +import akka.util.ByteString +import scala.concurrent.duration._ + +class MetadataContainerSpec extends AkkaSpec { + + "MetadataContainer" should { + "parse, given empty map" in { + val map = new MetadataMap[ByteString] + val container = new MetadataMapRendering(map) + + val rendered = container.render() + val back = MetadataMapParsing.parseRaw(rendered.asByteBuffer) + + map.toString() should ===(back.metadataMap.toString()) + } + "parse, given 1 allocated in map" in { + val map = new MetadataMap[ByteString] + val container = new MetadataMapRendering(map) + map.set(1, ByteString("!!!")) + + val rendered = container.render() + val back = MetadataMapParsing.parseRaw(rendered.asByteBuffer) + + map.toString() should ===(back.metadataMap.toString()) + } + + "apply, given 3 allocated in map" in { + val map = new MetadataMap[ByteString] + val container = new MetadataMapRendering(map) + map.set(1, ByteString("!!!")) + map.set(10, ByteString("??????")) + map.set(31, ByteString(".........")) + + val p = TestProbe() + + def testInstrument(id: Int): RemoteInstrument = { + new RemoteInstrument { + override def identifier: Byte = id.toByte + override def remoteMessageSent(recipient: ActorRef, message: Object, sender: ActorRef): ByteString = ??? + override def remoteMessageReceived(recipient: ActorRef, message: Object, sender: ActorRef, metadata: ByteString): Unit = + p.ref ! s"${identifier}-${metadata.utf8String}" + } + } + val instruments = Vector( + testInstrument(1), testInstrument(31), testInstrument(10) + ) + + val rendered = container.render() + + val mockEnvelope = new ReusableInboundEnvelope + MetadataMapParsing.applyAllRemoteMessageReceivedRaw(instruments, mockEnvelope, rendered.asByteBuffer) + + p.expectMsgAllOf("1-!!!", "10-??????", "31-.........") + p.expectNoMsg(100.millis) + } + + } +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/MetadataMapSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/MetadataMapSpec.scala new file mode 100644 index 0000000000..af0b6c69f9 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/MetadataMapSpec.scala @@ -0,0 +1,47 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.util.OptionVal +import org.scalatest.{ Matchers, WordSpec } + +class MetadataMapSpec extends WordSpec with Matchers { + + "MetadataMap" must { + "hasValueFor" in { + val a = MetadataMap[String]() + + a.hasValueFor(0) should ===(false) + a.set(0, "0") + a.hasValueFor(0) should ===(true) + a.hasValueFor(1) should ===(false) + + a.clear() + a.isEmpty should ===(true) + a.nonEmpty should ===(false) + a.hasValueFor(12) should ===(false) + a.hasValueFor(0) should ===(false) + a.set(0, "0") + a.hasValueFor(0) should ===(true) + } + "setting values" in { + val a = MetadataMap[String]() + + a(0) should ===(OptionVal.None) + a.usedSlots should ===(0) + a.set(0, "0") + a(0) should ===(OptionVal.Some("0")) + a.usedSlots should ===(1) + + a.set(0, "1") + a(0) should ===(OptionVal.Some("1")) + a.usedSlots should ===(1) + + a.set(0, null) + a(0) should ===(OptionVal.None) + a.usedSlots should ===(0) + } + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala new file mode 100644 index 0000000000..58192b5099 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala @@ -0,0 +1,71 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.duration._ +import akka.actor.Address +import akka.remote.EndpointManager.Send +import akka.remote.RemoteActorRef +import akka.remote.UniqueAddress +import akka.remote.artery.SystemMessageDelivery._ +import akka.stream.ActorMaterializer +import akka.stream.ActorMaterializerSettings +import akka.stream.scaladsl.Keep +import akka.stream.testkit.scaladsl.TestSink +import akka.stream.testkit.scaladsl.TestSource +import akka.testkit.AkkaSpec +import akka.testkit.ImplicitSender +import akka.util.OptionVal + +object OutboundControlJunctionSpec { + case object Control1 extends ControlMessage + case object Control2 extends ControlMessage + case object Control3 extends ControlMessage +} + +class OutboundControlJunctionSpec extends AkkaSpec with ImplicitSender { + import OutboundControlJunctionSpec._ + + val matSettings = ActorMaterializerSettings(system).withFuzzing(true) + implicit val mat = ActorMaterializer(matSettings)(system) + + val addressA = UniqueAddress(Address("akka", "sysA", "hostA", 1001), 1) + val addressB = UniqueAddress(Address("akka", "sysB", "hostB", 1002), 2) + + private val outboundEnvelopePool = ReusableOutboundEnvelope.createObjectPool(capacity = 16) + + "Control messages" must { + + "be injected via side channel" in { + val inboundContext = new TestInboundContext(localAddress = addressA) + val outboundContext = inboundContext.association(addressB.address) + + val ((upstream, controlIngress), downstream) = TestSource.probe[String] + .map(msg ⇒ outboundEnvelopePool.acquire().init(OptionVal.None, msg, OptionVal.None)) + .viaMat(new OutboundControlJunction(outboundContext, outboundEnvelopePool))(Keep.both) + .map(env ⇒ env.message) + .toMat(TestSink.probe[Any])(Keep.both) + .run() + + controlIngress.sendControlMessage(Control1) + downstream.request(1) + downstream.expectNext(Control1) + upstream.sendNext("msg1") + downstream.request(1) + downstream.expectNext("msg1") + upstream.sendNext("msg2") + downstream.request(1) + downstream.expectNext("msg2") + controlIngress.sendControlMessage(Control2) + upstream.sendNext("msg3") + downstream.request(10) + downstream.expectNextUnorderedN(List("msg3", Control2)) + controlIngress.sendControlMessage(Control3) + downstream.expectNext(Control3) + downstream.cancel() + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala new file mode 100644 index 0000000000..b5a87f0940 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala @@ -0,0 +1,138 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.duration._ +import akka.actor.Address +import akka.remote.EndpointManager.Send +import akka.remote.RemoteActorRef +import akka.remote.UniqueAddress +import akka.remote.artery.OutboundHandshake.HandshakeReq +import akka.remote.artery.OutboundHandshake.HandshakeTimeoutException +import akka.remote.artery.SystemMessageDelivery._ +import akka.stream.ActorMaterializer +import akka.stream.ActorMaterializerSettings +import akka.stream.scaladsl.Keep +import akka.stream.testkit.TestPublisher +import akka.stream.testkit.TestSubscriber +import akka.stream.testkit.scaladsl.TestSink +import akka.stream.testkit.scaladsl.TestSource +import akka.testkit.AkkaSpec +import akka.testkit.ImplicitSender +import akka.testkit.TestProbe +import akka.util.OptionVal + +class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { + + val matSettings = ActorMaterializerSettings(system).withFuzzing(true) + implicit val mat = ActorMaterializer(matSettings)(system) + + val addressA = UniqueAddress(Address("akka", "sysA", "hostA", 1001), 1) + val addressB = UniqueAddress(Address("akka", "sysB", "hostB", 1002), 2) + + private val outboundEnvelopePool = ReusableOutboundEnvelope.createObjectPool(capacity = 16) + + private def setupStream( + outboundContext: OutboundContext, timeout: FiniteDuration = 5.seconds, + retryInterval: FiniteDuration = 10.seconds, + injectHandshakeInterval: FiniteDuration = 10.seconds): (TestPublisher.Probe[String], TestSubscriber.Probe[Any]) = { + + TestSource.probe[String] + .map(msg ⇒ outboundEnvelopePool.acquire().init(OptionVal.None, msg, OptionVal.None)) + .via(new OutboundHandshake(system, outboundContext, outboundEnvelopePool, timeout, retryInterval, injectHandshakeInterval)) + .map(env ⇒ env.message) + .toMat(TestSink.probe[Any])(Keep.both) + .run() + } + + "OutboundHandshake stage" must { + "send HandshakeReq when first pulled" in { + val inboundContext = new TestInboundContext(localAddress = addressA) + val outboundContext = inboundContext.association(addressB.address) + val (upstream, downstream) = setupStream(outboundContext) + + downstream.request(10) + downstream.expectNext(HandshakeReq(addressA, addressB.address)) + downstream.cancel() + } + + "send HandshakeReq also when uniqueRemoteAddress future completed at startup" in { + val inboundContext = new TestInboundContext(localAddress = addressA) + val outboundContext = inboundContext.association(addressB.address) + inboundContext.completeHandshake(addressB) + val (upstream, downstream) = setupStream(outboundContext) + + upstream.sendNext("msg1") + downstream.request(10) + downstream.expectNext(HandshakeReq(addressA, addressB.address)) + downstream.expectNext("msg1") + downstream.cancel() + } + + "timeout if handshake not completed" in { + val inboundContext = new TestInboundContext(localAddress = addressA) + val outboundContext = inboundContext.association(addressB.address) + val (upstream, downstream) = setupStream(outboundContext, timeout = 200.millis) + + downstream.request(1) + downstream.expectNext(HandshakeReq(addressA, addressB.address)) + downstream.expectError().getClass should be(classOf[HandshakeTimeoutException]) + } + + "retry HandshakeReq" in { + val inboundContext = new TestInboundContext(localAddress = addressA) + val outboundContext = inboundContext.association(addressB.address) + val (upstream, downstream) = setupStream(outboundContext, retryInterval = 100.millis) + + downstream.request(10) + downstream.expectNext(HandshakeReq(addressA, addressB.address)) + downstream.expectNext(HandshakeReq(addressA, addressB.address)) + downstream.expectNext(HandshakeReq(addressA, addressB.address)) + downstream.cancel() + } + + "not deliver messages from upstream until handshake completed" in { + val inboundContext = new TestInboundContext(localAddress = addressA) + val outboundContext = inboundContext.association(addressB.address) + val (upstream, downstream) = setupStream(outboundContext) + + downstream.request(10) + downstream.expectNext(HandshakeReq(addressA, addressB.address)) + upstream.sendNext("msg1") + downstream.expectNoMsg(200.millis) + // InboundHandshake stage will complete the handshake when receiving HandshakeRsp + inboundContext.completeHandshake(addressB) + downstream.expectNext("msg1") + upstream.sendNext("msg2") + downstream.expectNext("msg2") + downstream.cancel() + } + + "inject HandshakeReq" in { + val inboundContext = new TestInboundContext(localAddress = addressA) + val outboundContext = inboundContext.association(addressB.address) + val (upstream, downstream) = setupStream(outboundContext, injectHandshakeInterval = 500.millis) + + downstream.request(10) + upstream.sendNext("msg1") + downstream.expectNext(HandshakeReq(addressA, addressB.address)) + inboundContext.completeHandshake(addressB) + downstream.expectNext("msg1") + + downstream.expectNoMsg(600.millis) + upstream.sendNext("msg2") + upstream.sendNext("msg3") + upstream.sendNext("msg4") + downstream.expectNext(HandshakeReq(addressA, addressB.address)) + downstream.expectNext("msg2") + downstream.expectNext("msg3") + downstream.expectNext("msg4") + downstream.expectNoMsg(600.millis) + + downstream.cancel() + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorForSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorForSpec.scala new file mode 100644 index 0000000000..9474b11be5 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorForSpec.scala @@ -0,0 +1,100 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.actor.{ Actor, ActorRef, ActorRefScope, PoisonPill, Props } +import akka.pattern.ask +import akka.remote.RemoteActorRef +import akka.remote.RemotingSpec.ActorForReq +import akka.testkit.{ EventFilter, _ } +import akka.util.Timeout + +import scala.concurrent.duration._ + +object RemoteActorForSpec { + final case class ActorForReq(s: String) extends JavaSerializable +} + +class RemoteActorForSpec extends ArteryMultiNodeSpec("akka.loglevel=INFO") with ImplicitSender with DefaultTimeout { + + val remoteSystem = newRemoteSystem() + val remotePort = port(remoteSystem) + + "Remote lookups" should { + + "support remote look-ups" in { + remoteSystem.actorOf(TestActors.echoActorProps, "remote-look-ups") + val remoteRef = localSystem.actorFor(s"akka://${remoteSystem.name}@localhost:$remotePort/user/remote-look-ups") + remoteRef ! "ping" + expectMsg("ping") + } + + // FIXME does not log anything currently + "send warning message for wrong address" ignore { + filterEvents(EventFilter.warning(pattern = "Address is now gated for ", occurrences = 1)) { + localSystem.actorFor("akka://nonexistingsystem@localhost:12346/user/echo") ! "ping" + } + } + + "support ask" in { + remoteSystem.actorOf(TestActors.echoActorProps, "support-ask") + val remoteRef = localSystem.actorFor(s"akka://${remoteSystem.name}@localhost:$remotePort/user/support-ask") + + implicit val timeout: Timeout = 10.seconds + (remoteRef ? "ping").futureValue should ===("ping") + } + + "send dead letters on remote if actor does not exist" in { + EventFilter.warning(pattern = "dead.*buh", occurrences = 1).intercept { + localSystem.actorFor(s"akka://${remoteSystem.name}@localhost:$remotePort/dead-letters-on-remote") ! "buh" + }(remoteSystem) + } + + // FIXME needs remote deployment section + "look-up actors across node boundaries" ignore { + val l = localSystem.actorOf(Props(new Actor { + def receive = { + case (p: Props, n: String) ⇒ sender() ! context.actorOf(p, n) + case ActorForReq(s) ⇒ sender() ! context.actorFor(s) + } + }), "looker1") + // child is configured to be deployed on remote-sys (remoteSystem) + l ! ((TestActors.echoActorProps, "child")) + val child = expectMsgType[ActorRef] + // grandchild is configured to be deployed on RemotingSpec (system) + child ! ((TestActors.echoActorProps, "grandchild")) + val grandchild = expectMsgType[ActorRef] + grandchild.asInstanceOf[ActorRefScope].isLocal should ===(true) + grandchild ! 43 + expectMsg(43) + val myref = localSystem.actorFor(system / "looker1" / "child" / "grandchild") + myref.isInstanceOf[RemoteActorRef] should ===(true) + myref ! 44 + expectMsg(44) + lastSender should ===(grandchild) + lastSender should be theSameInstanceAs grandchild + child.asInstanceOf[RemoteActorRef].getParent should ===(l) + localSystem.actorFor("/user/looker1/child") should be theSameInstanceAs child + (l ? ActorForReq("child/..")).mapTo[AnyRef].futureValue should be theSameInstanceAs l + (localSystem.actorFor(system / "looker1" / "child") ? ActorForReq("..")).mapTo[AnyRef].futureValue should be theSameInstanceAs l + + watch(child) + child ! PoisonPill + expectMsg("postStop") + expectTerminated(child) + l ! ((TestActors.echoActorProps, "child")) + val child2 = expectMsgType[ActorRef] + child2 ! 45 + expectMsg(45) + // msg to old ActorRef (different uid) should not get through + child2.path.uid should not be (child.path.uid) + child ! 46 + expectNoMsg(1.second) + system.actorFor(system / "looker1" / "child") ! 47 + expectMsg(47) + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala new file mode 100644 index 0000000000..586f41e8cb --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala @@ -0,0 +1,75 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.actor.{ EmptyLocalActorRef, InternalActorRef } +import akka.remote.RemoteActorRef +import akka.testkit.{ EventFilter, TestActors } +import akka.actor.Props +import akka.actor.ExtendedActorSystem +import akka.actor.ActorRefScope + +class RemoteActorRefProviderSpec extends ArteryMultiNodeSpec { + + val addressA = address(localSystem) + system.actorOf(TestActors.echoActorProps, "echo") + + val systemB = newRemoteSystem() + val addressB = address(systemB) + systemB.actorOf(TestActors.echoActorProps, "echo") + + "RemoteActorRefProvider" must { + + "resolve local actor selection" in { + val sel = system.actorSelection(s"akka://${system.name}@${addressA.host.get}:${addressA.port.get}/user/echo") + sel.anchor.asInstanceOf[InternalActorRef].isLocal should be(true) + } + + "resolve remote actor selection" in { + val sel = system.actorSelection(s"akka://${systemB.name}@${addressB.host.get}:${addressB.port.get}/user/echo") + sel.anchor.getClass should ===(classOf[RemoteActorRef]) + sel.anchor.asInstanceOf[InternalActorRef].isLocal should be(false) + } + + "cache resolveActorRef for local ref" in { + val provider = localSystem.asInstanceOf[ExtendedActorSystem].provider + val path = s"akka://${system.name}@${addressA.host.get}:${addressA.port.get}/user/echo" + val ref1 = provider.resolveActorRef(path) + ref1.getClass should !==(classOf[EmptyLocalActorRef]) + ref1.asInstanceOf[ActorRefScope].isLocal should ===(true) + + val ref2 = provider.resolveActorRef(path) + ref1 should be theSameInstanceAs (ref2) + } + + "not cache resolveActorRef for unresolved ref" in { + val provider = localSystem.asInstanceOf[ExtendedActorSystem].provider + val path = s"akka://${system.name}@${addressA.host.get}:${addressA.port.get}/user/doesNotExist" + val ref1 = provider.resolveActorRef(path) + ref1.getClass should ===(classOf[EmptyLocalActorRef]) + + val ref2 = provider.resolveActorRef(path) + ref1 should not be theSameInstanceAs(ref2) + } + + "cache resolveActorRef for remote ref" in { + val provider = localSystem.asInstanceOf[ExtendedActorSystem].provider + val path = s"akka://${systemB.name}@${addressB.host.get}:${addressB.port.get}/user/echo" + val ref1 = provider.resolveActorRef(path) + ref1.getClass should ===(classOf[RemoteActorRef]) + + val ref2 = provider.resolveActorRef(path) + ref1 should be theSameInstanceAs (ref2) + } + + "detect wrong protocol" in { + EventFilter[IllegalArgumentException](start = "No root guardian at", occurrences = 1).intercept { + val sel = system.actorSelection(s"akka.tcp://${systemB.name}@${addressB.host.get}:${addressB.port.get}/user/echo") + sel.anchor.getClass should ===(classOf[EmptyLocalActorRef]) + } + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala new file mode 100644 index 0000000000..b4b023c81e --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala @@ -0,0 +1,162 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.actor.{ Actor, ActorIdentity, ActorLogging, ActorRef, ActorRefScope, ActorSelection, ActorSystem, ExtendedActorSystem, Identify, PoisonPill, Props, Terminated } +import akka.remote.RARP +import akka.testkit.{ AkkaSpec, ImplicitSender, SocketUtil, TestActors } +import com.typesafe.config.ConfigFactory + +import scala.concurrent.duration._ +import akka.testkit.JavaSerializable + +object RemoteActorSelectionSpec { + final case class ActorSelReq(s: String) extends JavaSerializable + final case class ActorCreateReq(props: Props, name: String) extends JavaSerializable + + class SelectionActor extends Actor with ActorLogging { + log.info("Started") + def receive = { + // if we get props and a name, create a child, send ref back + case ActorCreateReq(p, n) ⇒ + log.info(s"Creating child $n") + sender() ! context.actorOf(p, n) + // or select actor from here + case ActorSelReq(s) ⇒ sender() ! context.actorSelection(s) + } + } + def selectionActorProps = Props(new SelectionActor) +} + +class RemoteActorSelectionSpec extends ArteryMultiNodeSpec with ImplicitSender { + + import RemoteActorSelectionSpec._ + + "Remote actor selection" should { + + // TODO fails with not receiving the localGrandchild value, seems to go to dead letters + "select actors across node boundaries" ignore { + + val remotePort = SocketUtil.temporaryServerAddress(udp = true).getPort + val remoteSysName = "remote-" + system.name + + val localPort = SocketUtil.temporaryServerAddress(udp = true).getPort + val localSysName = "local-" + system.name + + def config(port: Int) = + s""" + akka { + remote.artery.port = $port + actor.deployment { + /looker2/child.remote = "akka://$remoteSysName@localhost:$remotePort" + /looker2/child/grandchild.remote = "akka://$localSysName@localhost:$localPort" + } + } + """ + + val localSystem = newRemoteSystem( + extraConfig = Some(config(localPort)), + name = Some(localSysName)) + + val remoteSystem = newRemoteSystem( + extraConfig = Some(config(remotePort)), + name = Some(remoteSysName)) + + val localLooker2 = localSystem.actorOf(selectionActorProps, "looker2") + + // child is configured to be deployed on remoteSystem + localLooker2 ! ActorCreateReq(selectionActorProps, "child") + val remoteChild = expectMsgType[ActorRef] + + // grandchild is configured to be deployed on local system but from remote system + remoteChild ! ActorCreateReq(selectionActorProps, "grandchild") + val localGrandchild = expectMsgType[ActorRef] + localGrandchild.asInstanceOf[ActorRefScope].isLocal should ===(true) + localGrandchild ! 53 + expectMsg(53) + + val localGrandchildSelection = localSystem.actorSelection(localSystem / "looker2" / "child" / "grandchild") + localGrandchildSelection ! 54 + expectMsg(54) + lastSender should ===(localGrandchild) + lastSender should be theSameInstanceAs localGrandchild + localGrandchildSelection ! Identify(localGrandchildSelection) + val grandchild2 = expectMsgType[ActorIdentity].ref + grandchild2 should ===(Some(localGrandchild)) + + localSystem.actorSelection("/user/looker2/child") ! Identify(None) + expectMsgType[ActorIdentity].ref should ===(Some(remoteChild)) + + localLooker2 ! ActorSelReq("child/..") + expectMsgType[ActorSelection] ! Identify(None) + expectMsgType[ActorIdentity].ref.get should be theSameInstanceAs localLooker2 + + localSystem.actorSelection(localSystem / "looker2" / "child") ! ActorSelReq("..") + expectMsgType[ActorSelection] ! Identify(None) + expectMsgType[ActorIdentity].ref.get should be theSameInstanceAs localLooker2 + + localGrandchild ! ((TestActors.echoActorProps, "grandgrandchild")) + val grandgrandchild = expectMsgType[ActorRef] + + localSystem.actorSelection("/user/looker2/child") ! Identify("idReq1") + expectMsg(ActorIdentity("idReq1", Some(remoteChild))) + localSystem.actorSelection(remoteChild.path) ! Identify("idReq2") + expectMsg(ActorIdentity("idReq2", Some(remoteChild))) + localSystem.actorSelection("/user/looker2/*") ! Identify("idReq3") + expectMsg(ActorIdentity("idReq3", Some(remoteChild))) + + localSystem.actorSelection("/user/looker2/child/grandchild") ! Identify("idReq4") + expectMsg(ActorIdentity("idReq4", Some(localGrandchild))) + localSystem.actorSelection(remoteChild.path / "grandchild") ! Identify("idReq5") + expectMsg(ActorIdentity("idReq5", Some(localGrandchild))) + localSystem.actorSelection("/user/looker2/*/grandchild") ! Identify("idReq6") + expectMsg(ActorIdentity("idReq6", Some(localGrandchild))) + localSystem.actorSelection("/user/looker2/child/*") ! Identify("idReq7") + expectMsg(ActorIdentity("idReq7", Some(localGrandchild))) + localSystem.actorSelection(remoteChild.path / "*") ! Identify("idReq8") + expectMsg(ActorIdentity("idReq8", Some(localGrandchild))) + + localSystem.actorSelection("/user/looker2/child/grandchild/grandgrandchild") ! Identify("idReq9") + expectMsg(ActorIdentity("idReq9", Some(grandgrandchild))) + localSystem.actorSelection(remoteChild.path / "grandchild" / "grandgrandchild") ! Identify("idReq10") + expectMsg(ActorIdentity("idReq10", Some(grandgrandchild))) + localSystem.actorSelection("/user/looker2/child/*/grandgrandchild") ! Identify("idReq11") + expectMsg(ActorIdentity("idReq11", Some(grandgrandchild))) + localSystem.actorSelection("/user/looker2/child/*/*") ! Identify("idReq12") + expectMsg(ActorIdentity("idReq12", Some(grandgrandchild))) + localSystem.actorSelection(remoteChild.path / "*" / "grandgrandchild") ! Identify("idReq13") + expectMsg(ActorIdentity("idReq13", Some(grandgrandchild))) + + val sel1 = localSystem.actorSelection("/user/looker2/child/grandchild/grandgrandchild") + localSystem.actorSelection(sel1.toSerializationFormat) ! Identify("idReq18") + expectMsg(ActorIdentity("idReq18", Some(grandgrandchild))) + + remoteChild ! Identify("idReq14") + expectMsg(ActorIdentity("idReq14", Some(remoteChild))) + watch(remoteChild) + remoteChild ! PoisonPill + expectMsg("postStop") + expectMsgType[Terminated].actor should ===(remoteChild) + localLooker2 ! ((TestActors.echoActorProps, "child")) + val child2 = expectMsgType[ActorRef] + child2 ! Identify("idReq15") + expectMsg(ActorIdentity("idReq15", Some(child2))) + localSystem.actorSelection(remoteChild.path) ! Identify("idReq16") + expectMsg(ActorIdentity("idReq16", Some(child2))) + remoteChild ! Identify("idReq17") + expectMsg(ActorIdentity("idReq17", None)) + + child2 ! 55 + expectMsg(55) + // msg to old ActorRef (different uid) should not get through + child2.path.uid should not be (remoteChild.path.uid) + remoteChild ! 56 + expectNoMsg(1.second) + localSystem.actorSelection(localSystem / "looker2" / "child") ! 57 + expectMsg(57) + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteConnectionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteConnectionSpec.scala new file mode 100644 index 0000000000..8389797538 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteConnectionSpec.scala @@ -0,0 +1,84 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.actor.{ ActorSystem, ExtendedActorSystem } +import akka.remote.RARP +import akka.testkit.SocketUtil._ +import akka.testkit.{ AkkaSpec, EventFilter, ImplicitSender, TestActors, TestEvent, TestProbe } +import com.typesafe.config.ConfigFactory + +import scala.concurrent.duration._ + +class RemoteConnectionSpec extends ArteryMultiNodeSpec("akka.remote.retry-gate-closed-for = 5s") with ImplicitSender { + + def muteSystem(system: ActorSystem) { + system.eventStream.publish(TestEvent.Mute( + EventFilter.error(start = "AssociationError"), + EventFilter.warning(start = "AssociationError"), + EventFilter.warning(pattern = "received dead letter.*"))) + } + + "Remoting between systems" should { + + "be able to connect to system even if it's not there at first" in { + muteSystem(localSystem) + val localProbe = new TestProbe(localSystem) + + val remotePort = temporaryServerAddress(udp = true).getPort + + // try to talk to it before it is up + val selection = localSystem.actorSelection(s"akka://$nextGeneratedSystemName@localhost:$remotePort/user/echo") + selection.tell("ping", localProbe.ref) + localProbe.expectNoMsg(1.seconds) + + // then start the remote system and try again + val remoteSystem = newRemoteSystem(extraConfig = Some(s"akka.remote.artery.canonical.port=$remotePort")) + + muteSystem(remoteSystem) + localProbe.expectNoMsg(2.seconds) + remoteSystem.actorOf(TestActors.echoActorProps, "echo") + + within(5.seconds) { + awaitAssert { + selection.tell("ping", localProbe.ref) + localProbe.expectMsg(500.millis, "ping") + } + } + } + + "allow other system to connect even if it's not there at first" in { + val localSystem = newRemoteSystem() + + val localPort = port(localSystem) + muteSystem(localSystem) + + val localProbe = new TestProbe(localSystem) + localSystem.actorOf(TestActors.echoActorProps, "echo") + + val remotePort = temporaryServerAddress(udp = true).getPort + + // try to talk to remote before it is up + val selection = localSystem.actorSelection(s"akka://$nextGeneratedSystemName@localhost:$remotePort/user/echo") + selection.tell("ping", localProbe.ref) + localProbe.expectNoMsg(1.seconds) + + // then when it is up, talk from other system + val remoteSystem = newRemoteSystem(extraConfig = Some(s"akka.remote.artery.canonical.port=$remotePort")) + + muteSystem(remoteSystem) + localProbe.expectNoMsg(2.seconds) + val otherProbe = new TestProbe(remoteSystem) + val otherSender = otherProbe.ref + val thisSelection = remoteSystem.actorSelection(s"akka://${localSystem.name}@localhost:$localPort/user/echo") + within(5.seconds) { + awaitAssert { + thisSelection.tell("ping", otherSender) + otherProbe.expectMsg(500.millis, "ping") + } + } + } + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala new file mode 100644 index 0000000000..635b73dc8e --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala @@ -0,0 +1,94 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.testkit._ +import akka.actor._ +import com.typesafe.config.ConfigFactory +import akka.actor.RootActorPath +import scala.concurrent.duration._ +import akka.testkit.SocketUtil +import akka.event.Logging.Warning +import akka.remote.QuarantinedEvent +import akka.remote.RARP +import akka.remote.RemoteActorRef + +object RemoteDeathWatchSpec { + val otherPort = SocketUtil.temporaryServerAddress("localhost", udp = true).getPort + + val config = ConfigFactory.parseString(s""" + akka { + actor { + provider = remote + deployment { + /watchers.remote = "akka://other@localhost:$otherPort" + } + } + remote.watch-failure-detector.acceptable-heartbeat-pause = 3s + remote.artery.enabled = on + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 + } + """) +} + +class RemoteDeathWatchSpec extends AkkaSpec(RemoteDeathWatchSpec.config) with ImplicitSender with DefaultTimeout with DeathWatchSpec { + import RemoteDeathWatchSpec._ + + system.eventStream.publish(TestEvent.Mute( + EventFilter[io.aeron.exceptions.RegistrationException]())) + + val other = ActorSystem("other", ConfigFactory.parseString(s"akka.remote.artery.canonical.port=$otherPort") + .withFallback(system.settings.config)) + + override def afterTermination() { + shutdown(other) + } + + override def expectedTestDuration: FiniteDuration = 120.seconds + + "receive Terminated when system of de-serialized ActorRef is not running" in { + val probe = TestProbe() + system.eventStream.subscribe(probe.ref, classOf[QuarantinedEvent]) + val rarp = RARP(system).provider + // pick an unused port + val port = SocketUtil.temporaryServerAddress("localhost", udp = true).getPort + // simulate de-serialized ActorRef + val ref = rarp.resolveActorRef(s"akka://OtherSystem@localhost:$port/user/foo/bar#1752527294") + + // we don't expect real quarantine when the UID is unknown, i.e. QuarantinedEvent is not published + EventFilter.warning(pattern = "Quarantine of .* ignored because unknown UID", occurrences = 1).intercept { + EventFilter.warning(start = "Detected unreachable", occurrences = 1).intercept { + + system.actorOf(Props(new Actor { + context.watch(ref) + def receive = { + case Terminated(r) ⇒ testActor ! r + } + }).withDeploy(Deploy.local)) + + expectMsg(10.seconds, ref) + } + } + } + + "receive Terminated when watched node is unknown host" in { + val path = RootActorPath(Address("akka", system.name, "unknownhost", 2552)) / "user" / "subject" + system.actorOf(Props(new Actor { + context.watch(context.actorFor(path)) + def receive = { + case t: Terminated ⇒ testActor ! t.actor.path + } + }).withDeploy(Deploy.local), name = "observer2") + + expectMsg(60.seconds, path) + } + + "receive ActorIdentity(None) when identified node is unknown host" in { + val path = RootActorPath(Address("akka", system.name, "unknownhost2", 2552)) / "user" / "subject" + system.actorSelection(path) ! Identify(path) + expectMsg(60.seconds, ActorIdentity(path, None)) + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeployerSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeployerSpec.scala new file mode 100644 index 0000000000..b4152ae111 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeployerSpec.scala @@ -0,0 +1,60 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.testkit._ +import akka.actor._ +import akka.routing._ +import com.typesafe.config._ +import akka.ConfigurationException +import akka.remote.RemoteScope + +object RemoteDeployerSpec { + val deployerConf = ConfigFactory.parseString(""" + akka.actor.provider = remote + akka.actor.deployment { + /service2 { + router = round-robin-pool + nr-of-instances = 3 + remote = "akka://sys@wallace:2552" + dispatcher = mydispatcher + } + } + akka.remote.artery.enabled = on + akka.remote.artery.canonical.hostname = localhost + akka.remote.artery.canonical.port = 0 + """, ConfigParseOptions.defaults) + + class RecipeActor extends Actor { + def receive = { case _ ⇒ } + } + +} + +class RemoteDeployerSpec extends AkkaSpec(RemoteDeployerSpec.deployerConf) { + + "A RemoteDeployer" must { + + "be able to parse 'akka.actor.deployment._' with specified remote nodes" in { + val service = "/service2" + val deployment = system.asInstanceOf[ActorSystemImpl].provider.deployer.lookup(service.split("/").drop(1)) + + deployment should ===(Some( + Deploy( + service, + deployment.get.config, + RoundRobinPool(3), + RemoteScope(Address("akka", "sys", "wallace", 2552)), + "mydispatcher"))) + } + + "reject remote deployment when the source requires LocalScope" in { + intercept[ConfigurationException] { + system.actorOf(Props.empty.withDeploy(Deploy.local), "service2") + }.getMessage should ===("configuration requested remote deployment for local-only Props at [akka://RemoteDeployerSpec/user/service2]") + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala new file mode 100644 index 0000000000..90f89c3147 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala @@ -0,0 +1,81 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.collection.immutable +import akka.testkit._ +import akka.routing._ +import akka.actor._ +import akka.remote.routing._ +import com.typesafe.config._ +import akka.testkit.TestActors.echoActorProps +import akka.remote.{ RARP, RemoteScope } + +object RemoteDeploymentSpec { + class Echo1 extends Actor { + var target: ActorRef = context.system.deadLetters + + def receive = { + case ex: Exception ⇒ throw ex + case x ⇒ target = sender(); sender() ! x + } + + override def preStart() {} + override def preRestart(cause: Throwable, msg: Option[Any]) { + target ! "preRestart" + } + override def postRestart(cause: Throwable) {} + override def postStop() { + target ! "postStop" + } + } +} + +class RemoteDeploymentSpec extends AkkaSpec(""" + #akka.loglevel=DEBUG + akka.actor.provider = remote + akka.remote.artery.enabled = on + akka.remote.artery.canonical.hostname = localhost + akka.remote.artery.canonical.port = 0 + """) { + + import RemoteDeploymentSpec._ + + val port = RARP(system).provider.getDefaultAddress.port.get + val conf = ConfigFactory.parseString( + s""" + akka.actor.deployment { + /blub.remote = "akka://${system.name}@localhost:$port" + } + """).withFallback(system.settings.config) + + val masterSystem = ActorSystem("Master" + system.name, conf) + val masterPort = RARP(masterSystem).provider.getDefaultAddress.port.get + + override def afterTermination(): Unit = { + shutdown(masterSystem) + } + + "Remoting" must { + + "create and supervise children on remote node" in { + val senderProbe = TestProbe()(masterSystem) + val r = masterSystem.actorOf(Props[Echo1], "blub") + r.path.toString should ===(s"akka://${system.name}@localhost:${port}/remote/akka/${masterSystem.name}@localhost:${masterPort}/user/blub") + + r.tell(42, senderProbe.ref) + senderProbe.expectMsg(42) + EventFilter[Exception]("crash", occurrences = 1).intercept { + r ! new Exception("crash") + }(masterSystem) + senderProbe.expectMsg("preRestart") + r.tell(43, senderProbe.ref) + senderProbe.expectMsg(43) + system.stop(r) + senderProbe.expectMsg("postStop") + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteFailureSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteFailureSpec.scala new file mode 100644 index 0000000000..f719dfb6b8 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteFailureSpec.scala @@ -0,0 +1,58 @@ +package akka.remote.artery + +import akka.remote.EndpointDisassociatedException +import akka.testkit.{ EventFilter, ImplicitSender, TestActors, TestEvent } + +import scala.concurrent.duration._ + +class RemoteFailureSpec extends ArteryMultiNodeSpec with ImplicitSender { + + "Remoting" should { + + "not be exhausted by sending to broken connections" in { + val remoteSystems = Vector.fill(5)(newRemoteSystem()) + + remoteSystems foreach { sys ⇒ + sys.eventStream.publish(TestEvent.Mute( + EventFilter[EndpointDisassociatedException](), + EventFilter.warning(pattern = "received dead letter.*"))) + sys.actorOf(TestActors.echoActorProps, name = "echo") + } + val remoteSelections = remoteSystems map { sys ⇒ + system.actorSelection(rootActorPath(sys) / "user" / "echo") + } + + val echo = system.actorOf(TestActors.echoActorProps, name = "echo") + + val localSelection = system.actorSelection(rootActorPath(system) / "user" / "echo") + val n = 100 + + // first everything is up and running + 1 to n foreach { x ⇒ + localSelection ! "ping" + remoteSelections(x % remoteSystems.size) ! "ping" + } + + within(5.seconds) { + receiveN(n * 2) foreach { reply ⇒ reply should ===("ping") } + } + + // then we shutdown remote systems to simulate broken connections + remoteSystems foreach { sys ⇒ + shutdown(sys) + } + + 1 to n foreach { x ⇒ + localSelection ! "ping" + remoteSelections(x % remoteSystems.size) ! "ping" + } + + // ping messages to localEcho should go through even though we use many different broken connections + within(5.seconds) { + receiveN(n) foreach { reply ⇒ reply should ===("ping") } + } + + } + + } +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala new file mode 100644 index 0000000000..c84387077d --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala @@ -0,0 +1,118 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.io.NotSerializableException +import java.util.concurrent.ThreadLocalRandom + +import akka.actor.{ Actor, ActorRef, ActorSystem, Address, Deploy, ExtendedActorSystem, PoisonPill, Props } +import akka.remote.{ AssociationErrorEvent, DisassociatedEvent, OversizedPayloadException, RARP } +import akka.testkit.{ AkkaSpec, EventFilter, ImplicitSender, TestActors } +import akka.util.ByteString +import com.typesafe.config.ConfigFactory + +import scala.concurrent.duration._ + +object RemoteMessageSerializationSpec { + class ProxyActor(val one: ActorRef, val another: ActorRef) extends Actor { + def receive = { + case s if sender().path == one.path ⇒ another ! s + case s if sender().path == another.path ⇒ one ! s + } + } +} + +class RemoteMessageSerializationSpec extends ArteryMultiNodeSpec(""" + akka.actor.serialize-messages = off + akka.actor.serialize-creators = off + """) with ImplicitSender { + + import RemoteMessageSerializationSpec._ + + val maxPayloadBytes = RARP(system).provider.remoteSettings.Artery.Advanced.MaximumFrameSize + + val remoteSystem = newRemoteSystem() + val remotePort = port(remoteSystem) + + "Remote message serialization" should { + + "drop unserializable messages" in { + object Unserializable + EventFilter[NotSerializableException](pattern = ".*No configured serialization.*", occurrences = 1).intercept { + verifySend(Unserializable) { + expectNoMsg(1.second) // No AssocitionErrorEvent should be published + } + } + } + + "allow messages up to payload size" in { + val maxProtocolOverhead = 500 // Make sure we're still under size after the message is serialized, etc + val big = byteStringOfSize(maxPayloadBytes - maxProtocolOverhead) + verifySend(big) { + expectMsg(3.seconds, big) + } + } + + "drop sent messages over payload size" in { + val oversized = byteStringOfSize(maxPayloadBytes + 1) + EventFilter[OversizedPayloadException](start = "Failed to serialize oversized message", occurrences = 1).intercept { + verifySend(oversized) { + expectNoMsg(1.second) // No AssocitionErrorEvent should be published + } + } + } + + // TODO max payload size is not configurable yet, so we cannot send a too big message, it fails no sending side + "drop received messages over payload size" ignore { + // Receiver should reply with a message of size maxPayload + 1, which will be dropped and an error logged + EventFilter[OversizedPayloadException](pattern = ".*Discarding oversized payload received.*", occurrences = 1).intercept { + verifySend(maxPayloadBytes + 1) { + expectNoMsg(1.second) // No AssocitionErrorEvent should be published + } + } + } + + "be able to serialize a local actor ref from another actor system" in { + remoteSystem.actorOf(TestActors.echoActorProps, "echo") + val local = localSystem.actorOf(TestActors.echoActorProps, "echo") + + val remoteEcho = system.actorSelection(rootActorPath(remoteSystem) / "user" / "echo").resolveOne(3.seconds).futureValue + remoteEcho ! local + expectMsg(3.seconds, local) + } + + } + + private def verifySend(msg: Any)(afterSend: ⇒ Unit) { + val bigBounceId = s"bigBounce-${ThreadLocalRandom.current.nextInt()}" + val bigBounceOther = remoteSystem.actorOf(Props(new Actor { + def receive = { + case x: Int ⇒ sender() ! byteStringOfSize(x) + case x ⇒ sender() ! x + } + }), bigBounceId) + val bigBounceHere = localSystem.actorFor(s"akka://${remoteSystem.name}@localhost:$remotePort/user/$bigBounceId") + + val eventForwarder = localSystem.actorOf(Props(new Actor { + def receive = { + case x ⇒ testActor ! x + } + })) + localSystem.eventStream.subscribe(eventForwarder, classOf[AssociationErrorEvent]) + localSystem.eventStream.subscribe(eventForwarder, classOf[DisassociatedEvent]) + try { + bigBounceHere ! msg + afterSend + expectNoMsg(500.millis) + } finally { + localSystem.eventStream.unsubscribe(eventForwarder, classOf[AssociationErrorEvent]) + localSystem.eventStream.unsubscribe(eventForwarder, classOf[DisassociatedEvent]) + eventForwarder ! PoisonPill + bigBounceOther ! PoisonPill + } + } + + private def byteStringOfSize(size: Int) = ByteString.fromArray(Array.fill(size)(42: Byte)) + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala new file mode 100644 index 0000000000..80f133b7cc --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala @@ -0,0 +1,250 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.collection.immutable +import akka.testkit._ +import akka.routing._ +import akka.actor._ +import akka.remote.routing._ +import com.typesafe.config._ +import akka.testkit.TestActors.echoActorProps +import akka.remote.{ RARP, RemoteScope } + +object RemoteRouterSpec { + class Parent extends Actor { + def receive = { + case (p: Props, name: String) ⇒ + sender() ! context.actorOf(p, name) + } + } +} + +class RemoteRouterSpec extends AkkaSpec(""" + akka.actor.provider = remote + akka.remote.artery.enabled = on + akka.remote.artery.canonical.hostname = localhost + akka.remote.artery.canonical.port = 0 + akka.actor.deployment { + /remote-override { + router = round-robin-pool + nr-of-instances = 4 + } + /round { + router = round-robin-pool + nr-of-instances = 5 + } + /sys-parent/round { + router = round-robin-pool + nr-of-instances = 6 + } + }""") { + + import RemoteRouterSpec._ + + val port = RARP(system).provider.getDefaultAddress.port.get + val sysName = system.name + val conf = ConfigFactory.parseString( + s""" + akka { + actor.deployment { + /blub { + router = round-robin-pool + nr-of-instances = 2 + target.nodes = ["akka://${sysName}@localhost:${port}"] + } + /elastic-blub { + router = round-robin-pool + resizer { + lower-bound = 2 + upper-bound = 3 + } + target.nodes = ["akka://${sysName}@localhost:${port}"] + } + /remote-blub { + remote = "akka://${sysName}@localhost:${port}" + router = round-robin-pool + nr-of-instances = 2 + } + /local-blub { + remote = "akka://MasterRemoteRouterSpec" + router = round-robin-pool + nr-of-instances = 2 + target.nodes = ["akka://${sysName}@localhost:${port}"] + } + /local-blub2 { + router = round-robin-pool + nr-of-instances = 4 + target.nodes = ["akka://${sysName}@localhost:${port}"] + } + } + }""").withFallback(system.settings.config) + + val masterSystem = ActorSystem("Master" + sysName, conf) + + override def afterTermination(): Unit = { + shutdown(masterSystem) + } + + def collectRouteePaths(probe: TestProbe, router: ActorRef, n: Int): immutable.Seq[ActorPath] = { + for (i ← 1 to n) yield { + val msg = i.toString + router.tell(msg, probe.ref) + probe.expectMsg(msg) + probe.lastSender.path + } + } + + "A Remote Router" must { + + "deploy its children on remote host driven by configuration" in { + val probe = TestProbe()(masterSystem) + val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps), "blub") + val replies = collectRouteePaths(probe, router, 5) + val children = replies.toSet + children should have size 2 + children.map(_.parent) should have size 1 + children foreach (_.address.toString should ===(s"akka://${sysName}@localhost:${port}")) + masterSystem.stop(router) + } + + "deploy its children on remote host driven by programatic definition" in { + val probe = TestProbe()(masterSystem) + val router = masterSystem.actorOf(new RemoteRouterConfig( + RoundRobinPool(2), + Seq(Address("akka", sysName, "localhost", port))).props(echoActorProps), "blub2") + val replies = collectRouteePaths(probe, router, 5) + val children = replies.toSet + children should have size 2 + children.map(_.parent) should have size 1 + children foreach (_.address.toString should ===(s"akka://${sysName}@localhost:${port}")) + masterSystem.stop(router) + } + + "deploy dynamic resizable number of children on remote host driven by configuration" in { + val probe = TestProbe()(masterSystem) + val router = masterSystem.actorOf(FromConfig.props(echoActorProps), "elastic-blub") + val replies = collectRouteePaths(probe, router, 5000) + val children = replies.toSet + children.size should be >= 2 + children.map(_.parent) should have size 1 + children foreach (_.address.toString should ===(s"akka://${sysName}@localhost:${port}")) + masterSystem.stop(router) + } + + "deploy remote routers based on configuration" in { + val probe = TestProbe()(masterSystem) + val router = masterSystem.actorOf(FromConfig.props(echoActorProps), "remote-blub") + router.path.address.toString should ===(s"akka://${sysName}@localhost:${port}") + val replies = collectRouteePaths(probe, router, 5) + val children = replies.toSet + children should have size 2 + val parents = children.map(_.parent) + parents should have size 1 + parents.head should ===(router.path) + children foreach (_.address.toString should ===(s"akka://${sysName}@localhost:${port}")) + masterSystem.stop(router) + } + + "deploy remote routers based on explicit deployment" in { + val probe = TestProbe()(masterSystem) + val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps) + .withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka://${sysName}@localhost:${port}")))), "remote-blub2") + router.path.address.toString should ===(s"akka://${sysName}@localhost:${port}") + val replies = collectRouteePaths(probe, router, 5) + val children = replies.toSet + children should have size 2 + val parents = children.map(_.parent) + parents should have size 1 + parents.head should ===(router.path) + children foreach (_.address.toString should ===(s"akka://${sysName}@localhost:${port}")) + masterSystem.stop(router) + } + + "let remote deployment be overridden by local configuration" in { + val probe = TestProbe()(masterSystem) + val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps) + .withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka://${sysName}@localhost:${port}")))), "local-blub") + router.path.address.toString should ===("akka://MasterRemoteRouterSpec") + val replies = collectRouteePaths(probe, router, 5) + val children = replies.toSet + children should have size 2 + val parents = children.map(_.parent) + parents should have size 1 + parents.head.address should ===(Address("akka", sysName, "localhost", port)) + children foreach (_.address.toString should ===(s"akka://${sysName}@localhost:${port}")) + masterSystem.stop(router) + } + + "let remote deployment router be overridden by local configuration" in { + val probe = TestProbe()(masterSystem) + val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps) + .withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka://${sysName}@localhost:${port}")))), "local-blub2") + router.path.address.toString should ===(s"akka://${sysName}@localhost:${port}") + val replies = collectRouteePaths(probe, router, 5) + val children = replies.toSet + children should have size 4 + val parents = children.map(_.parent) + parents should have size 1 + parents.head should ===(router.path) + children foreach (_.address.toString should ===(s"akka://${sysName}@localhost:${port}")) + masterSystem.stop(router) + } + + "let remote deployment be overridden by remote configuration" in { + val probe = TestProbe()(masterSystem) + val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps) + .withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka://${sysName}@localhost:${port}")))), "remote-override") + router.path.address.toString should ===(s"akka://${sysName}@localhost:${port}") + val replies = collectRouteePaths(probe, router, 5) + val children = replies.toSet + children should have size 4 + val parents = children.map(_.parent) + parents should have size 1 + parents.head should ===(router.path) + children foreach (_.address.toString should ===(s"akka://${sysName}@localhost:${port}")) + masterSystem.stop(router) + } + + "set supplied supervisorStrategy" in { + val probe = TestProbe()(masterSystem) + val escalator = OneForOneStrategy() { + case e ⇒ probe.ref ! e; SupervisorStrategy.Escalate + } + val router = masterSystem.actorOf(new RemoteRouterConfig( + RoundRobinPool(1, supervisorStrategy = escalator), + Seq(Address("akka", sysName, "localhost", port))).props(Props.empty), "blub3") + + router.tell(GetRoutees, probe.ref) + EventFilter[ActorKilledException](occurrences = 1).intercept { + probe.expectMsgType[Routees].routees.head.send(Kill, testActor) + }(masterSystem) + probe.expectMsgType[ActorKilledException] + } + + "load settings from config for local router" in { + val probe = TestProbe()(masterSystem) + val router = masterSystem.actorOf(FromConfig.props(echoActorProps), "round") + val replies = collectRouteePaths(probe, router, 10) + val children = replies.toSet + children should have size 5 + masterSystem.stop(router) + } + + "load settings from config for local child router of system actor" in { + // we don't really support deployment configuration of system actors, but + // it's used for the pool of the SimpleDnsManager "/IO-DNS/inet-address" + val probe = TestProbe()(masterSystem) + val parent = masterSystem.asInstanceOf[ExtendedActorSystem].systemActorOf(Props[Parent], "sys-parent") + parent.tell((FromConfig.props(echoActorProps), "round"), probe.ref) + val router = probe.expectMsgType[ActorRef] + val replies = collectRouteePaths(probe, router, 10) + val children = replies.toSet + children should have size 6 + masterSystem.stop(router) + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala new file mode 100644 index 0000000000..2b6100be6a --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala @@ -0,0 +1,142 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.duration._ +import akka.actor.{ Actor, ActorIdentity, ActorSystem, Deploy, ExtendedActorSystem, Identify, Props, RootActorPath } +import akka.testkit.{ AkkaSpec, ImplicitSender } +import com.typesafe.config.ConfigFactory +import akka.actor.Actor.Receive +import akka.remote.RARP +import akka.testkit.TestActors +import akka.actor.PoisonPill +import akka.testkit.TestProbe +import akka.actor.ActorRef +import com.typesafe.config.Config + +object RemoteSendConsistencySpec { + + val config = ConfigFactory.parseString(s""" + akka { + actor.provider = remote + remote.artery.enabled = on + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 + } + """) + +} + +class RemoteSendConsistencySpec extends AbstractRemoteSendConsistencySpec(RemoteSendConsistencySpec.config) + +class RemoteSendConsistencyWithThreeLanesSpec extends AbstractRemoteSendConsistencySpec( + ConfigFactory.parseString(""" + akka.remote.artery.advanced.outbound-lanes = 3 + akka.remote.artery.advanced.inbound-lanes = 3 + """).withFallback(RemoteSendConsistencySpec.config)) + +abstract class AbstractRemoteSendConsistencySpec(config: Config) extends AkkaSpec(config) with ImplicitSender { + + val systemB = ActorSystem("systemB", system.settings.config) + val addressB = RARP(systemB).provider.getDefaultAddress + println(addressB) + val rootB = RootActorPath(addressB) + + "Artery" must { + + "be able to identify a remote actor and ping it" in { + val actorOnSystemB = systemB.actorOf(Props(new Actor { + def receive = { + case "ping" ⇒ sender() ! "pong" + } + }), "echo") + + val remoteRef = { + system.actorSelection(rootB / "user" / "echo") ! Identify(None) + expectMsgType[ActorIdentity](5.seconds).ref.get + } + + remoteRef ! "ping" + expectMsg("pong") + + remoteRef ! "ping" + expectMsg("pong") + + remoteRef ! "ping" + expectMsg("pong") + } + + "not send to remote re-created actor with same name" in { + val echo = systemB.actorOf(TestActors.echoActorProps, "otherEcho1") + echo ! 71 + expectMsg(71) + echo ! PoisonPill + echo ! 72 + val probe = TestProbe()(systemB) + probe.watch(echo) + probe.expectTerminated(echo) + expectNoMsg(1.second) + + val echo2 = systemB.actorOf(TestActors.echoActorProps, "otherEcho1") + echo2 ! 73 + expectMsg(73) + // msg to old ActorRef (different uid) should not get through + echo2.path.uid should not be (echo.path.uid) + echo ! 74 + expectNoMsg(1.second) + } + + "be able to send messages concurrently preserving order" in { + systemB.actorOf(TestActors.echoActorProps, "echoA") + systemB.actorOf(TestActors.echoActorProps, "echoB") + systemB.actorOf(TestActors.echoActorProps, "echoC") + + val remoteRefA = { + system.actorSelection(rootB / "user" / "echoA") ! Identify(None) + expectMsgType[ActorIdentity].ref.get + } + val remoteRefB = { + system.actorSelection(rootB / "user" / "echoB") ! Identify(None) + expectMsgType[ActorIdentity].ref.get + } + val remoteRefC = { + system.actorSelection(rootB / "user" / "echoC") ! Identify(None) + expectMsgType[ActorIdentity].ref.get + } + + def senderProps(remoteRef: ActorRef) = Props(new Actor { + var counter = 1000 + remoteRef ! counter + + override def receive: Receive = { + case i: Int ⇒ + if (i != counter) testActor ! s"Failed, expected $counter got $i" + else if (counter == 0) { + testActor ! "success" + context.stop(self) + } else { + counter -= 1 + remoteRef ! counter + } + } + }).withDeploy(Deploy.local) + + system.actorOf(senderProps(remoteRefA)) + system.actorOf(senderProps(remoteRefB)) + system.actorOf(senderProps(remoteRefC)) + system.actorOf(senderProps(remoteRefA)) + + within(10.seconds) { + expectMsg("success") + expectMsg("success") + expectMsg("success") + expectMsg("success") + } + } + + } + + override def afterTermination(): Unit = shutdown(systemB) + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala new file mode 100644 index 0000000000..dc236b272f --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala @@ -0,0 +1,321 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ +package akka.remote.artery + +import language.postfixOps +import scala.concurrent.duration._ +import akka.testkit._ +import akka.actor._ +import akka.remote._ + +object RemoteWatcherSpec { + + class TestActorProxy(testActor: ActorRef) extends Actor { + def receive = { + case msg ⇒ testActor forward msg + } + } + + class MyActor extends Actor { + def receive = Actor.emptyBehavior + } + + // turn off all periodic activity + val TurnOff = 5.minutes + + def createFailureDetector(): FailureDetectorRegistry[Address] = { + def createFailureDetector(): FailureDetector = + new PhiAccrualFailureDetector( + threshold = 8.0, + maxSampleSize = 200, + minStdDeviation = 100.millis, + acceptableHeartbeatPause = 3.seconds, + firstHeartbeatEstimate = 1.second) + + new DefaultFailureDetectorRegistry(() ⇒ createFailureDetector()) + } + + object TestRemoteWatcher { + final case class AddressTerm(address: Address) extends JavaSerializable + final case class Quarantined(address: Address, uid: Option[Long]) extends JavaSerializable + } + + class TestRemoteWatcher(heartbeatExpectedResponseAfter: FiniteDuration) extends RemoteWatcher( + createFailureDetector, + heartbeatInterval = TurnOff, + unreachableReaperInterval = TurnOff, + heartbeatExpectedResponseAfter = heartbeatExpectedResponseAfter) { + + def this() = this(heartbeatExpectedResponseAfter = TurnOff) + + override def publishAddressTerminated(address: Address): Unit = + // don't publish the real AddressTerminated, but a testable message, + // that doesn't interfere with the real watch that is going on in the background + context.system.eventStream.publish(TestRemoteWatcher.AddressTerm(address)) + + override def quarantine(address: Address, uid: Option[Long], reason: String): Unit = { + // don't quarantine in remoting, but publish a testable message + context.system.eventStream.publish(TestRemoteWatcher.Quarantined(address, uid)) + } + + } + +} + +class RemoteWatcherSpec extends AkkaSpec( + """akka { + loglevel = INFO + log-dead-letters-during-shutdown = false + actor.provider = remote + remote.artery.enabled = on + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 + }""") with ImplicitSender { + + import RemoteWatcherSpec._ + import RemoteWatcher._ + + override def expectedTestDuration = 2.minutes + + val remoteSystem = ActorSystem("RemoteSystem", system.settings.config) + val remoteAddress = RARP(remoteSystem).provider.getDefaultAddress + def remoteAddressUid = AddressUidExtension(remoteSystem).longAddressUid + + Seq(system, remoteSystem).foreach(muteDeadLetters( + akka.remote.transport.AssociationHandle.Disassociated.getClass, + akka.remote.transport.ActorTransportAdapter.DisassociateUnderlying.getClass)(_)) + + override def afterTermination() { + shutdown(remoteSystem) + } + + val heartbeatRspB = ArteryHeartbeatRsp(remoteAddressUid) + + def createRemoteActor(props: Props, name: String): InternalActorRef = { + remoteSystem.actorOf(props, name) + system.actorSelection(RootActorPath(remoteAddress) / "user" / name) ! Identify(name) + expectMsgType[ActorIdentity].ref.get.asInstanceOf[InternalActorRef] + } + + "A RemoteWatcher" must { + + "have correct interaction when watching" in { + + val fd = createFailureDetector() + val monitorA = system.actorOf(Props[TestRemoteWatcher], "monitor1") + val monitorB = createRemoteActor(Props(classOf[TestActorProxy], testActor), "monitor1") + + val a1 = system.actorOf(Props[MyActor], "a1").asInstanceOf[InternalActorRef] + val a2 = system.actorOf(Props[MyActor], "a2").asInstanceOf[InternalActorRef] + val b1 = createRemoteActor(Props[MyActor], "b1") + val b2 = createRemoteActor(Props[MyActor], "b2") + + monitorA ! WatchRemote(b1, a1) + monitorA ! WatchRemote(b2, a1) + monitorA ! WatchRemote(b2, a2) + monitorA ! Stats + // (a1->b1), (a1->b2), (a2->b2) + expectMsg(Stats.counts(watching = 3, watchingNodes = 1)) + expectNoMsg(100 millis) + monitorA ! HeartbeatTick + expectMsg(ArteryHeartbeat) + expectNoMsg(100 millis) + monitorA ! HeartbeatTick + expectMsg(ArteryHeartbeat) + expectNoMsg(100 millis) + monitorA.tell(heartbeatRspB, monitorB) + monitorA ! HeartbeatTick + expectMsg(ArteryHeartbeat) + expectNoMsg(100 millis) + + monitorA ! UnwatchRemote(b1, a1) + // still (a1->b2) and (a2->b2) left + monitorA ! Stats + expectMsg(Stats.counts(watching = 2, watchingNodes = 1)) + expectNoMsg(100 millis) + monitorA ! HeartbeatTick + expectMsg(ArteryHeartbeat) + expectNoMsg(100 millis) + + monitorA ! UnwatchRemote(b2, a2) + // still (a1->b2) left + monitorA ! Stats + expectMsg(Stats.counts(watching = 1, watchingNodes = 1)) + expectNoMsg(100 millis) + monitorA ! HeartbeatTick + expectMsg(ArteryHeartbeat) + expectNoMsg(100 millis) + + monitorA ! UnwatchRemote(b2, a1) + // all unwatched + monitorA ! Stats + expectMsg(Stats.empty) + expectNoMsg(100 millis) + monitorA ! HeartbeatTick + expectNoMsg(100 millis) + monitorA ! HeartbeatTick + expectNoMsg(100 millis) + + // make sure nothing floods over to next test + expectNoMsg(2 seconds) + } + + "generate AddressTerminated when missing heartbeats" in { + val p = TestProbe() + val q = TestProbe() + system.eventStream.subscribe(p.ref, classOf[TestRemoteWatcher.AddressTerm]) + system.eventStream.subscribe(q.ref, classOf[TestRemoteWatcher.Quarantined]) + + val monitorA = system.actorOf(Props[TestRemoteWatcher], "monitor4") + val monitorB = createRemoteActor(Props(classOf[TestActorProxy], testActor), "monitor4") + + val a = system.actorOf(Props[MyActor], "a4").asInstanceOf[InternalActorRef] + val b = createRemoteActor(Props[MyActor], "b4") + + monitorA ! WatchRemote(b, a) + + monitorA ! HeartbeatTick + expectMsg(ArteryHeartbeat) + monitorA.tell(heartbeatRspB, monitorB) + expectNoMsg(1 second) + monitorA ! HeartbeatTick + expectMsg(ArteryHeartbeat) + monitorA.tell(heartbeatRspB, monitorB) + + within(10 seconds) { + awaitAssert { + monitorA ! HeartbeatTick + expectMsg(ArteryHeartbeat) + // but no HeartbeatRsp + monitorA ! ReapUnreachableTick + p.expectMsg(1 second, TestRemoteWatcher.AddressTerm(b.path.address)) + q.expectMsg(1 second, TestRemoteWatcher.Quarantined(b.path.address, Some(remoteAddressUid))) + } + } + + // make sure nothing floods over to next test + expectNoMsg(2 seconds) + } + + "generate AddressTerminated when missing first heartbeat" in { + val p = TestProbe() + val q = TestProbe() + system.eventStream.subscribe(p.ref, classOf[TestRemoteWatcher.AddressTerm]) + system.eventStream.subscribe(q.ref, classOf[TestRemoteWatcher.Quarantined]) + + val fd = createFailureDetector() + val heartbeatExpectedResponseAfter = 2.seconds + val monitorA = system.actorOf(Props(classOf[TestRemoteWatcher], heartbeatExpectedResponseAfter), "monitor5") + val monitorB = createRemoteActor(Props(classOf[TestActorProxy], testActor), "monitor5") + + val a = system.actorOf(Props[MyActor], "a5").asInstanceOf[InternalActorRef] + val b = createRemoteActor(Props[MyActor], "b5") + + monitorA ! WatchRemote(b, a) + + monitorA ! HeartbeatTick + expectMsg(ArteryHeartbeat) + // no HeartbeatRsp sent + + within(20 seconds) { + awaitAssert { + monitorA ! HeartbeatTick + expectMsg(ArteryHeartbeat) + // but no HeartbeatRsp + monitorA ! ReapUnreachableTick + p.expectMsg(1 second, TestRemoteWatcher.AddressTerm(b.path.address)) + // no real quarantine when missing first heartbeat, uid unknown + q.expectMsg(1 second, TestRemoteWatcher.Quarantined(b.path.address, None)) + } + } + + // make sure nothing floods over to next test + expectNoMsg(2 seconds) + } + + "generate AddressTerminated for new watch after broken connection that was re-established and broken again" in { + val p = TestProbe() + val q = TestProbe() + system.eventStream.subscribe(p.ref, classOf[TestRemoteWatcher.AddressTerm]) + system.eventStream.subscribe(q.ref, classOf[TestRemoteWatcher.Quarantined]) + + val monitorA = system.actorOf(Props[TestRemoteWatcher], "monitor6") + val monitorB = createRemoteActor(Props(classOf[TestActorProxy], testActor), "monitor6") + + val a = system.actorOf(Props[MyActor], "a6").asInstanceOf[InternalActorRef] + val b = createRemoteActor(Props[MyActor], "b6") + + monitorA ! WatchRemote(b, a) + + monitorA ! HeartbeatTick + expectMsg(ArteryHeartbeat) + monitorA.tell(heartbeatRspB, monitorB) + expectNoMsg(1 second) + monitorA ! HeartbeatTick + expectMsg(ArteryHeartbeat) + monitorA.tell(heartbeatRspB, monitorB) + + within(10 seconds) { + awaitAssert { + monitorA ! HeartbeatTick + expectMsg(ArteryHeartbeat) + // but no HeartbeatRsp + monitorA ! ReapUnreachableTick + p.expectMsg(1 second, TestRemoteWatcher.AddressTerm(b.path.address)) + q.expectMsg(1 second, TestRemoteWatcher.Quarantined(b.path.address, Some(remoteAddressUid))) + } + } + + // real AddressTerminated would trigger Terminated for b6, simulate that here + remoteSystem.stop(b) + awaitAssert { + monitorA ! Stats + expectMsg(Stats.empty) + } + expectNoMsg(2 seconds) + + // assume that connection comes up again, or remote system is restarted + val c = createRemoteActor(Props[MyActor], "c6") + + monitorA ! WatchRemote(c, a) + + monitorA ! HeartbeatTick + expectMsg(ArteryHeartbeat) + monitorA.tell(heartbeatRspB, monitorB) + expectNoMsg(1 second) + monitorA ! HeartbeatTick + expectMsg(ArteryHeartbeat) + monitorA.tell(heartbeatRspB, monitorB) + monitorA ! HeartbeatTick + expectMsg(ArteryHeartbeat) + monitorA ! ReapUnreachableTick + p.expectNoMsg(1 second) + monitorA ! HeartbeatTick + expectMsg(ArteryHeartbeat) + monitorA.tell(heartbeatRspB, monitorB) + monitorA ! HeartbeatTick + expectMsg(ArteryHeartbeat) + monitorA ! ReapUnreachableTick + p.expectNoMsg(1 second) + q.expectNoMsg(1 second) + + // then stop heartbeating again, should generate new AddressTerminated + within(10 seconds) { + awaitAssert { + monitorA ! HeartbeatTick + expectMsg(ArteryHeartbeat) + // but no HeartbeatRsp + monitorA ! ReapUnreachableTick + p.expectMsg(1 second, TestRemoteWatcher.AddressTerm(c.path.address)) + q.expectMsg(1 second, TestRemoteWatcher.Quarantined(c.path.address, Some(remoteAddressUid))) + } + } + + // make sure nothing floods over to next test + expectNoMsg(2 seconds) + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/RestartCounterSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RestartCounterSpec.scala new file mode 100644 index 0000000000..c0ac272975 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RestartCounterSpec.scala @@ -0,0 +1,42 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.duration._ + +import org.scalatest.Matchers +import org.scalatest.WordSpec + +class RestartCounterSpec extends WordSpec with Matchers { + + "RestartCounter" must { + + "count max restarts within duration" in { + val counter = new RestartCounter(3, 3.seconds) + counter.restart() should ===(true) + counter.restart() should ===(true) + counter.restart() should ===(true) + counter.restart() should ===(false) + counter.count() should ===(4) + } + + "allow sporadic restarts" in { + val counter = new RestartCounter(3, 10.millis) + for (_ ← 1 to 10) { + counter.restart() should ===(true) + Thread.sleep(20) + } + } + + "reset count after timeout" in { + val counter = new RestartCounter(3, 500.millis) + counter.restart() + counter.restart() + counter.count() should ===(2) + Thread.sleep(600) + counter.restart() + counter.count() should ===(1) + } + } +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/RollingEventLogSimulationSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RollingEventLogSimulationSpec.scala new file mode 100644 index 0000000000..fe5e4e55dc --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RollingEventLogSimulationSpec.scala @@ -0,0 +1,215 @@ +package akka.remote.artery + +import akka.testkit.AkkaSpec + +import scala.annotation.tailrec +import scala.util.Random +import scala.util.control.NonFatal + +/* + * This test is a simulation of the actual concurrent rolling log implemented in SnapshottableRollingEventLog. It + * is not possible to test the implementation to such extent than this simulation allows, however, the two implementations + * must be kept in sync manually (it is expected to remain simple though). + * + * It is very important to not get corrupted results from the Flight Recorder as they can lead to completely misinterpreted + * results when debugging using the logs. This simulation tries to uncover many race scenarios by simulating the + * algorithm down to the individual byte write level. + */ +class RollingEventLogSimulationSpec extends AkkaSpec { + + val Committed: Byte = 0.toByte + val Dirty: Byte = 1.toByte + + val letterCodes = Array("A", "B", "C", "D", "E", "F") + val EntrySize = 4 + + class Writer(writerId: Int, entryCount: Int, totalWrites: Int) { + val letterCode = letterCodes(writerId) + val bufSize = entryCount * EntrySize + + // Models an instruction that does read or write to some shared location + sealed trait Instruction { + def apply(simulator: Simulator): String + } + + // getAndIncrement on the header and store it in local variable + case object AdvanceHeader extends Instruction { + override def apply(simulator: Simulator): String = { + seenHeader = simulator.headPointer + slot = seenHeader % bufSize + simulator.headPointer += EntrySize + writePointer = slot + 1 // Leave one byte for the commit header + advance() + s"$letterCode sees header $seenHeader advances it to ${simulator.headPointer}" + } + } + + // CAS on the commit status field, if fails jump to start of loop + case object TryMarkDirty extends Instruction { + override def apply(simulator: Simulator): String = { + if (simulator.simulatedBuffer(slot) == Dirty) { + instructionPtr = 0 // Retry loop + s"$letterCode sees dirty record at $seenHeader, retries" + } else { + simulator.simulatedBuffer(slot) = Dirty + advance() + s"$letterCode sees committed record at $seenHeader, proceeds" + } + } + } + + // This step is just to be able to do consistency checks. Simply writes the ID of the writer as the first + // byte of the record. + case object WriteId extends Instruction { + override def apply(simulator: Simulator): String = { + simulator.simulatedBuffer(writePointer) = writerId.toByte // Avoid zero since we start from zeroed buf + writePointer += 1 + advance() + s"$letterCode writes ID to offset ${writePointer - 1}" + } + } + + // Write an individual byte to the record. Visibility issues are not modeled, but they are likely relevant + // since writing Commit will be the proper barrier anyway. + case object WriteByte extends Instruction { + override def apply(simulator: Simulator): String = { + simulator.simulatedBuffer(writePointer) = (writeCount + 1).toByte // Avoid zero since we start from zeroed buf + writePointer += 1 + advance() + s"$letterCode writes byte ${writeCount + 1} to offset ${writePointer - 1}" + } + } + + // Sets the commit status to Committed + case object Commit extends Instruction { + override def apply(simulator: Simulator): String = { + simulator.simulatedBuffer(slot) = Committed + advance() + s"$letterCode commits at $seenHeader" + } + } + + var instructionPtr = 0 + var writeCount = 0 + var seenHeader = 0 + var slot = 0 + var writePointer = 0 + + val instructions: Array[Instruction] = + (Array(AdvanceHeader, TryMarkDirty) :+ + WriteId) ++ + Array.fill(EntrySize - 2)(WriteByte) :+ + Commit + + def step(simulator: Simulator): String = { + instructions(instructionPtr)(simulator) + } + + private def advance(): Unit = { + instructionPtr += 1 + if (instructionPtr == instructions.size) { + instructionPtr = 0 + writeCount += 1 + } + } + + def isFinished: Boolean = writeCount == totalWrites + + } + + class Simulator(writerCount: Int, entryCount: Int, totalWrites: Int) { + var headPointer = 0 + val simulatedBuffer = Array.ofDim[Byte](4 * entryCount) + val writers = Array.tabulate(writerCount)(new Writer(_, entryCount, totalWrites)) + var activeWriters = writerCount + var log: List[String] = Nil + + @tailrec private def chooseWriter: Writer = { + val idx = Random.nextInt(writerCount) + val writer = writers(idx) + if (writer.isFinished) chooseWriter + else writer + } + + def run(): Unit = { + try { + while (activeWriters > 0) { + val writer = chooseWriter + val event = writer.step(this) + log ::= event + if (writer.isFinished) activeWriters -= 1 + consistencyChecks() + } + } catch { + case NonFatal(e) ⇒ + println(log.reverse.mkString("\n")) + println("----------- BUFFER CONTENT -------------") + println(simulatedBuffer.grouped(EntrySize).map(_.mkString("[", ",", "]")).mkString(", ")) + throw e + } + allRecordsCommitted() + } + + def consistencyChecks(): Unit = { + checkNoPartialWrites() + checkGaplessWrites() + } + + // No Committed records should contain bytes from two different writes (Dirty records might, though). + def checkNoPartialWrites(): Unit = { + for (entry ← 0 until entryCount if simulatedBuffer(entry * EntrySize) == Committed) { + val ofs = entry * EntrySize + if (simulatedBuffer(ofs + 2) != simulatedBuffer(ofs + 3)) + fail(s"Entry $entry is corrupted, partial writes are visible") + } + } + + // All writes for a given ID must: + // - contain the last write, or no writes at all + // - any writes in addition to the last write should be gapless (but possibly truncated) + // good examples (showing the write numbers, assuming latest is 4): + // [2, 3, 4] + // [4] + // [] + // [3, 4] + // bad examples + // [2, 3] + // [2, 4] + def checkGaplessWrites(): Unit = { + for (id ← 0 until writerCount) { + val writeCount = writers(id).writeCount + val lastWrittenSlot = (headPointer - EntrySize) % EntrySize + var nextExpected = writeCount + val totalWrittenEntries = headPointer % EntrySize + + for (i ← 0 until math.min(entryCount, totalWrittenEntries)) { + val slot = (entryCount + lastWrittenSlot - i) % entryCount + val offs = slot * EntrySize + if (simulatedBuffer(offs) == Committed && simulatedBuffer(offs + 1) == id) { + if (simulatedBuffer(offs + 2) != nextExpected) + fail(s"Entry $slot is corrupted, contains write ${simulatedBuffer(offs + 2)} but expected $nextExpected") + nextExpected -= 1 + } + } + } + } + + def allRecordsCommitted(): Unit = { + for (entry ← 0 until entryCount) { + if (simulatedBuffer(entry * EntrySize) != Committed) + fail(s"Entry $entry is not Committed") + } + } + } + + "RollingEventLog algorithm" must { + + "ensure write consistency in simulation" in { + // 600 record writes, roughly 3600 instructions in total, racing for 32 memory locations (plus the head pointer) + val sim = new Simulator(writerCount = 6, entryCount = 8, totalWrites = 100) + sim.run() + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/SendQueueSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SendQueueSpec.scala new file mode 100644 index 0000000000..1655432cdc --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/SendQueueSpec.scala @@ -0,0 +1,152 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.Queue + +import scala.concurrent.duration._ + +import akka.actor.Actor +import akka.actor.Props +import akka.stream.ActorMaterializer +import akka.stream.ActorMaterializerSettings +import akka.stream.scaladsl.Keep +import akka.stream.scaladsl.Source +import akka.stream.testkit.scaladsl.TestSink +import akka.testkit.AkkaSpec +import akka.testkit.ImplicitSender +import org.agrona.concurrent.ManyToOneConcurrentArrayQueue + +object SendQueueSpec { + + case class ProduceToQueue(from: Int, until: Int, queue: Queue[Msg]) + case class ProduceToQueueValue(from: Int, until: Int, queue: SendQueue.QueueValue[Msg]) + case class Msg(fromProducer: String, value: Int) + + def producerProps(producerId: String): Props = + Props(new Producer(producerId)) + + class Producer(producerId: String) extends Actor { + def receive = { + case ProduceToQueue(from, until, queue) ⇒ + var i = from + while (i < until) { + if (!queue.offer(Msg(producerId, i))) + throw new IllegalStateException(s"offer failed from $producerId value $i") + i += 1 + } + case ProduceToQueueValue(from, until, queue) ⇒ + var i = from + while (i < until) { + if (!queue.offer(Msg(producerId, i))) + throw new IllegalStateException(s"offer failed from $producerId value $i") + i += 1 + } + } + + } +} + +class SendQueueSpec extends AkkaSpec("akka.actor.serialize-messages = off") with ImplicitSender { + import SendQueueSpec._ + + val matSettings = ActorMaterializerSettings(system).withFuzzing(true) + implicit val mat = ActorMaterializer(matSettings)(system) + + "SendQueue" must { + + "deliver all messages" in { + val queue = new ManyToOneConcurrentArrayQueue[String](128) + val (sendQueue, downstream) = Source.fromGraph(new SendQueue[String]) + .toMat(TestSink.probe)(Keep.both).run() + + downstream.request(10) + sendQueue.inject(queue) + sendQueue.offer("a") + sendQueue.offer("b") + sendQueue.offer("c") + downstream.expectNext("a") + downstream.expectNext("b") + downstream.expectNext("c") + downstream.cancel() + } + + "deliver messages enqueued before materialization" in { + val queue = new ManyToOneConcurrentArrayQueue[String](128) + queue.offer("a") + queue.offer("b") + + val (sendQueue, downstream) = Source.fromGraph(new SendQueue[String]) + .toMat(TestSink.probe)(Keep.both).run() + + downstream.request(10) + downstream.expectNoMsg(200.millis) + sendQueue.inject(queue) + downstream.expectNext("a") + downstream.expectNext("b") + + sendQueue.offer("c") + downstream.expectNext("c") + downstream.cancel() + } + + "deliver bursts of messages" in { + // this test verifies that the wakeup signal is triggered correctly + val queue = new ManyToOneConcurrentArrayQueue[Int](128) + val burstSize = 100 + val (sendQueue, downstream) = Source.fromGraph(new SendQueue[Int]) + .grouped(burstSize) + .async + .toMat(TestSink.probe)(Keep.both).run() + + downstream.request(10) + sendQueue.inject(queue) + + for (round ← 1 to 100000) { + for (n ← 1 to burstSize) { + if (!sendQueue.offer(round * 1000 + n)) + fail(s"offer failed at round $round message $n") + } + downstream.expectNext((1 to burstSize).map(_ + round * 1000).toList) + downstream.request(1) + } + + downstream.cancel() + } + + "support multiple producers" in { + val numberOfProducers = 5 + val queue = new ManyToOneConcurrentArrayQueue[Msg](numberOfProducers * 512) + val producers = Vector.tabulate(numberOfProducers)(i ⇒ system.actorOf(producerProps(s"producer-$i"))) + + // send 100 per producer before materializing + producers.foreach(_ ! ProduceToQueue(0, 100, queue)) + + val (sendQueue, downstream) = Source.fromGraph(new SendQueue[Msg]) + .toMat(TestSink.probe)(Keep.both).run() + + sendQueue.inject(queue) + producers.foreach(_ ! ProduceToQueueValue(100, 200, sendQueue)) + + // send 100 more per producer + downstream.request(producers.size * 200) + val msgByProducer = downstream.expectNextN(producers.size * 200).groupBy(_.fromProducer) + (0 until producers.size).foreach { i ⇒ + msgByProducer(s"producer-$i").map(_.value) should ===(0 until 200) + } + + // send 500 per producer + downstream.request(producers.size * 1000) // more than enough + producers.foreach(_ ! ProduceToQueueValue(200, 700, sendQueue)) + val msgByProducer2 = downstream.expectNextN(producers.size * 500).groupBy(_.fromProducer) + (0 until producers.size).foreach { i ⇒ + msgByProducer2(s"producer-$i").map(_.value) should ===(200 until 700) + } + + downstream.cancel() + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala new file mode 100644 index 0000000000..63b19320d4 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala @@ -0,0 +1,88 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.duration._ +import akka.actor.{ ActorIdentity, ActorSystem, ExtendedActorSystem, Identify, RootActorPath } +import akka.remote.RARP +import akka.testkit.{ AkkaSpec, ImplicitSender } +import akka.testkit.TestActors +import com.typesafe.config.ConfigFactory +import akka.testkit.EventFilter + +object SerializationErrorSpec { + + val config = ConfigFactory.parseString(s""" + akka { + actor.provider = remote + remote.artery.enabled = on + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 + actor { + serialize-creators = false + serialize-messages = false + } + } + """) + + object NotSerializableMsg + +} + +class SerializationErrorSpec extends AkkaSpec(SerializationErrorSpec.config) with ImplicitSender { + import SerializationErrorSpec._ + + val configB = ConfigFactory.parseString(""" + akka.actor.serialization-identifiers { + # this will cause deserialization error + "akka.serialization.ByteArraySerializer" = -4 + } + """).withFallback(system.settings.config) + val systemB = ActorSystem("systemB", configB) + systemB.actorOf(TestActors.echoActorProps, "echo") + val addressB = RARP(systemB).provider.getDefaultAddress + val rootB = RootActorPath(addressB) + + override def afterTermination(): Unit = shutdown(systemB) + + "Serialization error" must { + + "be logged when serialize fails" in { + val remoteRef = { + system.actorSelection(rootB / "user" / "echo") ! Identify(None) + expectMsgType[ActorIdentity].ref.get + } + + remoteRef ! "ping" + expectMsg("ping") + + EventFilter[java.io.NotSerializableException](start = "Failed to serialize message", occurrences = 1).intercept { + remoteRef ! NotSerializableMsg + } + + remoteRef ! "ping2" + expectMsg("ping2") + } + + "be logged when deserialize fails" in { + val remoteRef = { + system.actorSelection(rootB / "user" / "echo") ! Identify(None) + expectMsgType[ActorIdentity].ref.get + } + + remoteRef ! "ping" + expectMsg("ping") + + EventFilter.warning( + start = "Failed to deserialize message with serializer id [4]", occurrences = 1).intercept { + remoteRef ! "boom".getBytes("utf-8") + }(systemB) + + remoteRef ! "ping2" + expectMsg("ping2") + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageAckerSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageAckerSpec.scala new file mode 100644 index 0000000000..fd0d44fdf1 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageAckerSpec.scala @@ -0,0 +1,132 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.duration._ + +import akka.actor.Address +import akka.remote.UniqueAddress +import akka.remote.artery.SystemMessageDelivery._ +import akka.stream.ActorMaterializer +import akka.stream.ActorMaterializerSettings +import akka.stream.scaladsl.Keep +import akka.stream.testkit.TestPublisher +import akka.stream.testkit.TestSubscriber +import akka.stream.testkit.scaladsl.TestSink +import akka.stream.testkit.scaladsl.TestSource +import akka.testkit.AkkaSpec +import akka.testkit.ImplicitSender +import akka.testkit.TestProbe +import akka.util.OptionVal + +class SystemMessageAckerSpec extends AkkaSpec with ImplicitSender { + + val matSettings = ActorMaterializerSettings(system).withFuzzing(true) + implicit val mat = ActorMaterializer(matSettings)(system) + + val addressA = UniqueAddress(Address("akka", "sysA", "hostA", 1001), 1) + val addressB = UniqueAddress(Address("akka", "sysB", "hostB", 1002), 2) + val addressC = UniqueAddress(Address("akka", "sysC", "hostB", 1003), 3) + + private def setupStream(inboundContext: InboundContext, timeout: FiniteDuration = 5.seconds): (TestPublisher.Probe[AnyRef], TestSubscriber.Probe[Any]) = { + val recipient = OptionVal.None // not used + TestSource.probe[AnyRef] + .map { + case sysMsg @ SystemMessageEnvelope(_, _, ackReplyTo) ⇒ + InboundEnvelope(recipient, sysMsg, OptionVal.None, ackReplyTo.uid, + inboundContext.association(ackReplyTo.uid)) + } + .via(new SystemMessageAcker(inboundContext)) + .map { case env: InboundEnvelope ⇒ env.message } + .toMat(TestSink.probe[Any])(Keep.both) + .run() + } + + "SystemMessageAcker stage" must { + + "send Ack for expected message" in { + val replyProbe = TestProbe() + val inboundContext = new TestInboundContext(addressA, controlProbe = Some(replyProbe.ref)) + val (upstream, downstream) = setupStream(inboundContext) + + downstream.request(10) + upstream.sendNext(SystemMessageEnvelope("b1", 1, addressB)) + replyProbe.expectMsg(Ack(1, addressA)) + upstream.sendNext(SystemMessageEnvelope("b2", 2, addressB)) + replyProbe.expectMsg(Ack(2, addressA)) + downstream.cancel() + } + + "send Ack for duplicate message" in { + val replyProbe = TestProbe() + val inboundContext = new TestInboundContext(addressA, controlProbe = Some(replyProbe.ref)) + val (upstream, downstream) = setupStream(inboundContext) + + downstream.request(10) + upstream.sendNext(SystemMessageEnvelope("b1", 1, addressB)) + replyProbe.expectMsg(Ack(1, addressA)) + upstream.sendNext(SystemMessageEnvelope("b2", 2, addressB)) + replyProbe.expectMsg(Ack(2, addressA)) + upstream.sendNext(SystemMessageEnvelope("b1", 1, addressB)) + replyProbe.expectMsg(Ack(2, addressA)) + downstream.cancel() + } + + "send Nack for unexpected message" in { + val replyProbe = TestProbe() + val inboundContext = new TestInboundContext(addressA, controlProbe = Some(replyProbe.ref)) + val (upstream, downstream) = setupStream(inboundContext) + + downstream.request(10) + upstream.sendNext(SystemMessageEnvelope("b1", 1, addressB)) + replyProbe.expectMsg(Ack(1, addressA)) + upstream.sendNext(SystemMessageEnvelope("b3", 3, addressB)) + replyProbe.expectMsg(Nack(1, addressA)) + downstream.cancel() + } + + "send Nack for unexpected first message" in { + val replyProbe = TestProbe() + val inboundContext = new TestInboundContext(addressA, controlProbe = Some(replyProbe.ref)) + val (upstream, downstream) = setupStream(inboundContext) + + downstream.request(10) + upstream.sendNext(SystemMessageEnvelope("b2", 2, addressB)) + replyProbe.expectMsg(Nack(0, addressA)) + downstream.cancel() + } + + "keep track of sequence numbers per sending system" in { + val replyProbe = TestProbe() + val inboundContext = new TestInboundContext(addressA, controlProbe = Some(replyProbe.ref)) + val (upstream, downstream) = setupStream(inboundContext) + + downstream.request(10) + upstream.sendNext(SystemMessageEnvelope("b1", 1, addressB)) + replyProbe.expectMsg(Ack(1, addressA)) + upstream.sendNext(SystemMessageEnvelope("b2", 2, addressB)) + replyProbe.expectMsg(Ack(2, addressA)) + + upstream.sendNext(SystemMessageEnvelope("c1", 1, addressC)) + replyProbe.expectMsg(Ack(1, addressA)) + upstream.sendNext(SystemMessageEnvelope("c3", 3, addressC)) + replyProbe.expectMsg(Nack(1, addressA)) + upstream.sendNext(SystemMessageEnvelope("c2", 2, addressC)) + replyProbe.expectMsg(Ack(2, addressA)) + upstream.sendNext(SystemMessageEnvelope("c3", 3, addressC)) + replyProbe.expectMsg(Ack(3, addressA)) + upstream.sendNext(SystemMessageEnvelope("c4", 4, addressC)) + replyProbe.expectMsg(Ack(4, addressA)) + + upstream.sendNext(SystemMessageEnvelope("b4", 4, addressB)) + replyProbe.expectMsg(Nack(2, addressA)) + upstream.sendNext(SystemMessageEnvelope("b3", 3, addressB)) + replyProbe.expectMsg(Ack(3, addressA)) + + downstream.cancel() + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala new file mode 100644 index 0000000000..a702272148 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -0,0 +1,285 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.concurrent.ThreadLocalRandom + +import scala.concurrent.Await +import scala.concurrent.duration._ +import akka.NotUsed +import akka.actor.ActorIdentity +import akka.actor.ActorSystem +import akka.actor.Identify +import akka.actor.PoisonPill +import akka.actor.RootActorPath +import akka.remote.{ AddressUidExtension, RARP, UniqueAddress } +import akka.remote.artery.SystemMessageDelivery._ +import akka.stream.ActorMaterializer +import akka.stream.ActorMaterializerSettings +import akka.stream.ThrottleMode +import akka.stream.scaladsl.Flow +import akka.stream.scaladsl.Sink +import akka.stream.scaladsl.Source +import akka.stream.testkit.scaladsl.TestSink +import akka.testkit.AkkaSpec +import akka.testkit.ImplicitSender +import akka.testkit.TestActors +import akka.testkit.TestProbe +import com.typesafe.config.ConfigFactory +import akka.util.OptionVal + +object SystemMessageDeliverySpec { + + val config = ConfigFactory.parseString(s""" + akka.loglevel=INFO + akka { + actor.provider = remote + remote.artery.enabled = on + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 + } + akka.actor.serialize-creators = off + akka.actor.serialize-messages = off + """) + + case class TestSysMsg(s: String) extends SystemMessageDelivery.AckedDeliveryMessage + +} + +class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.config) with ImplicitSender { + import SystemMessageDeliverySpec._ + + val addressA = UniqueAddress( + RARP(system).provider.getDefaultAddress, + AddressUidExtension(system).longAddressUid) + val systemB = ActorSystem("systemB", system.settings.config) + val addressB = UniqueAddress( + RARP(systemB).provider.getDefaultAddress, + AddressUidExtension(systemB).longAddressUid) + val rootB = RootActorPath(addressB.address) + val matSettings = ActorMaterializerSettings(system).withFuzzing(true) + implicit val mat = ActorMaterializer(matSettings)(system) + + private val outboundEnvelopePool = ReusableOutboundEnvelope.createObjectPool(capacity = 16) + + override def afterTermination(): Unit = shutdown(systemB) + + private def send(sendCount: Int, resendInterval: FiniteDuration, outboundContext: OutboundContext): Source[OutboundEnvelope, NotUsed] = { + val deadLetters = TestProbe().ref + Source(1 to sendCount) + .map(n ⇒ outboundEnvelopePool.acquire().init(OptionVal.None, TestSysMsg("msg-" + n), OptionVal.None)) + .via(new SystemMessageDelivery(outboundContext, deadLetters, resendInterval, maxBufferSize = 1000)) + } + + private def inbound(inboundContext: InboundContext): Flow[OutboundEnvelope, InboundEnvelope, NotUsed] = { + val recipient = OptionVal.None // not used + Flow[OutboundEnvelope] + .map(outboundEnvelope ⇒ outboundEnvelope.message match { + case sysEnv: SystemMessageEnvelope ⇒ + InboundEnvelope(recipient, sysEnv, OptionVal.None, addressA.uid, + inboundContext.association(addressA.uid)) + }) + .async + .via(new SystemMessageAcker(inboundContext)) + } + + private def drop(dropSeqNumbers: Vector[Long]): Flow[OutboundEnvelope, OutboundEnvelope, NotUsed] = { + Flow[OutboundEnvelope] + .statefulMapConcat(() ⇒ { + var dropping = dropSeqNumbers + + { + outboundEnvelope ⇒ + outboundEnvelope.message match { + case SystemMessageEnvelope(_, seqNo, _) ⇒ + val i = dropping.indexOf(seqNo) + if (i >= 0) { + dropping = dropping.updated(i, -1L) + Nil + } else + List(outboundEnvelope) + case _ ⇒ Nil + } + } + }) + } + + private def randomDrop[T](dropRate: Double): Flow[T, T, NotUsed] = Flow[T].mapConcat { elem ⇒ + if (ThreadLocalRandom.current().nextDouble() < dropRate) Nil + else List(elem) + } + + "System messages" must { + + "be delivered with real actors" in { + systemB.actorOf(TestActors.echoActorProps, "echo") + + val remoteRef = { + system.actorSelection(rootB / "user" / "echo") ! Identify(None) + expectMsgType[ActorIdentity].ref.get + } + + watch(remoteRef) + remoteRef ! PoisonPill + expectTerminated(remoteRef) + } + + "be flushed on shutdown" in { + val systemC = ActorSystem("systemC", system.settings.config) + try { + systemC.actorOf(TestActors.echoActorProps, "echo") + + val addressC = RARP(systemC).provider.getDefaultAddress + val rootC = RootActorPath(addressC) + + val remoteRef = { + system.actorSelection(rootC / "user" / "echo") ! Identify(None) + expectMsgType[ActorIdentity].ref.get + } + + watch(remoteRef) + remoteRef ! "hello" + expectMsg("hello") + systemC.terminate() + // DeathWatchNotification is sent from systemC, failure detection takes longer than 3 seconds + expectTerminated(remoteRef, 5.seconds) + } finally { + shutdown(systemC) + } + } + + "be resent when some in the middle are lost" in { + val replyProbe = TestProbe() + val controlSubject = new TestControlMessageSubject + val inboundContextB = new ManualReplyInboundContext(replyProbe.ref, addressB, controlSubject) + val inboundContextA = new TestInboundContext(addressB, controlSubject) + val outboundContextA = inboundContextA.association(addressB.address) + + val sink = send(sendCount = 5, resendInterval = 60.seconds, outboundContextA) + .via(drop(dropSeqNumbers = Vector(3L, 4L))) + .via(inbound(inboundContextB)) + .map(_.message.asInstanceOf[TestSysMsg]) + .runWith(TestSink.probe) + + sink.request(100) + sink.expectNext(TestSysMsg("msg-1")) + sink.expectNext(TestSysMsg("msg-2")) + replyProbe.expectMsg(Ack(1L, addressB)) + replyProbe.expectMsg(Ack(2L, addressB)) + // 3 and 4 was dropped + replyProbe.expectMsg(Nack(2L, addressB)) + sink.expectNoMsg(100.millis) // 3 was dropped + inboundContextB.deliverLastReply() + // resending 3, 4, 5 + sink.expectNext(TestSysMsg("msg-3")) + replyProbe.expectMsg(Ack(3L, addressB)) + sink.expectNext(TestSysMsg("msg-4")) + replyProbe.expectMsg(Ack(4L, addressB)) + sink.expectNext(TestSysMsg("msg-5")) + replyProbe.expectMsg(Ack(5L, addressB)) + replyProbe.expectNoMsg(100.millis) + inboundContextB.deliverLastReply() + sink.expectComplete() + } + + "be resent when first is lost" in { + val replyProbe = TestProbe() + val controlSubject = new TestControlMessageSubject + val inboundContextB = new ManualReplyInboundContext(replyProbe.ref, addressB, controlSubject) + val inboundContextA = new TestInboundContext(addressB, controlSubject) + val outboundContextA = inboundContextA.association(addressB.address) + + val sink = send(sendCount = 3, resendInterval = 60.seconds, outboundContextA) + .via(drop(dropSeqNumbers = Vector(1L))) + .via(inbound(inboundContextB)) + .map(_.message.asInstanceOf[TestSysMsg]) + .runWith(TestSink.probe) + + sink.request(100) + replyProbe.expectMsg(Nack(0L, addressB)) // from receiving 2 + replyProbe.expectMsg(Nack(0L, addressB)) // from receiving 3 + sink.expectNoMsg(100.millis) // 1 was dropped + inboundContextB.deliverLastReply() // it's ok to not delivery all nacks + // resending 1, 2, 3 + sink.expectNext(TestSysMsg("msg-1")) + replyProbe.expectMsg(Ack(1L, addressB)) + sink.expectNext(TestSysMsg("msg-2")) + replyProbe.expectMsg(Ack(2L, addressB)) + sink.expectNext(TestSysMsg("msg-3")) + replyProbe.expectMsg(Ack(3L, addressB)) + inboundContextB.deliverLastReply() + sink.expectComplete() + } + + "be resent when last is lost" in { + val replyProbe = TestProbe() + val controlSubject = new TestControlMessageSubject + val inboundContextB = new ManualReplyInboundContext(replyProbe.ref, addressB, controlSubject) + val inboundContextA = new TestInboundContext(addressB, controlSubject) + val outboundContextA = inboundContextA.association(addressB.address) + + val sink = send(sendCount = 3, resendInterval = 2.seconds, outboundContextA) + .via(drop(dropSeqNumbers = Vector(3L))) + .via(inbound(inboundContextB)) + .map(_.message.asInstanceOf[TestSysMsg]) + .runWith(TestSink.probe) + + sink.request(100) + sink.expectNext(TestSysMsg("msg-1")) + replyProbe.expectMsg(Ack(1L, addressB)) + inboundContextB.deliverLastReply() + sink.expectNext(TestSysMsg("msg-2")) + replyProbe.expectMsg(Ack(2L, addressB)) + inboundContextB.deliverLastReply() + sink.expectNoMsg(200.millis) // 3 was dropped + // resending 3 due to timeout + sink.expectNext(TestSysMsg("msg-3")) + replyProbe.expectMsg(4.seconds, Ack(3L, addressB)) + // continue resending + replyProbe.expectMsg(4.seconds, Ack(3L, addressB)) + inboundContextB.deliverLastReply() + replyProbe.expectNoMsg(2200.millis) + sink.expectComplete() + } + + "deliver all during stress and random dropping" in { + val N = 10000 + val dropRate = 0.1 + val controlSubject = new TestControlMessageSubject + val inboundContextB = new TestInboundContext(addressB, controlSubject, replyDropRate = dropRate) + val inboundContextA = new TestInboundContext(addressB, controlSubject) + val outboundContextA = inboundContextA.association(addressB.address) + + val output = + send(N, 1.second, outboundContextA) + .via(randomDrop(dropRate)) + .via(inbound(inboundContextB)) + .map(_.message.asInstanceOf[TestSysMsg]) + .runWith(Sink.seq) + + Await.result(output, 20.seconds) should ===((1 to N).map(n ⇒ TestSysMsg("msg-" + n)).toVector) + } + + "deliver all during throttling and random dropping" in { + val N = 500 + val dropRate = 0.1 + val controlSubject = new TestControlMessageSubject + val inboundContextB = new TestInboundContext(addressB, controlSubject, replyDropRate = dropRate) + val inboundContextA = new TestInboundContext(addressB, controlSubject) + val outboundContextA = inboundContextA.association(addressB.address) + + val output = + send(N, 1.second, outboundContextA) + .throttle(200, 1.second, 10, ThrottleMode.shaping) + .via(randomDrop(dropRate)) + .via(inbound(inboundContextB)) + .map(_.message.asInstanceOf[TestSysMsg]) + .runWith(Sink.seq) + + Await.result(output, 20.seconds) should ===((1 to N).map(n ⇒ TestSysMsg("msg-" + n)).toVector) + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala new file mode 100644 index 0000000000..422211634b --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -0,0 +1,143 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.CopyOnWriteArrayList +import java.util.concurrent.ThreadLocalRandom +import scala.concurrent.Future +import scala.concurrent.Promise +import scala.util.Success +import akka.Done +import akka.actor.ActorRef +import akka.actor.Address +import akka.remote.UniqueAddress +import akka.remote.artery.InboundControlJunction.ControlMessageObserver +import akka.remote.artery.InboundControlJunction.ControlMessageSubject +import akka.util.OptionVal +import akka.dispatch.ExecutionContexts +import com.typesafe.config.ConfigFactory + +private[remote] class TestInboundContext( + override val localAddress: UniqueAddress, + val controlSubject: TestControlMessageSubject = new TestControlMessageSubject, + val controlProbe: Option[ActorRef] = None, + val replyDropRate: Double = 0.0) extends InboundContext { + + private val associationsByAddress = new ConcurrentHashMap[Address, OutboundContext]() + private val associationsByUid = new ConcurrentHashMap[Long, OutboundContext]() + + override def sendControl(to: Address, message: ControlMessage) = { + if (ThreadLocalRandom.current().nextDouble() >= replyDropRate) + association(to).sendControl(message) + } + + override def association(remoteAddress: Address): OutboundContext = + associationsByAddress.get(remoteAddress) match { + case null ⇒ + val a = createAssociation(remoteAddress) + associationsByAddress.putIfAbsent(remoteAddress, a) match { + case null ⇒ a + case existing ⇒ existing + } + case existing ⇒ existing + } + + override def association(uid: Long): OptionVal[OutboundContext] = + OptionVal(associationsByUid.get(uid)) + + override def completeHandshake(peer: UniqueAddress): Future[Done] = { + val a = association(peer.address).asInstanceOf[TestOutboundContext] + val done = a.completeHandshake(peer) + done.foreach { _ ⇒ + associationsByUid.put(peer.uid, a) + }(ExecutionContexts.sameThreadExecutionContext) + done + } + + protected def createAssociation(remoteAddress: Address): TestOutboundContext = + new TestOutboundContext(localAddress, remoteAddress, controlSubject, controlProbe) + + override lazy val settings: ArterySettings = + ArterySettings(ConfigFactory.load().getConfig("akka.remote.artery")) +} + +private[remote] class TestOutboundContext( + override val localAddress: UniqueAddress, + override val remoteAddress: Address, + override val controlSubject: TestControlMessageSubject, + val controlProbe: Option[ActorRef] = None) extends OutboundContext { + + // access to this is synchronized (it's a test utility) + private var _associationState = AssociationState() + + override def associationState: AssociationState = synchronized { + _associationState + } + + def completeHandshake(peer: UniqueAddress): Future[Done] = synchronized { + _associationState.uniqueRemoteAddressPromise.trySuccess(peer) + _associationState.uniqueRemoteAddress.value match { + case Some(Success(`peer`)) ⇒ // our value + case _ ⇒ + _associationState = _associationState.newIncarnation(Promise.successful(peer)) + } + Future.successful(Done) + } + + override def quarantine(reason: String): Unit = synchronized { + _associationState = _associationState.newQuarantined() + } + + override def sendControl(message: ControlMessage) = { + controlProbe.foreach(_ ! message) + controlSubject.sendControl(InboundEnvelope(OptionVal.None, message, OptionVal.None, localAddress.uid, + OptionVal.None)) + } + + override lazy val settings: ArterySettings = + ArterySettings(ConfigFactory.load().getConfig("akka.remote.artery")) + +} + +private[remote] class TestControlMessageSubject extends ControlMessageSubject { + + private val observers = new CopyOnWriteArrayList[ControlMessageObserver] + + override def attach(observer: ControlMessageObserver): Future[Done] = { + observers.add(observer) + Future.successful(Done) + } + + override def detach(observer: ControlMessageObserver): Unit = { + observers.remove(observer) + } + + override def stopped: Future[Done] = Promise[Done]().future + + def sendControl(env: InboundEnvelope): Unit = { + val iter = observers.iterator() + while (iter.hasNext()) + iter.next().notify(env) + } + +} + +private[remote] class ManualReplyInboundContext( + replyProbe: ActorRef, + localAddress: UniqueAddress, + controlSubject: TestControlMessageSubject) extends TestInboundContext(localAddress, controlSubject) { + + private var lastReply: Option[(Address, ControlMessage)] = None + + override def sendControl(to: Address, message: ControlMessage): Unit = synchronized { + lastReply = Some((to, message)) + replyProbe ! message + } + + def deliverLastReply(): Unit = synchronized { + lastReply.foreach { case (to, message) ⇒ super.sendControl(to, message) } + lastReply = None + } +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala new file mode 100644 index 0000000000..2d901f492e --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala @@ -0,0 +1,189 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ + +package akka.remote.artery + +import scala.concurrent.duration._ +import com.typesafe.config.ConfigFactory +import akka.actor.Actor +import akka.actor.ActorIdentity +import akka.actor.ActorRef +import akka.actor.ActorSystem +import akka.actor.Deploy +import akka.actor.ExtendedActorSystem +import akka.actor.Identify +import akka.actor.PoisonPill +import akka.actor.Props +import akka.actor.RootActorPath +import akka.actor.Terminated +import akka.testkit.AkkaSpec +import akka.testkit.ImplicitSender +import akka.testkit.TestProbe +import akka.actor.ActorSelection +import akka.testkit.TestEvent +import akka.event.Logging +import akka.remote.RARP +import akka.testkit.EventFilter +import akka.testkit.JavaSerializable + +object UntrustedSpec { + final case class IdentifyReq(path: String) extends JavaSerializable + final case class StopChild(name: String) extends JavaSerializable + + class Receptionist(testActor: ActorRef) extends Actor { + context.actorOf(Props(classOf[Child], testActor), "child1") + context.actorOf(Props(classOf[Child], testActor), "child2") + context.actorOf(Props(classOf[FakeUser], testActor), "user") + + def receive = { + case IdentifyReq(path) ⇒ context.actorSelection(path).tell(Identify(None), sender()) + case StopChild(name) ⇒ context.child(name) foreach context.stop + case msg ⇒ testActor forward msg + } + } + + class Child(testActor: ActorRef) extends Actor { + override def postStop(): Unit = { + testActor ! s"${self.path.name} stopped" + } + def receive = { + case msg ⇒ testActor forward msg + } + } + + class FakeUser(testActor: ActorRef) extends Actor { + context.actorOf(Props(classOf[Child], testActor), "receptionist") + def receive = { + case msg ⇒ testActor forward msg + } + } + +} + +class UntrustedSpec extends AkkaSpec(""" + akka.actor.provider = remote + akka.remote.artery.untrusted-mode = on + akka.remote.artery.trusted-selection-paths = ["/user/receptionist", ] + akka.remote.artery.enabled = on + akka.remote.artery.canonical.hostname = localhost + akka.remote.artery.canonical.port = 0 + akka.loglevel = DEBUG # the test is verifying some Debug logging + """) with ImplicitSender { + + import UntrustedSpec._ + + val client = ActorSystem("UntrustedSpec-client", ConfigFactory.parseString(""" + akka.actor.provider = remote + akka.remote.artery.enabled = on + akka.remote.artery.canonical.hostname = localhost + akka.remote.artery.canonical.port = 0 + """)) + val addr = RARP(system).provider.getDefaultAddress + + val receptionist = system.actorOf(Props(classOf[Receptionist], testActor), "receptionist") + + lazy val remoteDaemon = { + { + val p = TestProbe()(client) + client.actorSelection(RootActorPath(addr) / receptionist.path.elements).tell(IdentifyReq("/remote"), p.ref) + p.expectMsgType[ActorIdentity].ref.get + } + } + + lazy val target2 = { + val p = TestProbe()(client) + client.actorSelection(RootActorPath(addr) / receptionist.path.elements).tell( + IdentifyReq("child2"), p.ref) + p.expectMsgType[ActorIdentity].ref.get + } + + override def afterTermination() { + shutdown(client) + } + + // need to enable debug log-level without actually printing those messages + system.eventStream.publish(TestEvent.Mute(EventFilter.debug())) + + "UntrustedMode" must { + + "allow actor selection to configured white list" in { + val sel = client.actorSelection(RootActorPath(addr) / receptionist.path.elements) + sel ! "hello" + expectMsg("hello") + } + + "discard harmful messages to /remote" in { + val logProbe = TestProbe() + // but instead install our own listener + system.eventStream.subscribe(system.actorOf(Props(new Actor { + import Logging._ + def receive = { + case d @ Debug(_, _, msg: String) if msg contains "dropping" ⇒ logProbe.ref ! d + case _ ⇒ + } + }).withDeploy(Deploy.local), "debugSniffer"), classOf[Logging.Debug]) + + remoteDaemon ! "hello" + logProbe.expectMsgType[Logging.Debug] + } + + "discard harmful messages to testActor" in { + target2 ! Terminated(remoteDaemon)(existenceConfirmed = true, addressTerminated = false) + target2 ! PoisonPill + client.stop(target2) + target2 ! "blech" + expectMsg("blech") + } + + "discard watch messages" in { + client.actorOf(Props(new Actor { + context.watch(target2) + def receive = { + case x ⇒ testActor forward x + } + }).withDeploy(Deploy.local)) + receptionist ! StopChild("child2") + expectMsg("child2 stopped") + // no Terminated msg, since watch was discarded + expectNoMsg(1.second) + } + + "discard actor selection" in { + val sel = client.actorSelection(RootActorPath(addr) / testActor.path.elements) + sel ! "hello" + expectNoMsg(1.second) + } + + "discard actor selection with non root anchor" in { + val p = TestProbe()(client) + client.actorSelection(RootActorPath(addr) / receptionist.path.elements).tell( + Identify(None), p.ref) + val clientReceptionistRef = p.expectMsgType[ActorIdentity].ref.get + + val sel = ActorSelection(clientReceptionistRef, receptionist.path.toStringWithoutAddress) + sel ! "hello" + expectNoMsg(1.second) + } + + "discard actor selection to child of matching white list" in { + val sel = client.actorSelection(RootActorPath(addr) / receptionist.path.elements / "child1") + sel ! "hello" + expectNoMsg(1.second) + } + + "discard actor selection with wildcard" in { + val sel = client.actorSelection(RootActorPath(addr) / receptionist.path.elements / "*") + sel ! "hello" + expectNoMsg(1.second) + } + + "discard actor selection containing harmful message" in { + val sel = client.actorSelection(RootActorPath(addr) / receptionist.path.elements) + sel ! PoisonPill + expectNoMsg(1.second) + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala new file mode 100644 index 0000000000..655b8f7401 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala @@ -0,0 +1,335 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import com.typesafe.config.ConfigFactory +import akka.actor._ +import akka.pattern.ask +import akka.remote.artery.compress.CompressionProtocol.Events +import akka.testkit._ +import akka.util.Timeout +import com.typesafe.config.ConfigFactory +import org.scalatest.BeforeAndAfter + +import scala.concurrent.Await +import scala.concurrent.duration._ +import akka.actor.ExtendedActorSystem +import akka.serialization.SerializerWithStringManifest +import akka.remote.artery.ArteryMultiNodeSpec + +object CompressionIntegrationSpec { + + val commonConfig = ConfigFactory.parseString(s""" + akka { + loglevel = INFO + + actor { + serializers { + test-message = "akka.remote.artery.compress.TestMessageSerializer" + } + serialization-bindings { + "akka.remote.artery.compress.TestMessage" = test-message + } + } + + remote.artery.advanced.compression { + actor-refs.advertisement-interval = 2 seconds + manifests.advertisement-interval = 2 seconds + } + } + """) + +} + +class CompressionIntegrationSpec extends ArteryMultiNodeSpec(CompressionIntegrationSpec.commonConfig) + with ImplicitSender { + import CompressionIntegrationSpec._ + + val systemB = newRemoteSystem(name = Some("systemB")) + val messagesToExchange = 10 + + "Compression table" must { + "be advertised for chatty ActorRef and manifest" in { + // listen for compression table events + val aManifestProbe = TestProbe()(system) + val bManifestProbe = TestProbe()(systemB) + system.eventStream.subscribe(aManifestProbe.ref, classOf[CompressionProtocol.Events.ReceivedClassManifestCompressionTable]) + systemB.eventStream.subscribe(bManifestProbe.ref, classOf[CompressionProtocol.Events.ReceivedClassManifestCompressionTable]) + val aRefProbe = TestProbe()(system) + val bRefProbe = TestProbe()(systemB) + system.eventStream.subscribe(aRefProbe.ref, classOf[CompressionProtocol.Events.ReceivedActorRefCompressionTable]) + systemB.eventStream.subscribe(bRefProbe.ref, classOf[CompressionProtocol.Events.ReceivedActorRefCompressionTable]) + + val echoRefB = systemB.actorOf(TestActors.echoActorProps, "echo") + + system.actorSelection(rootActorPath(systemB) / "user" / "echo") ! Identify(None) + val echoRefA = expectMsgType[ActorIdentity].ref.get + + // cause TestMessage manifest to become a heavy hitter + // cause echo to become a heavy hitter + (1 to messagesToExchange).foreach { i ⇒ echoRefA ! TestMessage("hello") } + receiveN(messagesToExchange) // the replies + + within(10.seconds) { + // on system A side + awaitAssert { + val a1 = aManifestProbe.expectMsgType[Events.ReceivedClassManifestCompressionTable](2.seconds) + info("System [A] received: " + a1) + a1.table.version.toInt should be >= (1) + a1.table.dictionary.keySet should contain("TestMessageManifest") + } + awaitAssert { + val a1 = aRefProbe.expectMsgType[Events.ReceivedActorRefCompressionTable](2.seconds) + info("System [A] received: " + a1) + a1.table.version.toInt should be >= (1) + a1.table.dictionary.keySet should contain(echoRefA) // recipient + a1.table.dictionary.keySet should contain(testActor) // sender + } + + // on system B side + awaitAssert { + val b1 = bManifestProbe.expectMsgType[Events.ReceivedClassManifestCompressionTable](2.seconds) + info("System [B] received: " + b1) + b1.table.version.toInt should be >= (1) + b1.table.dictionary.keySet should contain("TestMessageManifest") + } + awaitAssert { + val b1 = bRefProbe.expectMsgType[Events.ReceivedActorRefCompressionTable](2.seconds) + info("System [B] received: " + b1) + b1.table.version.toInt should be >= (1) + b1.table.dictionary.keySet should contain(echoRefB) + } + } + + // and if we continue sending new advertisements with higher version number are advertised + within(20.seconds) { + val ignore = TestProbe()(system) + awaitAssert { + echoRefA.tell(TestMessage("hello2"), ignore.ref) + val a2 = aManifestProbe.expectMsgType[Events.ReceivedClassManifestCompressionTable](2.seconds) + info("System [A] received more: " + a2) + a2.table.version.toInt should be >= (3) + } + awaitAssert { + echoRefA.tell(TestMessage("hello2"), ignore.ref) + val a2 = aRefProbe.expectMsgType[Events.ReceivedActorRefCompressionTable](2.seconds) + info("System [A] received more: " + a2) + a2.table.version.toInt should be >= (3) + } + + awaitAssert { + echoRefA.tell(TestMessage("hello3"), ignore.ref) + val b2 = bManifestProbe.expectMsgType[Events.ReceivedClassManifestCompressionTable](2.seconds) + info("System [B] received more: " + b2) + b2.table.version.toInt should be >= (3) + } + awaitAssert { + echoRefA.tell(TestMessage("hello3"), ignore.ref) + val b2 = bRefProbe.expectMsgType[Events.ReceivedActorRefCompressionTable](2.seconds) + info("System [B] received more: " + b2) + b2.table.version.toInt should be >= (3) + } + } + } + + } + + "handle noSender sender" in { + val aRefProbe = TestProbe()(systemB) + system.eventStream.subscribe(aRefProbe.ref, classOf[CompressionProtocol.Events.ReceivedActorRefCompressionTable]) + + val probeB = TestProbe()(systemB) + systemB.actorOf(TestActors.forwardActorProps(probeB.ref), "fw1") + + system.actorSelection(rootActorPath(systemB) / "user" / "fw1") ! Identify(None) + val fwRefA = expectMsgType[ActorIdentity].ref.get + + fwRefA.tell(TestMessage("hello-fw1-a"), ActorRef.noSender) + probeB.expectMsg(TestMessage("hello-fw1-a")) + + within(10.seconds) { + // on system A side + awaitAssert { + val a1 = aRefProbe.expectMsgType[Events.ReceivedActorRefCompressionTable](2.seconds) + info("System [A] received: " + a1) + a1.table.dictionary.keySet should contain(fwRefA) // recipient + a1.table.dictionary.keySet should not contain (system.deadLetters) // sender + } + } + + fwRefA.tell(TestMessage("hello-fw1-b"), ActorRef.noSender) + probeB.expectMsg(TestMessage("hello-fw1-b")) + } + + "handle deadLetters sender" in { + val aRefProbe = TestProbe()(systemB) + system.eventStream.subscribe(aRefProbe.ref, classOf[CompressionProtocol.Events.ReceivedActorRefCompressionTable]) + + val probeB = TestProbe()(systemB) + systemB.actorOf(TestActors.forwardActorProps(probeB.ref), "fw2") + + system.actorSelection(rootActorPath(systemB) / "user" / "fw2") ! Identify(None) + val fwRefA = expectMsgType[ActorIdentity].ref.get + + fwRefA.tell(TestMessage("hello-fw2-a"), ActorRef.noSender) + probeB.expectMsg(TestMessage("hello-fw2-a")) + + within(10.seconds) { + // on system A side + awaitAssert { + val a1 = aRefProbe.expectMsgType[Events.ReceivedActorRefCompressionTable](2.seconds) + info("System [A] received: " + a1) + a1.table.dictionary.keySet should contain(fwRefA) // recipient + a1.table.dictionary.keySet should not contain (system.deadLetters) // sender + } + } + + fwRefA.tell(TestMessage("hello-fw2-b"), ActorRef.noSender) + probeB.expectMsg(TestMessage("hello-fw2-b")) + } + + "work when starting new ActorSystem with same hostname:port" in { + val port = address(systemB).port.get + shutdown(systemB) + val systemB2 = newRemoteSystem( + extraConfig = Some(s"akka.remote.artery.canonical.port=$port"), + name = Some("systemB")) + + // listen for compression table events + val aManifestProbe = TestProbe()(system) + val bManifestProbe = TestProbe()(systemB2) + system.eventStream.subscribe(aManifestProbe.ref, classOf[CompressionProtocol.Events.ReceivedClassManifestCompressionTable]) + systemB2.eventStream.subscribe(bManifestProbe.ref, classOf[CompressionProtocol.Events.ReceivedClassManifestCompressionTable]) + val aRefProbe = TestProbe()(system) + val bRefProbe = TestProbe()(systemB2) + system.eventStream.subscribe(aRefProbe.ref, classOf[CompressionProtocol.Events.ReceivedActorRefCompressionTable]) + systemB2.eventStream.subscribe(bRefProbe.ref, classOf[CompressionProtocol.Events.ReceivedActorRefCompressionTable]) + + val echoRefB2 = systemB2.actorOf(TestActors.echoActorProps, "echo2") + + // messages to the new system might be dropped, before new handshake is completed + within(5.seconds) { + awaitAssert { + val p = TestProbe()(system) + system.actorSelection(rootActorPath(systemB2) / "user" / "echo2").tell(Identify(None), p.ref) + p.expectMsgType[ActorIdentity](1.second).ref.get + } + } + + system.actorSelection(rootActorPath(systemB2) / "user" / "echo2") ! Identify(None) + val echoRefA = expectMsgType[ActorIdentity].ref.get + + // cause TestMessage manifest to become a heavy hitter + (1 to messagesToExchange).foreach { i ⇒ echoRefA ! TestMessage("hello") } + receiveN(messagesToExchange) // the replies + + within(10.seconds) { + // on system A side + awaitAssert { + val a2 = aManifestProbe.expectMsgType[Events.ReceivedClassManifestCompressionTable](2.seconds) + info("System [A] received: " + a2) + a2.table.version.toInt should be >= (1) + a2.table.version.toInt should be < (3) + a2.table.dictionary.keySet should contain("TestMessageManifest") + } + awaitAssert { + val a2 = aRefProbe.expectMsgType[Events.ReceivedActorRefCompressionTable](2.seconds) + info("System [A] received: " + a2) + a2.table.version.toInt should be >= (1) + a2.table.version.toInt should be < (3) + a2.table.dictionary.keySet should contain(echoRefA) // recipient + a2.table.dictionary.keySet should contain(testActor) // sender + } + + // on system B2 side + awaitAssert { + val b2 = bManifestProbe.expectMsgType[Events.ReceivedClassManifestCompressionTable](2.seconds) + info("System [B2] received: " + b2) + b2.table.version.toInt should be >= (1) + b2.table.dictionary.keySet should contain("TestMessageManifest") + } + awaitAssert { + val b2 = bRefProbe.expectMsgType[Events.ReceivedActorRefCompressionTable](2.seconds) + info("System [B] received: " + b2) + b2.table.version.toInt should be >= (1) + b2.table.dictionary.keySet should contain(echoRefB2) + } + } + } + + "wrap around" in { + val extraConfig = """ + akka.remote.artery.advanced.compression { + actor-refs.advertisement-interval = 10 millis + } + """ + + val systemWrap = newRemoteSystem(extraConfig = Some(extraConfig)) + + val wrapRefProbe = TestProbe()(systemWrap) + system.eventStream.subscribe(wrapRefProbe.ref, classOf[CompressionProtocol.Events.ReceivedActorRefCompressionTable]) + + def createAndIdentify(i: Int) = { + val echoWrap = systemWrap.actorOf(TestActors.echoActorProps, s"echo_$i") + system.actorSelection(rootActorPath(systemWrap) / "user" / s"echo_$i") ! Identify(None) + expectMsgType[ActorIdentity].ref.get + } + + val maxTableVersions = 130 // so table version wraps around at least once + val maxDuplicateTables = 40 // max duplicate tables that will not fail the test + var tableVersionsSeen = 0 + var lastTableVersion = 0 + var iteration = 0 + + while (tableVersionsSeen < maxTableVersions) { + iteration += 1 + if (iteration - maxTableVersions > maxDuplicateTables) { + throw new Error("Too much duplicate tables. Giving up on the test.") + } + + val echoWrap = createAndIdentify(iteration) // create a different actor for every iteration + + // cause echo to become a heavy hitter + (1 to messagesToExchange).foreach { i ⇒ echoWrap ! TestMessage("hello") } + receiveN(messagesToExchange) // the replies + + // on system A side + val a1 = wrapRefProbe.expectMsgType[Events.ReceivedActorRefCompressionTable](2.seconds) + val currentTableVersion = a1.table.version.toInt + + if (currentTableVersion != lastTableVersion) { // if we get a new table + lastTableVersion = currentTableVersion + tableVersionsSeen += 1 + } + currentTableVersion should ===(tableVersionsSeen & 0x7F) + } + } + +} + +final case class TestMessage(name: String) + +class TestMessageSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest { + + val TestMessageManifest = "TestMessageManifest" + + override val identifier: Int = 101 + + override def manifest(o: AnyRef): String = + o match { + case _: TestMessage ⇒ TestMessageManifest + } + + override def toBinary(o: AnyRef): Array[Byte] = o match { + case msg: TestMessage ⇒ msg.name.getBytes + } + + override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef = { + manifest match { + case TestMessageManifest ⇒ TestMessage(new String(bytes)) + case unknown ⇒ throw new Exception("Unknown manifest: " + unknown) + } + } +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTableSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTableSpec.scala new file mode 100644 index 0000000000..ae8c9f17ea --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTableSpec.scala @@ -0,0 +1,38 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import akka.testkit.AkkaSpec + +class CompressionTableSpec extends AkkaSpec { + + "CompressionTable" must { + "should invert" in { + val decomp = CompressionTable(17L, 1, Map("0" → 0, "1" → 1, "2" → 2, "3" → 3)).invert + decomp.table should ===(Array("0", "1", "2", "3")) + decomp.originUid should ===(17L) + decomp.version should ===(1) + } + + "enforce to start allocating from 0th index" in { + val compressionTable = CompressionTable(17L, 1, Map("1" → 1, "3" → 3)) // missing 0 is a gap too + + val ex = intercept[IllegalArgumentException] { + compressionTable.invert + } + ex.getMessage should include("Compression table should start allocating from 0, yet lowest allocated id was 1") + } + + "should not allow having gaps in compression ids (inversion would fail)" in { + val compressionTable = CompressionTable(17L, 1, Map("0" → 0, "1" → 1, "3" → 3)) // missing 0 is a gap too + + val ex = intercept[IllegalArgumentException] { + compressionTable.invert + } + ex.getMessage should include("Given compression map does not seem to be gap-less") + } + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTestUtils.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTestUtils.scala new file mode 100644 index 0000000000..6214859564 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTestUtils.scala @@ -0,0 +1,17 @@ +/* + * Copyright (C) 2009-2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import akka.actor._ + +object CompressionTestUtils { + + def minimalRef(name: String)(implicit system: ActorSystem): ActorRef = + new MinimalActorRef { + override def provider: ActorRefProvider = system.asInstanceOf[ActorSystemImpl].provider + override def path: ActorPath = RootActorPath(provider.getDefaultAddress) / name + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala new file mode 100644 index 0000000000..8cc12fc4d8 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala @@ -0,0 +1,151 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import akka.actor.{ ActorIdentity, ActorRef, ActorSystem, Identify } +import akka.remote.artery.compress.CompressionProtocol.Events +import akka.testkit._ +import akka.util.Timeout +import akka.pattern.ask +import akka.remote.RARP +import akka.remote.artery.ArteryTransport +import akka.remote.artery.compress.CompressionProtocol.Events.{ Event, ReceivedActorRefCompressionTable } +import com.typesafe.config.ConfigFactory +import org.scalatest.BeforeAndAfter + +import scala.concurrent.Await +import scala.concurrent.duration._ + +object HandshakeShouldDropCompressionTableSpec { + // need the port before systemB is started + val portB = SocketUtil.temporaryServerAddress("localhost", udp = true).getPort + + val commonConfig = ConfigFactory.parseString(s""" + akka { + loglevel = INFO + + actor.provider = "akka.remote.RemoteActorRefProvider" + remote.artery.enabled = on + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 + remote.artery.advanced.handshake-timeout = 10s + remote.artery.advanced.image-liveness-timeout = 7s + + remote.artery.advanced.compression { + actor-refs { + # we'll trigger advertisement manually + advertisement-interval = 10 hours + } + } + } + """) + + val configB = ConfigFactory.parseString(s"akka.remote.artery.canonical.port = $portB") + .withFallback(commonConfig) + +} + +class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDropCompressionTableSpec.commonConfig) + with ImplicitSender with BeforeAndAfter { + import HandshakeShouldDropCompressionTableSpec._ + + implicit val t = Timeout(3.seconds) + var systemB: ActorSystem = null + + before { + systemB = ActorSystem("systemB", configB) + } + + "Outgoing compression table" must { + "be dropped on system restart" in { + val messagesToExchange = 10 + val systemATransport = RARP(system).provider.transport.asInstanceOf[ArteryTransport] + def systemBTransport = RARP(systemB).provider.transport.asInstanceOf[ArteryTransport] + + // listen for compression table events + val aProbe = TestProbe() + val a1Probe = TestProbe() + val b1Probe = TestProbe()(systemB) + system.eventStream.subscribe(aProbe.ref, classOf[Event]) + systemB.eventStream.subscribe(b1Probe.ref, classOf[Event]) + + def echoSel = system.actorSelection(s"akka://systemB@localhost:$portB/user/echo") + systemB.actorOf(TestActors.echoActorProps, "echo") + + // cause testActor-1 to become a heavy hitter + (1 to messagesToExchange).foreach { i ⇒ echoSel ! s"hello-$i" } // does not reply, but a hot receiver should be advertised + waitForEcho(this, s"hello-$messagesToExchange") + systemBTransport.triggerCompressionAdvertisements(actorRef = true, manifest = false) + + val a0 = aProbe.expectMsgType[ReceivedActorRefCompressionTable](10.seconds) + info("System [A] received: " + a0) + a0.table.dictionary.keySet should contain(testActor) + + // cause a1Probe to become a heavy hitter (we want to not have it in the 2nd compression table later) + (1 to messagesToExchange).foreach { i ⇒ echoSel.tell(s"hello-$i", a1Probe.ref) } + waitForEcho(a1Probe, s"hello-$messagesToExchange") + systemBTransport.triggerCompressionAdvertisements(actorRef = true, manifest = false) + + val a1 = aProbe.expectMsgType[ReceivedActorRefCompressionTable](10.seconds) + info("System [A] received: " + a1) + a1.table.dictionary.keySet should contain(a1Probe.ref) + + log.warning("SHUTTING DOWN system {}...", systemB) + shutdown(systemB) + systemB = ActorSystem("systemB", configB) + Thread.sleep(1000) + log.warning("SYSTEM READY {}...", systemB) + + val aNewProbe = TestProbe() + system.eventStream.subscribe(aNewProbe.ref, classOf[Event]) + + systemB.actorOf(TestActors.echoActorProps, "echo") // start it again + (1 to 5) foreach { _ ⇒ + // since some messages may end up being lost + (1 to messagesToExchange).foreach { i ⇒ echoSel ! s"hello-$i" } // does not reply, but a hot receiver should be advertised + Thread.sleep(100) + } + waitForEcho(this, s"hello-$messagesToExchange", max = 10.seconds) + systemBTransport.triggerCompressionAdvertisements(actorRef = true, manifest = false) + + val a2 = aNewProbe.expectMsgType[ReceivedActorRefCompressionTable](10.seconds) + info("System [A] received: " + a2) + a2.table.dictionary.keySet should contain(testActor) + + val aNew2Probe = TestProbe() + (1 to messagesToExchange).foreach { i ⇒ echoSel.tell(s"hello-$i", aNew2Probe.ref) } // does not reply, but a hot receiver should be advertised + waitForEcho(aNew2Probe, s"hello-$messagesToExchange") + systemBTransport.triggerCompressionAdvertisements(actorRef = true, manifest = false) + + val a3 = aNewProbe.expectMsgType[ReceivedActorRefCompressionTable](10.seconds) + info("Received second compression: " + a3) + a3.table.dictionary.keySet should contain(aNew2Probe.ref) + } + } + + def waitForEcho(probe: TestKit, m: String, max: Duration = 3.seconds): Any = + probe.fishForMessage(max = max, hint = s"waiting for '$m'") { + case `m` ⇒ true + case x ⇒ false + } + + def identify(_system: String, port: Int, name: String) = { + val selection = + system.actorSelection(s"akka://${_system}@localhost:$port/user/$name") + val ActorIdentity(1, ref) = Await.result(selection ? Identify(1), 3.seconds) + ref.get + } + + after { + shutdownAllActorSystems() + } + + override def afterTermination(): Unit = + shutdownAllActorSystems() + + private def shutdownAllActorSystems(): Unit = { + if (systemB != null) shutdown(systemB) + } +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/compress/HeavyHittersSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/HeavyHittersSpec.scala new file mode 100644 index 0000000000..ba18e46449 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/HeavyHittersSpec.scala @@ -0,0 +1,101 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import org.scalatest.{ Matchers, WordSpecLike } + +class HeavyHittersSpec extends WordSpecLike with Matchers { + + "TopHeavyHitters" must { + "should work" in { + val hitters = new TopHeavyHitters[String](4) + hitters.update("A", 10) shouldBe true + hitters.snapshot.filter(_ ne null).toSet should ===(Set("A")) + + hitters.update("B", 20) shouldBe true + hitters.snapshot.filter(_ ne null).toSet should ===(Set("A", "B")) + + hitters.update("C", 1) shouldBe true + hitters.snapshot.filter(_ ne null).toSet should ===(Set("A", "B", "C")) + + hitters.update("D", 100) shouldBe true + hitters.snapshot.filter(_ ne null).toSet should ===(Set("A", "B", "D", "C")) + + hitters.update("E", 200) shouldBe true + hitters.snapshot.filter(_ ne null).toSet should ===(Set("A", "B", "D", "E")) + + hitters.update("BB", 22) shouldBe true + hitters.snapshot.filter(_ ne null).toSet should ===(Set("B", "BB", "D", "E")) + + hitters.update("a", 1) shouldBe false + hitters.snapshot.filter(_ ne null).toSet should ===(Set("B", "BB", "D", "E")) + } + + "correctly replace a hitter" in { + val hitters = new TopHeavyHitters[String](4) + hitters.update("A", 10) shouldBe true + hitters.snapshot.filter(_ ne null).toSet should ===(Set("A")) + + hitters.update("A", 12) shouldBe false + hitters.update("A", 22) shouldBe false + hitters.snapshot.filter(_ ne null).toSet should ===(Set("A")) + } + + "correctly drop least heavy hitter when more than N are inserted" in { + val hitters = new TopHeavyHitters[String](4) + + hitters.update("A", 1) shouldBe true + hitters.snapshot.filter(_ ne null).toSet should ===(Set("A")) + + hitters.update("B", 22) shouldBe true + hitters.snapshot.filter(_ ne null).toSet should ===(Set("A", "B")) + + hitters.update("C", 33) shouldBe true + hitters.snapshot.filter(_ ne null).toSet should ===(Set("A", "B", "C")) + hitters.lowestHitterWeight should ===(0) + + // first item which forces dropping least heavy hitter + hitters.update("D", 100) shouldBe true + hitters.snapshot.filter(_ ne null).toSet should ===(Set("A", "B", "C", "D")) + + // second item which forces dropping least heavy hitter + hitters.update("X", 999) shouldBe true + hitters.snapshot.filter(_ ne null).toSet should ===(Set("X", "B", "C", "D")) + } + + "replace the right item even when hashCodes collide" in { + case class MockHashCode(override val toString: String, override val hashCode: Int) + val hitters = new TopHeavyHitters[MockHashCode](2) + + val a1 = MockHashCode("A", 1) + val b1 = MockHashCode("B", 1) + + hitters.update(a1, 1) + hitters.snapshot.filter(_ ne null).toSet should ===(Set(a1)) + hitters.lowestHitterWeight should ===(0) + + hitters.update(b1, 2) + hitters.snapshot.filter(_ ne null).toSet should ===(Set(a1, b1)) + hitters.lowestHitterWeight should ===(1) + + hitters.update(a1, 10) + hitters.snapshot.filter(_ ne null).toSet should ===(Set(a1, b1)) + hitters.lowestHitterWeight should ===(2) + + hitters.update(b1, 100) + hitters.snapshot.filter(_ ne null).toSet should ===(Set(a1, b1)) + hitters.lowestHitterWeight should ===(10) + } + + "behave when something drops from being a hitter and comes back" in { + val hitters = new TopHeavyHitters[String](2) + hitters.update("A", 1) should ===(true) + hitters.update("B", 2) should ===(true) + hitters.update("C", 3) should ===(true) // A was dropped now + hitters.update("A", 10) should ===(true) // TODO this is technically unexpected, we have already compressed A... + } + + } +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/compress/OutboundCompressionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/OutboundCompressionSpec.scala new file mode 100644 index 0000000000..3c18d06f58 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/OutboundCompressionSpec.scala @@ -0,0 +1,33 @@ +/* + * Copyright (C) 2009-2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import akka.actor._ +import akka.testkit.AkkaSpec + +class OutboundCompressionSpec extends AkkaSpec { + import CompressionTestUtils._ + + "Outbound ActorRef compression" must { + val alice = minimalRef("alice") + val bob = minimalRef("bob") + + "not compress unknown actor ref" in { + val table = CompressionTable.empty[ActorRef] + table.compress(alice) should ===(-1) // not compressed + } + + "compress previously registered actor ref" in { + val table = CompressionTable(17L, 1, Map(system.deadLetters → 0, alice → 1)) + table.compress(alice) should ===(1) // compressed + table.compress(bob) should ===(-1) // not compressed + + val table2 = table.copy(2, dictionary = table.dictionary.updated(bob, 2)) + table2.compress(alice) should ===(1) // compressed + table2.compress(bob) should ===(2) // compressed + } + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/serialization/ArteryMessageSerializerSpec.scala b/akka-remote/src/test/scala/akka/remote/serialization/ArteryMessageSerializerSpec.scala new file mode 100644 index 0000000000..cf7d409c47 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/serialization/ArteryMessageSerializerSpec.scala @@ -0,0 +1,73 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ + +package akka.remote.serialization + +import akka.actor._ +import akka.remote.{ RemoteWatcher, UniqueAddress } +import akka.remote.artery.OutboundHandshake.{ HandshakeReq, HandshakeRsp } +import akka.remote.artery.compress.CompressionProtocol.{ ActorRefCompressionAdvertisement, ActorRefCompressionAdvertisementAck, ClassManifestCompressionAdvertisement, ClassManifestCompressionAdvertisementAck } +import akka.remote.artery.compress.CompressionTable +import akka.remote.artery.{ ActorSystemTerminating, ActorSystemTerminatingAck, Quarantined, SystemMessageDelivery } +import akka.serialization.SerializationExtension +import akka.testkit.AkkaSpec + +class ArteryMessageSerializerSpec extends AkkaSpec { + "ArteryMessageSerializer" must { + val actorA = system.actorOf(Props.empty) + val actorB = system.actorOf(Props.empty) + + Seq( + "Quarantined" → Quarantined(uniqueAddress(), uniqueAddress()), + "ActorSystemTerminating" → ActorSystemTerminating(uniqueAddress()), + "ActorSystemTerminatingAck" → ActorSystemTerminatingAck(uniqueAddress()), + "HandshakeReq" → HandshakeReq(uniqueAddress(), uniqueAddress().address), + "HandshakeRsp" → HandshakeRsp(uniqueAddress()), + "ActorRefCompressionAdvertisement" → ActorRefCompressionAdvertisement(uniqueAddress(), CompressionTable(17L, 123, Map(actorA → 123, actorB → 456, system.deadLetters → 0))), + "ActorRefCompressionAdvertisementAck" → ActorRefCompressionAdvertisementAck(uniqueAddress(), 23), + "ClassManifestCompressionAdvertisement" → ClassManifestCompressionAdvertisement(uniqueAddress(), CompressionTable(17L, 42, Map("a" → 535, "b" → 23))), + "ClassManifestCompressionAdvertisementAck" → ClassManifestCompressionAdvertisementAck(uniqueAddress(), 23), + "SystemMessageDelivery.SystemMessageEnvelop" → SystemMessageDelivery.SystemMessageEnvelope("test", 1234567890123L, uniqueAddress()), + "SystemMessageDelivery.Ack" → SystemMessageDelivery.Ack(98765432109876L, uniqueAddress()), + "SystemMessageDelivery.Nack" → SystemMessageDelivery.Nack(98765432109876L, uniqueAddress()), + "RemoteWatcher.ArteryHeartbeat" → RemoteWatcher.ArteryHeartbeat, + "RemoteWatcher.ArteryHeartbeatRsp" → RemoteWatcher.ArteryHeartbeatRsp(Long.MaxValue) + ).foreach { + case (scenario, item) ⇒ + s"resolve serializer for $scenario" in { + val serializer = SerializationExtension(system) + serializer.serializerFor(item.getClass).getClass should ===(classOf[ArteryMessageSerializer]) + } + + s"serialize and de-serialize $scenario" in { + verifySerialization(item) + } + } + + "not support UniqueAddresses without host/port set" in pending + + "reject invalid manifest" in { + intercept[IllegalArgumentException] { + val serializer = new ArteryMessageSerializer(system.asInstanceOf[ExtendedActorSystem]) + serializer.manifest("INVALID") + } + } + + "reject deserialization with invalid manifest" in { + intercept[IllegalArgumentException] { + val serializer = new ArteryMessageSerializer(system.asInstanceOf[ExtendedActorSystem]) + serializer.fromBinary(Array.empty[Byte], "INVALID") + } + } + + def verifySerialization(msg: AnyRef): Unit = { + val serializer = new ArteryMessageSerializer(system.asInstanceOf[ExtendedActorSystem]) + serializer.fromBinary(serializer.toBinary(msg), serializer.manifest(msg)) should ===(msg) + } + + def uniqueAddress(): UniqueAddress = + UniqueAddress(Address("abc", "def", "host", 12345), 2342) + } +} + diff --git a/akka-remote/src/test/scala/akka/remote/serialization/MiscMessageSerializerSpec.scala b/akka-remote/src/test/scala/akka/remote/serialization/MiscMessageSerializerSpec.scala index 069b6c2b8f..5e0a2c3b4d 100644 --- a/akka-remote/src/test/scala/akka/remote/serialization/MiscMessageSerializerSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/serialization/MiscMessageSerializerSpec.scala @@ -5,26 +5,61 @@ package akka.remote.serialization import akka.actor._ -import akka.remote.MessageSerializer +import akka.remote.{ MessageSerializer, RemoteWatcher } import akka.serialization.SerializationExtension import akka.testkit.AkkaSpec import com.typesafe.config.ConfigFactory +import scala.util.control.NoStackTrace + object MiscMessageSerializerSpec { val serializationTestOverrides = """ - |akka.actor.serialization-bindings = { - | "akka.actor.Identify" = akka-misc - | "akka.actor.ActorIdentity" = akka-misc - | "scala.Some" = akka-misc - | "scala.None$" = akka-misc - |} - """.stripMargin + akka.actor.enable-additional-serialization-bindings=on + # or they can be enabled with + # akka.remote.artery.enabled=on + + akka.actor.serialization-bindings { + "akka.remote.serialization.MiscMessageSerializerSpec$TestException" = akka-misc + } + """ val testConfig = ConfigFactory.parseString(serializationTestOverrides).withFallback(AkkaSpec.testConf) + + class TestException(msg: String, cause: Throwable) extends RuntimeException(msg, cause) { + def this(msg: String) = this(msg, null) + + override def equals(other: Any): Boolean = other match { + case e: TestException ⇒ + e.getMessage == getMessage && e.stackTrace == stackTrace && e.getCause == getCause + case _ ⇒ false + } + + def stackTrace: List[StackTraceElement] = + if (getStackTrace == null) Nil + else getStackTrace.toList + } + + class TestExceptionNoStack(msg: String) extends TestException(msg) with NoStackTrace { + override def equals(other: Any): Boolean = other match { + case e: TestExceptionNoStack ⇒ + e.getMessage == getMessage && e.stackTrace == stackTrace + case _ ⇒ false + } + } + + class OtherException(msg: String) extends IllegalArgumentException(msg) { + override def equals(other: Any): Boolean = other match { + case e: OtherException ⇒ e.getMessage == getMessage + case _ ⇒ false + } + } } class MiscMessageSerializerSpec extends AkkaSpec(MiscMessageSerializerSpec.testConfig) { + import MiscMessageSerializerSpec._ + + val ref = system.actorOf(Props.empty, "hello") "MiscMessageSerializer" must { Seq( @@ -33,8 +68,20 @@ class MiscMessageSerializerSpec extends AkkaSpec(MiscMessageSerializerSpec.testC "Identify with Some" → Identify(Some("value")), "ActorIdentity without actor ref" → ActorIdentity("some-message", ref = None), "ActorIdentity with actor ref" → ActorIdentity("some-message", ref = Some(testActor)), + "TestException" → new TestException("err"), + "TestExceptionNoStack" → new TestExceptionNoStack("err2"), + "TestException with cause" → new TestException("err3", new TestException("cause")), + "Status.Success" → Status.Success("value"), + "Status.Failure" → Status.Failure(new TestException("err")), + "Status.Failure JavaSer" → Status.Failure(new OtherException("exc")), // exc with JavaSerializer + "ActorRef" → ref, "Some" → Some("value"), - "None" → None).foreach { + "None" → None, + "Kill" → Kill, + "PoisonPill" → PoisonPill, + "RemoteWatcher.Heartbeat" → RemoteWatcher.Heartbeat, + "RemoteWatcher.HertbeatRsp" → RemoteWatcher.HeartbeatRsp(65537) + ).foreach { case (scenario, item) ⇒ s"resolve serializer for $scenario" in { val serializer = SerializationExtension(system) @@ -64,6 +111,45 @@ class MiscMessageSerializerSpec extends AkkaSpec(MiscMessageSerializerSpec.testC val serializer = new MiscMessageSerializer(system.asInstanceOf[ExtendedActorSystem]) serializer.fromBinary(serializer.toBinary(msg), serializer.manifest(msg)) should ===(msg) } + + // Separate tests due to missing equality on ActorInitializationException + "resolve serializer for ActorInitializationException" in { + val serializer = SerializationExtension(system) + serializer.serializerFor(classOf[ActorInitializationException]).getClass should ===(classOf[MiscMessageSerializer]) + } + + "serialize and deserialze ActorInitializationException" in { + val aiex = ActorInitializationException(ref, "test", new TestException("err")) + val serializer = new MiscMessageSerializer(system.asInstanceOf[ExtendedActorSystem]) + val deserialized = serializer.fromBinary(serializer.toBinary(aiex), serializer.manifest(aiex)) + .asInstanceOf[ActorInitializationException] + + deserialized.getCause should ===(aiex.getCause) + deserialized.getMessage should ===(aiex.getMessage) + deserialized.getActor should ===(aiex.getActor) + } + + "serialize and deserialze ActorInitializationException if ref is null" in { + val aiex = ActorInitializationException(null, "test", new TestException("err")) + val serializer = new MiscMessageSerializer(system.asInstanceOf[ExtendedActorSystem]) + val deserialized = serializer.fromBinary(serializer.toBinary(aiex), serializer.manifest(aiex)) + .asInstanceOf[ActorInitializationException] + + deserialized.getCause should ===(aiex.getCause) + deserialized.getMessage should ===(aiex.getMessage) + deserialized.getActor should ===(aiex.getActor) + } + + "serialize and deserialze ActorInitializationException if cause is null" in { + val aiex = ActorInitializationException(ref, "test", null) + val serializer = new MiscMessageSerializer(system.asInstanceOf[ExtendedActorSystem]) + val deserialized = serializer.fromBinary(serializer.toBinary(aiex), serializer.manifest(aiex)) + .asInstanceOf[ActorInitializationException] + + deserialized.getCause should ===(aiex.getCause) + deserialized.getMessage should ===(aiex.getMessage) + deserialized.getActor should ===(aiex.getActor) + } } } diff --git a/akka-remote/src/test/scala/akka/remote/serialization/PrimitivesSerializationSpec.scala b/akka-remote/src/test/scala/akka/remote/serialization/PrimitivesSerializationSpec.scala new file mode 100644 index 0000000000..23f5229007 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/serialization/PrimitivesSerializationSpec.scala @@ -0,0 +1,166 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ +package akka.remote.serialization + +import java.nio.ByteBuffer + +import akka.actor.{ ActorIdentity, ExtendedActorSystem, Identify } +import akka.serialization.SerializationExtension +import akka.testkit.AkkaSpec +import akka.util.ByteString +import com.typesafe.config.ConfigFactory + +import scala.util.Random + +object PrimitivesSerializationSpec { + val serializationTestOverrides = + """ + akka.actor.enable-additional-serialization-bindings=on + # or they can be enabled with + # akka.remote.artery.enabled=on + """ + + val testConfig = ConfigFactory.parseString(serializationTestOverrides).withFallback(AkkaSpec.testConf) +} + +class PrimitivesSerializationSpec extends AkkaSpec(PrimitivesSerializationSpec.testConfig) { + + val buffer = ByteBuffer.allocate(1024) + + "LongSerializer" must { + Seq(0L, 1L, -1L, Long.MinValue, Long.MinValue + 1L, Long.MaxValue, Long.MaxValue - 1L).map(_.asInstanceOf[AnyRef]).foreach { + item ⇒ + s"resolve serializer for value $item" in { + val serializer = SerializationExtension(system) + serializer.serializerFor(item.getClass).getClass should ===(classOf[LongSerializer]) + } + + s"serialize and de-serialize value $item" in { + verifySerialization(item) + } + + s"serialize and de-serialize value $item using ByteBuffers" in { + verifySerializationByteBuffer(item) + } + } + + def verifySerialization(msg: AnyRef): Unit = { + val serializer = new LongSerializer(system.asInstanceOf[ExtendedActorSystem]) + serializer.fromBinary(serializer.toBinary(msg), None) should ===(msg) + } + + def verifySerializationByteBuffer(msg: AnyRef): Unit = { + val serializer = new LongSerializer(system.asInstanceOf[ExtendedActorSystem]) + buffer.clear() + serializer.toBinary(msg, buffer) + buffer.flip() + serializer.fromBinary(buffer, "") should ===(msg) + } + } + + "IntSerializer" must { + Seq(0, 1, -1, Int.MinValue, Int.MinValue + 1, Int.MaxValue, Int.MaxValue - 1).map(_.asInstanceOf[AnyRef]).foreach { + item ⇒ + s"resolve serializer for value $item" in { + val serializer = SerializationExtension(system) + serializer.serializerFor(item.getClass).getClass should ===(classOf[IntSerializer]) + } + + s"serialize and de-serialize value $item" in { + verifySerialization(item) + } + + s"serialize and de-serialize value $item using ByteBuffers" in { + verifySerializationByteBuffer(item) + } + } + + def verifySerialization(msg: AnyRef): Unit = { + val serializer = new IntSerializer(system.asInstanceOf[ExtendedActorSystem]) + serializer.fromBinary(serializer.toBinary(msg), None) should ===(msg) + } + + def verifySerializationByteBuffer(msg: AnyRef): Unit = { + val serializer = new IntSerializer(system.asInstanceOf[ExtendedActorSystem]) + buffer.clear() + serializer.toBinary(msg, buffer) + buffer.flip() + serializer.fromBinary(buffer, "") should ===(msg) + } + } + + "StringSerializer" must { + val random = Random.nextString(256) + Seq( + "empty string" → "", + "hello" → "hello", + "árvíztűrőütvefúrógép" → "árvíztűrőütvefúrógép", + "random" → random + ).foreach { + case (scenario, item) ⇒ + s"resolve serializer for [$scenario]" in { + val serializer = SerializationExtension(system) + serializer.serializerFor(item.getClass).getClass should ===(classOf[StringSerializer]) + } + + s"serialize and de-serialize [$scenario]" in { + verifySerialization(item) + } + + s"serialize and de-serialize value [$scenario] using ByteBuffers" in { + verifySerializationByteBuffer(item) + } + } + + def verifySerialization(msg: AnyRef): Unit = { + val serializer = new StringSerializer(system.asInstanceOf[ExtendedActorSystem]) + serializer.fromBinary(serializer.toBinary(msg), None) should ===(msg) + } + + def verifySerializationByteBuffer(msg: AnyRef): Unit = { + val serializer = new StringSerializer(system.asInstanceOf[ExtendedActorSystem]) + buffer.clear() + serializer.toBinary(msg, buffer) + buffer.flip() + serializer.fromBinary(buffer, "") should ===(msg) + } + } + + "ByteStringSerializer" must { + Seq( + "empty string" → ByteString.empty, + "simple content" → ByteString("hello"), + "concatenated content" → (ByteString("hello") ++ ByteString("world")), + "sliced content" → ByteString("helloabc").take(5) + ).foreach { + case (scenario, item) ⇒ + s"resolve serializer for [$scenario]" in { + val serializer = SerializationExtension(system) + serializer.serializerFor(item.getClass).getClass should ===(classOf[ByteStringSerializer]) + } + + s"serialize and de-serialize [$scenario]" in { + verifySerialization(item) + } + + s"serialize and de-serialize value [$scenario] using ByteBuffers" in { + verifySerializationByteBuffer(item) + } + } + + def verifySerialization(msg: AnyRef): Unit = { + val serializer = new ByteStringSerializer(system.asInstanceOf[ExtendedActorSystem]) + serializer.fromBinary(serializer.toBinary(msg), None) should ===(msg) + } + + def verifySerializationByteBuffer(msg: AnyRef): Unit = { + val serializer = new ByteStringSerializer(system.asInstanceOf[ExtendedActorSystem]) + buffer.clear() + serializer.toBinary(msg, buffer) + buffer.flip() + serializer.fromBinary(buffer, "") should ===(msg) + } + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/serialization/SystemMessageSerializationSpec.scala b/akka-remote/src/test/scala/akka/remote/serialization/SystemMessageSerializationSpec.scala new file mode 100644 index 0000000000..ad30a23327 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/serialization/SystemMessageSerializationSpec.scala @@ -0,0 +1,79 @@ +package akka.remote.serialization + +import akka.actor.{ ActorInitializationException, ActorRef, ExtendedActorSystem, InternalActorRef } +import akka.dispatch.sysmsg._ +import akka.serialization.SerializationExtension +import akka.testkit.{ AkkaSpec, TestProbe } +import akka.util.ByteString +import com.typesafe.config.ConfigFactory + +object SystemMessageSerializationSpec { + val serializationTestOverrides = + """ + akka.actor.enable-additional-serialization-bindings=on + # or they can be enabled with + # akka.remote.artery.enabled=on + """ + + val testConfig = ConfigFactory.parseString(serializationTestOverrides).withFallback(AkkaSpec.testConf) + + class TestException(msg: String) extends RuntimeException { + override def equals(other: Any): Boolean = other match { + case e: TestException ⇒ e.getMessage == getMessage + case _ ⇒ false + } + } +} + +class SystemMessageSerializationSpec extends AkkaSpec(PrimitivesSerializationSpec.testConfig) { + import SystemMessageSerializationSpec._ + + val testRef = TestProbe().ref.asInstanceOf[InternalActorRef] + val testRef2 = TestProbe().ref.asInstanceOf[InternalActorRef] + + "ByteStringSerializer" must { + Seq( + "Create(None)" → Create(None), + "Recreate(ex)" → Recreate(new TestException("test2")), + "Suspend()" → Suspend(), + "Resume(ex)" → Resume(new TestException("test3")), + "Terminate()" → Terminate(), + "Supervise(ref, async)" → Supervise(testRef, async = true), + "Watch(ref, ref)" → Watch(testRef, testRef2), + "Unwatch(ref, ref)" → Unwatch(testRef, testRef2), + "Failed(ref, ex, uid)" → Failed(testRef, new TestException("test4"), 42), + "DeathWatchNotification(ref, confimed, addressTerminated)" → + DeathWatchNotification(testRef, existenceConfirmed = true, addressTerminated = true) + ).foreach { + case (scenario, item) ⇒ + s"resolve serializer for [$scenario]" in { + val serializer = SerializationExtension(system) + serializer.serializerFor(item.getClass).getClass should ===(classOf[SystemMessageSerializer]) + } + + s"serialize and de-serialize [$scenario]" in { + verifySerialization(item) + } + } + + def verifySerialization(msg: AnyRef): Unit = { + val serializer = new SystemMessageSerializer(system.asInstanceOf[ExtendedActorSystem]) + serializer.fromBinary(serializer.toBinary(msg), None) should ===(msg) + } + + // ActorInitializationException has no proper equality + "serialize and de-serialize Create(Some(ex))" in { + val aiex = ActorInitializationException(testRef, "test", new TestException("test5")) + val createMsg = Create(Some(aiex)) + val serializer = new SystemMessageSerializer(system.asInstanceOf[ExtendedActorSystem]) + val deserialized = serializer.fromBinary(serializer.toBinary(createMsg), None).asInstanceOf[Create] + + deserialized.failure.get.getCause should ===(aiex.getCause) + deserialized.failure.get.getMessage should ===(aiex.getMessage) + deserialized.failure.get.getActor should ===(aiex.getActor) + + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/transport/AkkaProtocolSpec.scala b/akka-remote/src/test/scala/akka/remote/transport/AkkaProtocolSpec.scala index e67e2ab486..0f283aeb2d 100644 --- a/akka-remote/src/test/scala/akka/remote/transport/AkkaProtocolSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/transport/AkkaProtocolSpec.scala @@ -14,6 +14,7 @@ import com.typesafe.config.ConfigFactory import scala.concurrent.duration._ import scala.concurrent.{ Await, Promise } import java.util.concurrent.TimeoutException +import akka.util.OptionVal object AkkaProtocolSpec { @@ -29,7 +30,7 @@ object AkkaProtocolSpec { } -class AkkaProtocolSpec extends AkkaSpec("""akka.actor.provider = "akka.remote.RemoteActorRefProvider" """) with ImplicitSender { +class AkkaProtocolSpec extends AkkaSpec("""akka.actor.provider = remote """) with ImplicitSender { val conf = ConfigFactory.parseString( """ @@ -66,7 +67,7 @@ class AkkaProtocolSpec extends AkkaSpec("""akka.actor.provider = "akka.remote.Re val codec = AkkaPduProtobufCodec val testMsg = WireFormats.SerializedMessage.newBuilder().setSerializerId(0).setMessage(PByteString.copyFromUtf8("foo")).build - val testEnvelope = codec.constructMessage(localAkkaAddress, testActor, testMsg, None) + val testEnvelope = codec.constructMessage(localAkkaAddress, testActor, testMsg, OptionVal.None) val testMsgPdu: ByteString = codec.constructPayload(testEnvelope) def testHeartbeat = InboundPayload(codec.constructHeartbeat) diff --git a/akka-remote/src/test/scala/akka/remote/transport/AkkaProtocolStressTest.scala b/akka-remote/src/test/scala/akka/remote/transport/AkkaProtocolStressTest.scala index e8deaa2a83..18fb7f800f 100644 --- a/akka-remote/src/test/scala/akka/remote/transport/AkkaProtocolStressTest.scala +++ b/akka-remote/src/test/scala/akka/remote/transport/AkkaProtocolStressTest.scala @@ -15,7 +15,7 @@ object AkkaProtocolStressTest { akka { #loglevel = DEBUG actor.serialize-messages = off - actor.provider = "akka.remote.RemoteActorRefProvider" + actor.provider = remote remote.log-remote-lifecycle-events = on diff --git a/akka-remote/src/test/scala/akka/remote/transport/GenericTransportSpec.scala b/akka-remote/src/test/scala/akka/remote/transport/GenericTransportSpec.scala index ee16681b7e..61da926b3f 100644 --- a/akka-remote/src/test/scala/akka/remote/transport/GenericTransportSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/transport/GenericTransportSpec.scala @@ -11,7 +11,7 @@ import akka.remote.RemoteActorRefProvider import akka.remote.transport.TestTransport.{ DisassociateAttempt, WriteAttempt, ListenAttempt, AssociateAttempt } abstract class GenericTransportSpec(withAkkaProtocol: Boolean = false) - extends AkkaSpec("""akka.actor.provider = "akka.remote.RemoteActorRefProvider" """) + extends AkkaSpec("""akka.actor.provider = remote """) with DefaultTimeout with ImplicitSender { def transportName: String diff --git a/akka-remote/src/test/scala/akka/remote/transport/SystemMessageDeliveryStressTest.scala b/akka-remote/src/test/scala/akka/remote/transport/SystemMessageDeliveryStressTest.scala index b459d701bb..cebf7e8cc0 100644 --- a/akka-remote/src/test/scala/akka/remote/transport/SystemMessageDeliveryStressTest.scala +++ b/akka-remote/src/test/scala/akka/remote/transport/SystemMessageDeliveryStressTest.scala @@ -36,7 +36,7 @@ object SystemMessageDeliveryStressTest { val baseConfig: Config = ConfigFactory parseString (s""" akka { #loglevel = DEBUG - actor.provider = "akka.remote.RemoteActorRefProvider" + actor.provider = remote actor.serialize-messages = off remote.log-remote-lifecycle-events = on diff --git a/akka-remote/src/test/scala/akka/remote/transport/ThrottlerTransportAdapterSpec.scala b/akka-remote/src/test/scala/akka/remote/transport/ThrottlerTransportAdapterSpec.scala index a1e131f553..8667a0a4c8 100644 --- a/akka-remote/src/test/scala/akka/remote/transport/ThrottlerTransportAdapterSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/transport/ThrottlerTransportAdapterSpec.scala @@ -15,7 +15,7 @@ import akka.remote.EndpointException object ThrottlerTransportAdapterSpec { val configA: Config = ConfigFactory parseString (""" akka { - actor.provider = "akka.remote.RemoteActorRefProvider" + actor.provider = remote remote.netty.tcp.hostname = "localhost" remote.log-remote-lifecycle-events = off diff --git a/akka-remote/src/test/scala/akka/remote/transport/netty/NettyTransportSpec.scala b/akka-remote/src/test/scala/akka/remote/transport/netty/NettyTransportSpec.scala index 00cb0900b2..8f1c182211 100644 --- a/akka-remote/src/test/scala/akka/remote/transport/netty/NettyTransportSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/transport/netty/NettyTransportSpec.scala @@ -13,7 +13,7 @@ import scala.concurrent.duration.Duration object NettyTransportSpec { val commonConfig = ConfigFactory.parseString(""" - akka.actor.provider = "akka.remote.RemoteActorRefProvider" + akka.actor.provider = remote """) def getInternal()(implicit sys: ActorSystem) = diff --git a/akka-samples/akka-sample-cluster-java/src/main/resources/application.conf b/akka-samples/akka-sample-cluster-java/src/main/resources/application.conf index 4b2b44c9e9..5077fd7ae5 100644 --- a/akka-samples/akka-sample-cluster-java/src/main/resources/application.conf +++ b/akka-samples/akka-sample-cluster-java/src/main/resources/application.conf @@ -1,7 +1,7 @@ #//#snippet akka { actor { - provider = "akka.cluster.ClusterActorRefProvider" + provider = "cluster" } remote { log-remote-lifecycle-events = off diff --git a/akka-samples/akka-sample-cluster-java/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala b/akka-samples/akka-sample-cluster-java/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala index 68e0bb0e71..dd4701ae34 100644 --- a/akka-samples/akka-sample-cluster-java/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala +++ b/akka-samples/akka-sample-cluster-java/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala @@ -49,7 +49,7 @@ object StatsSampleSingleMasterSpecConfig extends MultiNodeConfig { // note that no fixed host names and ports are used commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.log-remote-lifecycle-events = off akka.cluster.roles = [compute] #//#router-deploy-config diff --git a/akka-samples/akka-sample-cluster-java/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala b/akka-samples/akka-sample-cluster-java/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala index e2bb09ab3e..3bdbbbca9d 100644 --- a/akka-samples/akka-sample-cluster-java/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala +++ b/akka-samples/akka-sample-cluster-java/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala @@ -44,7 +44,7 @@ object StatsSampleSpecConfig extends MultiNodeConfig { // this configuration will be used for all nodes // note that no fixed host names and ports are used commonConfig(ConfigFactory.parseString(""" - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.log-remote-lifecycle-events = off akka.cluster.roles = [compute] #//#router-lookup-config diff --git a/akka-samples/akka-sample-cluster-java/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala b/akka-samples/akka-sample-cluster-java/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala index 331cbb8580..58f8aadec2 100644 --- a/akka-samples/akka-sample-cluster-java/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala +++ b/akka-samples/akka-sample-cluster-java/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala @@ -43,7 +43,7 @@ object TransformationSampleSpecConfig extends MultiNodeConfig { // this configuration will be used for all nodes // note that no fixed host names and ports are used commonConfig(ConfigFactory.parseString(""" - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.log-remote-lifecycle-events = off """)) diff --git a/akka-samples/akka-sample-cluster-java/tutorial/index.html b/akka-samples/akka-sample-cluster-java/tutorial/index.html index d2a88b01fb..6af2d05293 100644 --- a/akka-samples/akka-sample-cluster-java/tutorial/index.html +++ b/akka-samples/akka-sample-cluster-java/tutorial/index.html @@ -27,7 +27,7 @@ Open applic

To enable cluster capabilities in your Akka project you should, at a minimum, add the remote settings, -and use akka.cluster.ClusterActorRefProvider. The akka.cluster.seed-nodes should +and use cluster for akka.actor.provider. The akka.cluster.seed-nodes should normally also be added to your application.conf file.

diff --git a/akka-samples/akka-sample-distributed-data-java/src/main/resources/application.conf b/akka-samples/akka-sample-distributed-data-java/src/main/resources/application.conf index 5bdd1d8522..b7f09dfe48 100644 --- a/akka-samples/akka-sample-distributed-data-java/src/main/resources/application.conf +++ b/akka-samples/akka-sample-distributed-data-java/src/main/resources/application.conf @@ -1,6 +1,6 @@ akka { actor { - provider = "akka.cluster.ClusterActorRefProvider" + provider = "cluster" } remote { log-remote-lifecycle-events = off diff --git a/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/ReplicatedCacheSpec.scala b/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/ReplicatedCacheSpec.scala index 9144f10f41..b5d2980b15 100644 --- a/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/ReplicatedCacheSpec.scala +++ b/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/ReplicatedCacheSpec.scala @@ -19,7 +19,7 @@ object ReplicatedCacheSpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.log-dead-letters-during-shutdown = off """)) diff --git a/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/ReplicatedMetricsSpec.scala b/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/ReplicatedMetricsSpec.scala index 2f3d0b78a0..61049162b1 100644 --- a/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/ReplicatedMetricsSpec.scala +++ b/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/ReplicatedMetricsSpec.scala @@ -19,7 +19,7 @@ object ReplicatedMetricsSpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.log-dead-letters-during-shutdown = off """)) diff --git a/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/ServiceRegistrySpec.scala b/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/ServiceRegistrySpec.scala index b7f13f9e1b..2cf248ca25 100644 --- a/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/ServiceRegistrySpec.scala +++ b/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/ServiceRegistrySpec.scala @@ -22,7 +22,7 @@ object ServiceRegistrySpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.log-dead-letters-during-shutdown = off """)) diff --git a/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/ShoppingCartSpec.scala b/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/ShoppingCartSpec.scala index 432dbc95da..ef5e302405 100644 --- a/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/ShoppingCartSpec.scala +++ b/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/ShoppingCartSpec.scala @@ -19,7 +19,7 @@ object ShoppingCartSpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.log-dead-letters-during-shutdown = off """)) diff --git a/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/VotingServiceSpec.scala b/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/VotingServiceSpec.scala index 6ae3471c49..8f16dfbe86 100644 --- a/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/VotingServiceSpec.scala +++ b/akka-samples/akka-sample-distributed-data-java/src/multi-jvm/scala/sample/distributeddata/VotingServiceSpec.scala @@ -20,7 +20,7 @@ object VotingServiceSpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.log-dead-letters-during-shutdown = off """)) diff --git a/akka-samples/akka-sample-distributed-data-scala/src/main/resources/application.conf b/akka-samples/akka-sample-distributed-data-scala/src/main/resources/application.conf index 5bdd1d8522..b7f09dfe48 100644 --- a/akka-samples/akka-sample-distributed-data-scala/src/main/resources/application.conf +++ b/akka-samples/akka-sample-distributed-data-scala/src/main/resources/application.conf @@ -1,6 +1,6 @@ akka { actor { - provider = "akka.cluster.ClusterActorRefProvider" + provider = "cluster" } remote { log-remote-lifecycle-events = off diff --git a/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/ReplicatedCacheSpec.scala b/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/ReplicatedCacheSpec.scala index 33551afd2a..6a61450fc3 100644 --- a/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/ReplicatedCacheSpec.scala +++ b/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/ReplicatedCacheSpec.scala @@ -18,7 +18,7 @@ object ReplicatedCacheSpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.log-dead-letters-during-shutdown = off """)) diff --git a/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/ReplicatedMetricsSpec.scala b/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/ReplicatedMetricsSpec.scala index dea81b1884..e590c0cf12 100644 --- a/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/ReplicatedMetricsSpec.scala +++ b/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/ReplicatedMetricsSpec.scala @@ -18,7 +18,7 @@ object ReplicatedMetricsSpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.log-dead-letters-during-shutdown = off """)) diff --git a/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/ServiceRegistrySpec.scala b/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/ServiceRegistrySpec.scala index 8f216849bc..dcecbf68c7 100644 --- a/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/ServiceRegistrySpec.scala +++ b/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/ServiceRegistrySpec.scala @@ -21,7 +21,7 @@ object ServiceRegistrySpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.log-dead-letters-during-shutdown = off """)) diff --git a/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/ShoppingCartSpec.scala b/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/ShoppingCartSpec.scala index 600cfc1ffd..81f64f8dff 100644 --- a/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/ShoppingCartSpec.scala +++ b/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/ShoppingCartSpec.scala @@ -18,7 +18,7 @@ object ShoppingCartSpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.log-dead-letters-during-shutdown = off """)) diff --git a/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/VotingServiceSpec.scala b/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/VotingServiceSpec.scala index 693d10ad42..3b0a22894e 100644 --- a/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/VotingServiceSpec.scala +++ b/akka-samples/akka-sample-distributed-data-scala/src/multi-jvm/scala/sample/distributeddata/VotingServiceSpec.scala @@ -19,7 +19,7 @@ object VotingServiceSpec extends MultiNodeConfig { commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.log-dead-letters-during-shutdown = off """)) diff --git a/akka-samples/akka-sample-osgi-dining-hakkers/core/src/main/resources/application.conf b/akka-samples/akka-sample-osgi-dining-hakkers/core/src/main/resources/application.conf index ae8ca13b32..368dcb8e60 100644 --- a/akka-samples/akka-sample-osgi-dining-hakkers/core/src/main/resources/application.conf +++ b/akka-samples/akka-sample-osgi-dining-hakkers/core/src/main/resources/application.conf @@ -4,7 +4,7 @@ akka { logging-filter = "akka.event.slf4j.Slf4jLoggingFilter" actor { - provider = "akka.cluster.ClusterActorRefProvider" + provider = "cluster" serialize-messages = on diff --git a/akka-samples/akka-sample-remote-java/src/main/resources/common.conf b/akka-samples/akka-sample-remote-java/src/main/resources/common.conf index 2c8f881372..9e99e7ab6f 100644 --- a/akka-samples/akka-sample-remote-java/src/main/resources/common.conf +++ b/akka-samples/akka-sample-remote-java/src/main/resources/common.conf @@ -1,7 +1,7 @@ akka { actor { - provider = "akka.remote.RemoteActorRefProvider" + provider = remote } remote { diff --git a/akka-samples/akka-sample-remote-scala/src/main/resources/common.conf b/akka-samples/akka-sample-remote-scala/src/main/resources/common.conf index 2c8f881372..9e99e7ab6f 100644 --- a/akka-samples/akka-sample-remote-scala/src/main/resources/common.conf +++ b/akka-samples/akka-sample-remote-scala/src/main/resources/common.conf @@ -1,7 +1,7 @@ akka { actor { - provider = "akka.remote.RemoteActorRefProvider" + provider = remote } remote { diff --git a/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala b/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala index 848de90e74..fa4f431e35 100644 --- a/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala +++ b/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala @@ -80,6 +80,22 @@ object ActorMaterializer { def apply(materializerSettings: ActorMaterializerSettings)(implicit context: ActorRefFactory): ActorMaterializer = apply(Some(materializerSettings), None) + /** + * INTERNAL API: Creates the `StreamSupervisor` as a system actor. + */ + private[akka] def systemMaterializer(materializerSettings: ActorMaterializerSettings, namePrefix: String, + system: ExtendedActorSystem): ActorMaterializer = { + val haveShutDown = new AtomicBoolean(false) + new ActorMaterializerImpl( + system, + materializerSettings, + system.dispatchers, + system.systemActorOf(StreamSupervisor.props(materializerSettings, haveShutDown) + .withDispatcher(materializerSettings.dispatcher), StreamSupervisor.nextName()), + haveShutDown, + FlowNames(system).name.copy(namePrefix)) + } + /** * Java API: Creates a ActorMaterializer which will execute every step of a transformation * pipeline within its own [[akka.actor.Actor]]. The required [[akka.actor.ActorRefFactory]] diff --git a/akka-stream/src/main/scala/akka/stream/impl/ConstantFun.scala b/akka-stream/src/main/scala/akka/stream/impl/ConstantFun.scala index 54a8f93bab..9d96c802a3 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/ConstantFun.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/ConstantFun.scala @@ -34,4 +34,5 @@ private[akka] object ConstantFun { val none = (_: Any) ⇒ None val two2none = (_: Any, _: Any) ⇒ None + } diff --git a/akka-testkit/src/main/resources/reference.conf b/akka-testkit/src/main/resources/reference.conf index 7adeb68331..ec4c093706 100644 --- a/akka-testkit/src/main/resources/reference.conf +++ b/akka-testkit/src/main/resources/reference.conf @@ -26,4 +26,8 @@ akka { type = akka.testkit.CallingThreadDispatcherConfigurator } } + + actor.serialization-bindings { + "akka.testkit.JavaSerializable" = java + } } diff --git a/akka-testkit/src/main/scala/akka/testkit/JavaSerializable.scala b/akka-testkit/src/main/scala/akka/testkit/JavaSerializable.scala new file mode 100644 index 0000000000..34fb7d9226 --- /dev/null +++ b/akka-testkit/src/main/scala/akka/testkit/JavaSerializable.scala @@ -0,0 +1,11 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.testkit + +import java.io.Serializable + +/** + * Marker trait for test messages that will use JavaSerializer. + */ +trait JavaSerializable extends Serializable diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActors.scala b/akka-testkit/src/main/scala/akka/testkit/TestActors.scala index 7603a3e735..e78b933428 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActors.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActors.scala @@ -19,6 +19,15 @@ object TestActors { } } + /** + * BlackholeActor does nothing for incoming messages, its like a blackhole. + */ + class BlackholeActor extends Actor { + override def receive = { + case _ ⇒ // ignore... + } + } + /** * ForwardActor forwards all messages as-is to specified ActorRef. * @@ -31,6 +40,7 @@ object TestActors { } val echoActorProps = Props[EchoActor]() + val blackholeProps = Props[BlackholeActor]() def forwardActorProps(ref: ActorRef) = Props(classOf[ForwardActor], ref) } diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 67ded6ba7b..ef30823021 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -148,7 +148,7 @@ object AkkaBuild extends Build { lazy val remote = Project( id = "akka-remote", base = file("akka-remote"), - dependencies = Seq(actor, actorTests % "test->test", testkit % "test->test", protobuf) + dependencies = Seq(actor, stream, actorTests % "test->test", testkit % "test->test", streamTestkit % "test", protobuf) ) lazy val multiNodeTestkit = Project( @@ -160,7 +160,7 @@ object AkkaBuild extends Build { lazy val remoteTests = Project( id = "akka-remote-tests", base = file("akka-remote-tests"), - dependencies = Seq(actorTests % "test->test", multiNodeTestkit) + dependencies = Seq(actorTests % "test->test", remote % "test->test", streamTestkit % "test", multiNodeTestkit) ).configs(MultiJvm) lazy val cluster = Project( diff --git a/project/Dependencies.scala b/project/Dependencies.scala index efce292b53..43166b674c 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -74,6 +74,9 @@ object Dependencies { // For Java 8 Conversions val java8Compat = Def.setting {"org.scala-lang.modules" %% "scala-java8-compat" % java8CompatVersion.value} // Scala License + + val aeronDriver = "io.aeron" % "aeron-driver" % "1.0.1" // ApacheV2 + val aeronClient = "io.aeron" % "aeron-client" % "1.0.1" // ApacheV2 object Docs { val sprayJson = "io.spray" %% "spray-json" % "1.3.2" % "test" @@ -103,7 +106,7 @@ object Dependencies { val metrics = "com.codahale.metrics" % "metrics-core" % "3.0.2" % "test" // ApacheV2 val metricsJvm = "com.codahale.metrics" % "metrics-jvm" % "3.0.2" % "test" // ApacheV2 val latencyUtils = "org.latencyutils" % "LatencyUtils" % "1.0.3" % "test" // Free BSD - val hdrHistogram = "org.hdrhistogram" % "HdrHistogram" % "2.1.8" % "test" // CC0 + val hdrHistogram = "org.hdrhistogram" % "HdrHistogram" % "2.1.9" % "test" // CC0 val metricsAll = Seq(metrics, metricsJvm, latencyUtils, hdrHistogram) // sigar logging @@ -136,7 +139,7 @@ object Dependencies { val actorTests = l ++= Seq(Test.junit, Test.scalatest.value, Test.commonsCodec, Test.commonsMath, Test.mockito, Test.scalacheck.value, Test.junitIntf) - val remote = l ++= Seq(netty, uncommonsMath, Test.junit, Test.scalatest.value) + val remote = l ++= Seq(netty, uncommonsMath, aeronDriver, aeronClient, Test.junit, Test.scalatest.value, Test.jimfs) val remoteTests = l ++= Seq(Test.junit, Test.scalatest.value, Test.scalaXml) diff --git a/project/MiMa.scala b/project/MiMa.scala index fb1fed0814..31e85dd9a8 100644 --- a/project/MiMa.scala +++ b/project/MiMa.scala @@ -969,7 +969,30 @@ object MiMa extends AutoPlugin { ), "2.4.10" -> Seq( // #21290 new zipWithIndex flow op - ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOps.zipWithIndex") + ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOps.zipWithIndex"), + + // Remove useUntrustedMode which is an internal API and not used anywhere anymore + ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.Remoting.useUntrustedMode"), + ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.RemoteTransport.useUntrustedMode"), + + // Use OptionVal in remote Send envelope + FilterAnyProblemStartingWith("akka.remote.EndpointManager"), + FilterAnyProblemStartingWith("akka.remote.Remoting"), + FilterAnyProblemStartingWith("akka.remote.RemoteTransport"), + FilterAnyProblemStartingWith("akka.remote.InboundMessageDispatcher"), + FilterAnyProblemStartingWith("akka.remote.DefaultMessageDispatcher"), + FilterAnyProblemStartingWith("akka.remote.transport"), + ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.RemoteActorRefProvider.quarantine"), + ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.RemoteWatcher.quarantine"), + + // #20644 long uids + ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.cluster.protobuf.msg.ClusterMessages#UniqueAddressOrBuilder.hasUid2"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.cluster.protobuf.msg.ClusterMessages#UniqueAddressOrBuilder.getUid2"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.cluster.ddata.protobuf.msg.ReplicatorMessages#UniqueAddressOrBuilder.hasUid2"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.cluster.ddata.protobuf.msg.ReplicatorMessages#UniqueAddressOrBuilder.getUid2"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.remote.RemoteWatcher.receiveHeartbeatRsp"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.remote.RemoteWatcher.selfHeartbeatRspMsg") + ) ) }