From a033d52b378bcdca8f2f681f844061fb1a89386b Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 8 Apr 2016 10:41:32 +0200 Subject: [PATCH 001/186] first prototype of Aeron Source & Sink * and test program to meassure latency and throughput --- .../main/java/akka/aeron/RateReporter.java | 115 ++++++++++ .../src/main/scala/akka/aeron/AeronSink.scala | 97 ++++++++ .../main/scala/akka/aeron/AeronSource.scala | 103 +++++++++ .../main/scala/akka/aeron/AeronStreams.scala | 210 ++++++++++++++++++ project/Dependencies.scala | 9 +- 5 files changed, 532 insertions(+), 2 deletions(-) create mode 100644 akka-bench-jmh/src/main/java/akka/aeron/RateReporter.java create mode 100644 akka-bench-jmh/src/main/scala/akka/aeron/AeronSink.scala create mode 100644 akka-bench-jmh/src/main/scala/akka/aeron/AeronSource.scala create mode 100644 akka-bench-jmh/src/main/scala/akka/aeron/AeronStreams.scala diff --git a/akka-bench-jmh/src/main/java/akka/aeron/RateReporter.java b/akka-bench-jmh/src/main/java/akka/aeron/RateReporter.java new file mode 100644 index 0000000000..e042e58bab --- /dev/null +++ b/akka-bench-jmh/src/main/java/akka/aeron/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.aeron; + +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-bench-jmh/src/main/scala/akka/aeron/AeronSink.scala b/akka-bench-jmh/src/main/scala/akka/aeron/AeronSink.scala new file mode 100644 index 0000000000..e446fed180 --- /dev/null +++ b/akka-bench-jmh/src/main/scala/akka/aeron/AeronSink.scala @@ -0,0 +1,97 @@ +package akka.aeron + +import java.nio.ByteBuffer +import java.util.concurrent.TimeUnit + +import scala.annotation.tailrec +import scala.concurrent.duration._ + +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 +import akka.stream.stage.TimerGraphStageLogic +import io.aeron.Aeron +import org.agrona.concurrent.BackoffIdleStrategy +import org.agrona.concurrent.UnsafeBuffer + +object AeronSink { + type Bytes = Array[Byte] + private case object Backoff +} + +/** + * @param channel eg. "aeron:udp?endpoint=localhost:40123" + */ +class AeronSink(channel: String, aeron: () => Aeron) extends GraphStage[SinkShape[AeronSink.Bytes]] { + import AeronSink._ + + val in: Inlet[Bytes] = Inlet("AeronSink") + override val shape: SinkShape[Bytes] = SinkShape(in) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new TimerGraphStageLogic(shape) with InHandler { + + private val buffer = new UnsafeBuffer(ByteBuffer.allocateDirect(128 * 1024)) + private val streamId = 10 + private val pub = aeron().addPublication(channel, streamId) + private val idleStrategy = new BackoffIdleStrategy( + 100, 10, TimeUnit.MICROSECONDS.toNanos(1), TimeUnit.MICROSECONDS.toNanos(100)) + private val retries = 120 + + private var backoffCount = retries + private var lastMsgSize = 0 + + override def preStart(): Unit = pull(in) + + override def postStop(): Unit = { + pub.close() + } + + // InHandler + override def onPush(): Unit = { + val msg = grab(in) + buffer.putBytes(0, msg); + idleStrategy.reset() + backoffCount = retries + lastMsgSize = msg.length + publish() + } + + @tailrec private def publish(): Unit = { + val result = pub.offer(buffer, 0, lastMsgSize) + // FIXME handle Publication.CLOSED + // TODO the backoff strategy should be measured and tuned + if (result < 0) { + if (backoffCount == 1) { + println(s"# drop") // FIXME + pull(in) // drop it + } else if (backoffCount <= 5) { + // println(s"# scheduled backoff ${6 - backoffCount}") // FIXME + backoffCount -= 1 + if (backoffCount <= 2) + scheduleOnce(Backoff, 50.millis) + else + scheduleOnce(Backoff, 1.millis) + } else { + idleStrategy.idle() + backoffCount -= 1 + publish() // recursive + } + } else { + pull(in) + } + } + + override protected def onTimer(timerKey: Any): Unit = { + timerKey match { + case Backoff => publish() + case msg => super.onTimer(msg) + } + } + + setHandler(in, this) + } +} diff --git a/akka-bench-jmh/src/main/scala/akka/aeron/AeronSource.scala b/akka-bench-jmh/src/main/scala/akka/aeron/AeronSource.scala new file mode 100644 index 0000000000..02bd73c9fb --- /dev/null +++ b/akka-bench-jmh/src/main/scala/akka/aeron/AeronSource.scala @@ -0,0 +1,103 @@ +package akka.aeron + +import java.nio.ByteBuffer +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicBoolean + +import scala.annotation.tailrec +import scala.concurrent.duration._ + +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 +import akka.stream.stage.TimerGraphStageLogic +import io.aeron.Aeron +import io.aeron.logbuffer.FragmentHandler +import io.aeron.logbuffer.Header +import org.agrona.DirectBuffer +import org.agrona.concurrent.BackoffIdleStrategy +import org.agrona.concurrent.UnsafeBuffer + +object AeronSource { + type Bytes = Array[Byte] + private case object Backoff +} + +/** + * @param channel eg. "aeron:udp?endpoint=localhost:40123" + */ +class AeronSource(channel: String, aeron: () => Aeron) extends GraphStage[SourceShape[AeronSource.Bytes]] { + import AeronSource._ + + val out: Outlet[Bytes] = Outlet("AeronSource") + override val shape: SourceShape[Bytes] = SourceShape(out) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new TimerGraphStageLogic(shape) with OutHandler { + + private val buffer = new UnsafeBuffer(ByteBuffer.allocateDirect(256)) + private val streamId = 10 + private val sub = aeron().addSubscription(channel, streamId) + private val running = new AtomicBoolean(true) + private val idleStrategy = new BackoffIdleStrategy( + 100, 10, TimeUnit.MICROSECONDS.toNanos(1), TimeUnit.MICROSECONDS.toNanos(100)) + private val retries = 115 + private var backoffCount = retries + + val receiveMessage = getAsyncCallback[Bytes] { data => + push(out, data) + } + + val fragmentHandler: FragmentHandler = new FragmentHandler { + override def onFragment(buffer: DirectBuffer, offset: Int, length: Int, header: Header): Unit = { + val data = Array.ofDim[Byte](length) + buffer.getBytes(offset, data); + receiveMessage.invoke(data) + } + } + + override def postStop(): Unit = { + running.set(false) + sub.close() + } + + // OutHandler + override def onPull(): Unit = { + idleStrategy.reset() + backoffCount = retries + subscriberLoop() + } + + @tailrec private def subscriberLoop(): Unit = + if (running.get) { + val fragmentsRead = sub.poll(fragmentHandler, 1) + if (fragmentsRead <= 0) { + // TODO the backoff strategy should be measured and tuned + if (backoffCount <= 0) { + // println(s"# scheduled backoff ${0 - backoffCount + 1}") // FIXME + backoffCount -= 1 + if (backoffCount <= -5) + scheduleOnce(Backoff, 50.millis) + else + scheduleOnce(Backoff, 1.millis) + } else { + idleStrategy.idle() + backoffCount -= 1 + subscriberLoop() // recursive + } + } + } + + override protected def onTimer(timerKey: Any): Unit = { + timerKey match { + case Backoff => subscriberLoop() + case msg => super.onTimer(msg) + } + } + + setHandler(out, this) + } +} diff --git a/akka-bench-jmh/src/main/scala/akka/aeron/AeronStreams.scala b/akka-bench-jmh/src/main/scala/akka/aeron/AeronStreams.scala new file mode 100644 index 0000000000..79a1f41b47 --- /dev/null +++ b/akka-bench-jmh/src/main/scala/akka/aeron/AeronStreams.scala @@ -0,0 +1,210 @@ +package akka.aeron + +import scala.concurrent.duration._ +import akka.actor.ActorSystem +import akka.stream.ActorMaterializer +import akka.stream.scaladsl.Source +import io.aeron.Aeron +import io.aeron.driver.MediaDriver +import java.util.concurrent.Executors +import scala.util.Success +import scala.util.Failure +import scala.concurrent.Future +import akka.Done +import org.HdrHistogram.Histogram +import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.CountDownLatch +import java.util.concurrent.CyclicBarrier +import java.util.concurrent.atomic.AtomicLongArray +import akka.stream.ThrottleMode + +object AeronStreams { + + val channel1 = "aeron:udp?endpoint=localhost:40123" + val channel2 = "aeron:udp?endpoint=localhost:40124" + val throughputN = 10000000 + val latencyN = 100000 + val payload = ("0" * 100).getBytes("utf-8") + lazy val sendTimes = new AtomicLongArray(latencyN) + + lazy val aeron = { + val ctx = new Aeron.Context + val driver = MediaDriver.launchEmbedded() + ctx.aeronDirectoryName(driver.aeronDirectoryName) + Aeron.connect(ctx) + } + + lazy val system = ActorSystem("AeronStreams") + lazy implicit val mat = ActorMaterializer()(system) + + lazy val reporter = new RateReporter(SECONDS.toNanos(1), new RateReporter.Reporter { + override def onReport(messagesPerSec: Double, bytesPerSec: Double, totalMessages: Long, totalBytes: Long): Unit = { + println("%.03g msgs/sec, %.03g bytes/sec, totals %d messages %d MB".format( + messagesPerSec, bytesPerSec, totalMessages, totalBytes / (1024 * 1024))) + } + }) + lazy val reporterExecutor = Executors.newFixedThreadPool(1) + + def stopReporter(): Unit = { + reporter.halt() + reporterExecutor.shutdown() + } + + def exit(status: Int): Unit = { + stopReporter() + + system.scheduler.scheduleOnce(10.seconds) { + mat.shutdown() + system.terminate() + new Thread { + Thread.sleep(3000) + System.exit(status) + }.run() + }(system.dispatcher) + } + + lazy val histogram = new Histogram(SECONDS.toNanos(10), 3) + + def printTotal(total: Int, pre: String, startTime: Long, payloadSize: Long): Unit = { + val d = (System.nanoTime - startTime).nanos.toMillis + println(f"### $total $pre of size ${payloadSize} bytes took $d ms, " + + f"${1000.0 * total / d}%.03g msg/s, ${1000.0 * total * payloadSize / d}%.03g bytes/s") + + if (histogram.getTotalCount > 0) { + println("Histogram of RTT latencies in microseconds.") + histogram.outputPercentileDistribution(System.out, 1000.0) + } + } + + def main(args: Array[String]): Unit = { + + // receiver of plain throughput testing + if (args.length == 0 || args(0) == "receiver") + runReceiver() + + // sender of plain throughput testing + if (args.length == 0 || args(0) == "sender") + runSender() + + // sender of ping-pong latency testing + if (args.length != 0 && args(0) == "echo-sender") + runEchoSender() + + // echo receiver of ping-pong latency testing + if (args.length != 0 && args(0) == "echo-receiver") + runEchoReceiver() + } + + def runReceiver(): Unit = { + import system.dispatcher + reporterExecutor.execute(reporter) + val r = reporter + var t0 = System.nanoTime() + var count = 0L + var payloadSize = 0L + Source.fromGraph(new AeronSource(channel1, () => aeron)) + .map { bytes => + r.onMessage(1, bytes.length) + bytes + } + .runForeach { bytes => + count += 1 + if (count == 1) { + t0 = System.nanoTime() + payloadSize = bytes.length + } else if (count == throughputN) { + exit(0) + printTotal(throughputN, "receive", t0, payloadSize) + } + }.onFailure { + case e => + e.printStackTrace + exit(-1) + } + + } + + def runSender(): Unit = { + reporterExecutor.execute(reporter) + val r = reporter + val t0 = System.nanoTime() + Source(1 to throughputN) + .map { n => + if (n == throughputN) { + exit(0) + printTotal(throughputN, "send", t0, payload.length) + } + n + } + .map { _ => + r.onMessage(1, payload.length) + payload + } + .runWith(new AeronSink(channel1, () => aeron)) + } + + def runEchoReceiver(): Unit = { + // just echo back on channel2 + reporterExecutor.execute(reporter) + val r = reporter + Source.fromGraph(new AeronSource(channel1, () => aeron)) + .map { bytes => + r.onMessage(1, bytes.length) + bytes + } + .runWith(new AeronSink(channel2, () => aeron)) + } + + def runEchoSender(): Unit = { + import system.dispatcher + reporterExecutor.execute(reporter) + val r = reporter + + val barrier = new CyclicBarrier(2) + var repeat = 3 + val count = new AtomicInteger + var t0 = System.nanoTime() + Source.fromGraph(new AeronSource(channel2, () => aeron)) + .map { bytes => + r.onMessage(1, bytes.length) + bytes + } + .runForeach { bytes => + val c = count.incrementAndGet() + val d = System.nanoTime() - sendTimes.get(c - 1) + if (c % 10000 == 0) + println(s"# receive offset $c => ${d / 1000} µs") // FIXME + histogram.recordValue(d) + if (c == latencyN) { + printTotal(latencyN, "ping-pong", t0, bytes.length) + barrier.await() // this is always the last party + } + }.onFailure { + case e => + e.printStackTrace + exit(-1) + } + + while (repeat > 0) { + repeat -= 1 + histogram.reset() + count.set(0) + t0 = System.nanoTime() + + Source(1 to latencyN) + .throttle(10000, 1.second, 100000, ThrottleMode.Shaping) + .map { n => + if (n % 10000 == 0) + println(s"# send offset $n") // FIXME + sendTimes.set(n - 1, System.nanoTime()) + payload + } + .runWith(new AeronSink(channel1, () => aeron)) + + barrier.await() + } + + exit(0) + } + +} diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 6424fcd3c0..1a714dee6b 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -66,6 +66,10 @@ object Dependencies { // For Java 8 Conversions val java8Compat = "org.scala-lang.modules" %% "scala-java8-compat" % "0.7.0" // Scala License + + val aeronDriver = "io.aeron" % "aeron-driver" % "0.9.5" // ApacheV2 + val aeronClient = "io.aeron" % "aeron-client" % "0.9.5" // ApacheV2 + val hdrHistogram = "org.hdrhistogram" % "HdrHistogram" % "2.1.8" // CC0 object Docs { val sprayJson = "io.spray" %% "spray-json" % "1.3.2" % "test" @@ -92,7 +96,8 @@ 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" % "1.1.4" % "test" // CC0 + val hdrHistogram = "org.hdrhistogram" % "HdrHistogram" % "2.1.8" % "test" // CC0 + val metricsAll = Seq(metrics, metricsJvm, latencyUtils, hdrHistogram) // sigar logging @@ -161,7 +166,7 @@ object Dependencies { val contrib = l ++= Seq(Test.junitIntf, Test.commonsIo) - val benchJmh = l ++= Seq(Provided.levelDB, Provided.levelDBNative) + val benchJmh = l ++= Seq(Provided.levelDB, Provided.levelDBNative, aeronDriver, aeronClient, hdrHistogram) // akka stream & http From 29429b0ea1cd425ee02332eaf7003b0a1a492e47 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 11 Apr 2016 08:09:13 +0200 Subject: [PATCH 002/186] debug sender/receiver --- .../src/main/scala/akka/aeron/AeronSink.scala | 3 ++ .../main/scala/akka/aeron/AeronSource.scala | 3 ++ .../main/scala/akka/aeron/AeronStreams.scala | 40 +++++++++++++++++-- project/Dependencies.scala | 3 +- 4 files changed, 43 insertions(+), 6 deletions(-) diff --git a/akka-bench-jmh/src/main/scala/akka/aeron/AeronSink.scala b/akka-bench-jmh/src/main/scala/akka/aeron/AeronSink.scala index e446fed180..c595075a43 100644 --- a/akka-bench-jmh/src/main/scala/akka/aeron/AeronSink.scala +++ b/akka-bench-jmh/src/main/scala/akka/aeron/AeronSink.scala @@ -69,6 +69,9 @@ class AeronSink(channel: String, aeron: () => Aeron) extends GraphStage[SinkShap println(s"# drop") // FIXME pull(in) // drop it } else if (backoffCount <= 5) { + // TODO Instead of using the scheduler we should handoff the task of + // retrying/polling to a separate thread that performs the polling for + // all sources/sinks and notifies back when there is some news. // println(s"# scheduled backoff ${6 - backoffCount}") // FIXME backoffCount -= 1 if (backoffCount <= 2) diff --git a/akka-bench-jmh/src/main/scala/akka/aeron/AeronSource.scala b/akka-bench-jmh/src/main/scala/akka/aeron/AeronSource.scala index 02bd73c9fb..0a7ff3f6e1 100644 --- a/akka-bench-jmh/src/main/scala/akka/aeron/AeronSource.scala +++ b/akka-bench-jmh/src/main/scala/akka/aeron/AeronSource.scala @@ -77,6 +77,9 @@ class AeronSource(channel: String, aeron: () => Aeron) extends GraphStage[Source if (fragmentsRead <= 0) { // TODO the backoff strategy should be measured and tuned if (backoffCount <= 0) { + // TODO Instead of using the scheduler we should handoff the task of + // retrying/polling to a separate thread that performs the polling for + // all sources/sinks and notifies back when there is some news. // println(s"# scheduled backoff ${0 - backoffCount + 1}") // FIXME backoffCount -= 1 if (backoffCount <= -5) diff --git a/akka-bench-jmh/src/main/scala/akka/aeron/AeronStreams.scala b/akka-bench-jmh/src/main/scala/akka/aeron/AeronStreams.scala index 79a1f41b47..e73d87c994 100644 --- a/akka-bench-jmh/src/main/scala/akka/aeron/AeronStreams.scala +++ b/akka-bench-jmh/src/main/scala/akka/aeron/AeronStreams.scala @@ -23,7 +23,8 @@ object AeronStreams { val channel1 = "aeron:udp?endpoint=localhost:40123" val channel2 = "aeron:udp?endpoint=localhost:40124" val throughputN = 10000000 - val latencyN = 100000 + val latencyRate = 10000 // per second + val latencyN = 10 * latencyRate val payload = ("0" * 100).getBytes("utf-8") lazy val sendTimes = new AtomicLongArray(latencyN) @@ -93,6 +94,12 @@ object AeronStreams { // echo receiver of ping-pong latency testing if (args.length != 0 && args(0) == "echo-receiver") runEchoReceiver() + + if (args(0) == "debug-receiver") + runDebugReceiver() + + if (args(0) == "debug-sender") + runDebugSender() } def runReceiver(): Unit = { @@ -172,7 +179,7 @@ object AeronStreams { .runForeach { bytes => val c = count.incrementAndGet() val d = System.nanoTime() - sendTimes.get(c - 1) - if (c % 10000 == 0) + if (c % (latencyN / 10) == 0) println(s"# receive offset $c => ${d / 1000} µs") // FIXME histogram.recordValue(d) if (c == latencyN) { @@ -192,9 +199,9 @@ object AeronStreams { t0 = System.nanoTime() Source(1 to latencyN) - .throttle(10000, 1.second, 100000, ThrottleMode.Shaping) + .throttle(latencyRate, 1.second, latencyRate / 10, ThrottleMode.Shaping) .map { n => - if (n % 10000 == 0) + if (n % (latencyN / 10) == 0) println(s"# send offset $n") // FIXME sendTimes.set(n - 1, System.nanoTime()) payload @@ -207,4 +214,29 @@ object AeronStreams { exit(0) } + def runDebugReceiver(): Unit = { + import system.dispatcher + Source.fromGraph(new AeronSource(channel1, () => aeron)) + .map(bytes => new String(bytes, "utf-8")) + .runForeach { s => + println(s) + }.onFailure { + case e => + e.printStackTrace + exit(-1) + } + } + + def runDebugSender(): Unit = { + val fill = "0000" + Source(1 to 1000) + .throttle(1, 1.second, 1, ThrottleMode.Shaping) + .map { n => + val s = (fill + n.toString).takeRight(4) + println(s) + s.getBytes("utf-8") + } + .runWith(new AeronSink(channel1, () => aeron)) + } + } diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 1a714dee6b..65438b552c 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -96,8 +96,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" % "1.1.4" % "test" // CC0 val metricsAll = Seq(metrics, metricsJvm, latencyUtils, hdrHistogram) // sigar logging From 3fd052b868af923eb9b2e6e1fcd5f89514521926 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Endre=20S=C3=A1ndor=20Varga?= Date: Thu, 14 Apr 2016 10:26:09 +0200 Subject: [PATCH 003/186] Proof-of-concept for the new Remoting codenamed Artery --- akka-remote/src/main/resources/reference.conf | 6 + .../akka/remote/RemoteActorRefProvider.scala | 12 +- .../scala/akka/remote/RemoteSettings.scala | 3 + .../scala/akka/remote/RemoteTransport.scala | 5 - .../src/main/scala/akka/remote/Remoting.scala | 3 - .../akka/remote/artery/ArterySubsystem.scala | 124 ++++++++++++++++++ .../scala/akka/remote/artery/Transport.scala | 78 +++++++++++ .../akka/remote/artery/ArterySmokeTest.scala | 57 ++++++++ project/AkkaBuild.scala | 2 +- project/MiMa.scala | 6 +- 10 files changed, 283 insertions(+), 13 deletions(-) create mode 100644 akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/Transport.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/ArterySmokeTest.scala diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 4121c83527..f23382ba44 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -71,6 +71,12 @@ akka { remote { + ### FIXME: Temporary switch for the PoC + artery { + enabled = off + port = 20200 + } + ### General settings # Timeout after which the startup of the remoting subsystem is considered diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 7546dc7ee1..0d6065da83 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -7,16 +7,19 @@ 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.ArterySubsystem /** * INTERNAL API @@ -179,7 +182,7 @@ private[akka] class RemoteActorRefProvider( d }, serialization = SerializationExtension(system), - transport = new Remoting(system, this)) + transport = if (remoteSettings.EnableArtery) new ArterySubsystem(system, this) else new Remoting(system, this)) _internals = internals remotingTerminator ! internals @@ -422,6 +425,7 @@ 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 @@ -448,6 +452,8 @@ private[akka] class RemoteActorRef private[akka] ( deploy: Option[Deploy]) extends InternalActorRef with RemoteRef { + @volatile var cachedAssociation: artery.Association = null + def getChild(name: Iterator[String]): InternalActorRef = { val s = name.toStream s.headOption match { diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index f6138bb5fb..406ced337e 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -20,6 +20,9 @@ final class RemoteSettings(val config: Config) { import config._ import scala.collection.JavaConverters._ + val EnableArtery: Boolean = getBoolean("akka.remote.artery.enabled") + val ArteryPort: Int = getInt("akka.remote.artery.port") + 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..0797e2d2cf 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala @@ -91,9 +91,4 @@ private[akka] abstract class RemoteTransport(val system: ExtendedActorSystem, va */ 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 - } diff --git a/akka-remote/src/main/scala/akka/remote/Remoting.scala b/akka-remote/src/main/scala/akka/remote/Remoting.scala index 8b77da60ae..04a5c0e966 100644 --- a/akka-remote/src/main/scala/akka/remote/Remoting.scala +++ b/akka-remote/src/main/scala/akka/remote/Remoting.scala @@ -227,9 +227,6 @@ private[remote] class Remoting(_system: ExtendedActorSystem, _provider: RemoteAc 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) ⇒ diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala new file mode 100644 index 0000000000..0b007d6415 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala @@ -0,0 +1,124 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ + +package akka.remote.artery + +import java.util.concurrent.ConcurrentHashMap + +import akka.actor.{ ActorRef, Address, ExtendedActorSystem } +import akka.event.{ Logging, LoggingAdapter } +import akka.remote.EndpointManager.Send +import akka.remote.transport.AkkaPduProtobufCodec +import akka.remote.{ DefaultMessageDispatcher, RemoteActorRef, RemoteActorRefProvider, RemoteTransport } +import akka.stream.scaladsl.{ Sink, Source, SourceQueueWithComplete, Tcp } +import akka.stream.{ ActorMaterializer, Materializer, OverflowStrategy } +import akka.{ Done, NotUsed } + +import scala.concurrent.duration._ +import scala.concurrent.{ Await, Future } + +/** + * INTERNAL API + */ +private[remote] class ArterySubsystem(_system: ExtendedActorSystem, _provider: RemoteActorRefProvider) extends RemoteTransport(_system, _provider) { + @volatile private[this] var address: Address = _ + @volatile private[this] var transport: Transport = _ + @volatile private[this] var binding: Tcp.ServerBinding = _ + @volatile private[this] var materializer: Materializer = _ + override val log: LoggingAdapter = Logging(system.eventStream, getClass.getName) + + override def defaultAddress: Address = address + override def addresses: Set[Address] = Set(address) + override def localAddressForRemote(remote: Address): Address = defaultAddress + + // FIXME: This does locking on putIfAbsent, we need something smarter + private[this] val associations = new ConcurrentHashMap[Address, Association]() + + override def start(): Unit = { + // TODO: Configure materializer properly + // TODO: Have a supervisor actor + address = Address("akka.artery", system.name, "localhost", provider.remoteSettings.ArteryPort) + materializer = ActorMaterializer()(system) + transport = new Transport( + address, + system, + provider, + AkkaPduProtobufCodec, + new DefaultMessageDispatcher(system, provider, log)) + + binding = Await.result( + Tcp(system).bindAndHandle(transport.inboundFlow, address.host.get, address.port.get)(materializer), + 3.seconds) + + log.info("Artery started up with address {}", binding.localAddress) + } + + override def shutdown(): Future[Done] = { + import system.dispatcher + binding.unbind().map(_ ⇒ Done).andThen { + case _ ⇒ transport.killSwitch.abort(new Exception("System shut down")) + } + } + + override def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = { + val cached = recipient.cachedAssociation + val remoteAddress = recipient.path.address + + val association = + if (cached ne null) cached + else { + val association = getAssociation(remoteAddress) + association.associate() + recipient.cachedAssociation = association + association + } + + association.send(message, senderOption, recipient) + } + + private def getAssociation(remoteAddress: Address): Association = { + val current = associations.get(remoteAddress) + if (current ne null) current + else { + val newAssociation = new Association(materializer, remoteAddress, transport) + val currentAssociation = associations.putIfAbsent(remoteAddress, newAssociation) + if (currentAssociation eq null) { + newAssociation + } else currentAssociation + } + } + + override def quarantine(remoteAddress: Address, uid: Option[Int]): Unit = { + getAssociation(remoteAddress).quarantine(uid) + } + +} + +/** + * 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 materializer: Materializer, + val remoteAddress: Address, + val transport: Transport) { + @volatile private[this] var queue: SourceQueueWithComplete[Send] = _ + private[this] val sink: Sink[Send, Any] = transport.outbound(remoteAddress) + + def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = { + // TODO: lookup subchannel + // FIXME: Use a different envelope than the old Send, but make sure the new is handled by deadLetters properly + queue.offer(Send(message, senderOption, recipient, None)) + } + + def quarantine(uid: Option[Int]): Unit = () + + // Idempotent + def associate(): Unit = { + queue = Source.queue(256, OverflowStrategy.dropBuffer).to(sink).run()(materializer) + } +} + diff --git a/akka-remote/src/main/scala/akka/remote/artery/Transport.scala b/akka-remote/src/main/scala/akka/remote/artery/Transport.scala new file mode 100644 index 0000000000..9957ce5d0d --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/Transport.scala @@ -0,0 +1,78 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ + +package akka.remote.artery + +import java.net.InetSocketAddress +import java.nio.ByteOrder + +import akka.NotUsed +import akka.actor.{ Address, ExtendedActorSystem } +import akka.remote.EndpointManager.Send +import akka.remote.{ InboundMessageDispatcher, MessageSerializer, RemoteActorRefProvider } +import akka.remote.transport.AkkaPduCodec +import akka.serialization.Serialization +import akka.stream.{ KillSwitches, SharedKillSwitch } +import akka.stream.scaladsl.{ Flow, Framing, Sink, Source, Tcp } +import akka.util.{ ByteString, ByteStringBuilder } + +/** + * INTERNAL API + */ +// FIXME: Replace the codec with a custom made, hi-perf one +private[remote] class Transport(val localAddress: Address, + val system: ExtendedActorSystem, + val provider: RemoteActorRefProvider, + val codec: AkkaPduCodec, + val inboundDispatcher: InboundMessageDispatcher) { + + val killSwitch: SharedKillSwitch = KillSwitches.shared("transportKillSwitch") + + def outbound(remoteAddress: Address): Sink[Send, Any] = { + val remoteInetSocketAddress = new InetSocketAddress( + remoteAddress.host.get, + remoteAddress.port.get) + + Flow.fromGraph(killSwitch.flow[Send]) + .via(encoder) + .via(Tcp(system).outgoingConnection(remoteInetSocketAddress, halfClose = false)) + .to(Sink.ignore) + } + + // TODO: Try out parallelized serialization (mapAsync) for performance + val encoder: Flow[Send, ByteString, NotUsed] = Flow[Send].map { sendEnvelope ⇒ + val pdu: ByteString = codec.constructMessage( + sendEnvelope.recipient.localAddressToUse, + sendEnvelope.recipient, + Serialization.currentTransportInformation.withValue(Serialization.Information(localAddress, system)) { + MessageSerializer.serialize(system, sendEnvelope.message.asInstanceOf[AnyRef]) + }, + sendEnvelope.senderOption, + seqOption = None, // FIXME: Acknowledgements will be handled differently I just reused the old codec + ackOption = None) + + // TODO: Drop unserializable messages + // TODO: Drop oversized messages + (new ByteStringBuilder).putInt(pdu.size)(ByteOrder.LITTLE_ENDIAN).result() ++ pdu + } + + val decoder: Flow[ByteString, AkkaPduCodec.Message, NotUsed] = + Framing.lengthField(4, maximumFrameLength = 256000) + .map { frame ⇒ + // TODO: Drop unserializable messages + val pdu = codec.decodeMessage(frame.drop(4), provider, localAddress)._2.get + pdu + } + + val messageDispatcher: Sink[AkkaPduCodec.Message, Any] = Sink.foreach[AkkaPduCodec.Message] { m ⇒ + inboundDispatcher.dispatch(m.recipient, m.recipientAddress, m.serializedMessage, m.senderOption) + } + + val inboundFlow: Flow[ByteString, ByteString, NotUsed] = { + Flow.fromSinkAndSource( + decoder.to(messageDispatcher), + Source.maybe[ByteString].via(killSwitch.flow)) + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/ArterySmokeTest.scala b/akka-remote/src/test/scala/akka/remote/artery/ArterySmokeTest.scala new file mode 100644 index 0000000000..9c78971bdd --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/ArterySmokeTest.scala @@ -0,0 +1,57 @@ +package akka.remote.artery + +import akka.actor.{ Actor, ActorIdentity, ActorSystem, ExtendedActorSystem, Identify, Props, RootActorPath } +import akka.testkit.{ AkkaSpec, ImplicitSender } +import com.typesafe.config.ConfigFactory + +import ArterySmokeTest._ + +object ArterySmokeTest { + + val commonConfig = """ + akka { + actor.provider = "akka.remote.RemoteActorRefProvider" + remote.artery.enabled = on + } + """ + +} + +class ArterySmokeTest extends AkkaSpec(commonConfig) with ImplicitSender { + + val configB = ConfigFactory.parseString("akka.remote.artery.port = 20201") + val systemB = ActorSystem("systemB", configB.withFallback(system.settings.config)) + val addressB = systemB.asInstanceOf[ExtendedActorSystem].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].ref.get + } + + remoteRef ! "ping" + expectMsg("pong") + + remoteRef ! "ping" + expectMsg("pong") + + remoteRef ! "ping" + expectMsg("pong") + + } + + } + + override def afterTermination(): Unit = shutdown(systemB) + +} diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index c3368df85b..a2a08c475a 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -120,7 +120,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", protobuf) ) lazy val multiNodeTestkit = Project( diff --git a/project/MiMa.scala b/project/MiMa.scala index dcc2959cd7..8a76c58cf5 100644 --- a/project/MiMa.scala +++ b/project/MiMa.scala @@ -714,7 +714,11 @@ object MiMa extends AutoPlugin { ProblemFilters.exclude[ReversedAbstractMethodProblem]("akka.persistence.Eventsourced#ProcessingState.onWriteMessageComplete"), // #19390 Add flow monitor - ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOpsMat.monitor") + ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOpsMat.monitor"), + + // 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") ) ) } From f5b88f76205bd04e9a8910b0e84ca2acc172d2e3 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 14 Apr 2016 07:30:18 +0200 Subject: [PATCH 004/186] move AeronSource, AeronSink to remote --- akka-bench-jmh/src/main/scala/akka/aeron/AeronStreams.scala | 2 ++ .../src/main/scala/akka/remote/artery}/AeronSink.scala | 2 +- .../src/main/scala/akka/remote/artery}/AeronSource.scala | 2 +- project/AkkaBuild.scala | 2 +- project/Dependencies.scala | 4 ++-- 5 files changed, 7 insertions(+), 5 deletions(-) rename {akka-bench-jmh/src/main/scala/akka/aeron => akka-remote/src/main/scala/akka/remote/artery}/AeronSink.scala (99%) rename {akka-bench-jmh/src/main/scala/akka/aeron => akka-remote/src/main/scala/akka/remote/artery}/AeronSource.scala (99%) diff --git a/akka-bench-jmh/src/main/scala/akka/aeron/AeronStreams.scala b/akka-bench-jmh/src/main/scala/akka/aeron/AeronStreams.scala index e73d87c994..9857db6e69 100644 --- a/akka-bench-jmh/src/main/scala/akka/aeron/AeronStreams.scala +++ b/akka-bench-jmh/src/main/scala/akka/aeron/AeronStreams.scala @@ -17,6 +17,8 @@ import java.util.concurrent.CountDownLatch import java.util.concurrent.CyclicBarrier import java.util.concurrent.atomic.AtomicLongArray import akka.stream.ThrottleMode +import akka.remote.artery.AeronSink +import akka.remote.artery.AeronSource object AeronStreams { diff --git a/akka-bench-jmh/src/main/scala/akka/aeron/AeronSink.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala similarity index 99% rename from akka-bench-jmh/src/main/scala/akka/aeron/AeronSink.scala rename to akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala index c595075a43..c002695696 100644 --- a/akka-bench-jmh/src/main/scala/akka/aeron/AeronSink.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala @@ -1,4 +1,4 @@ -package akka.aeron +package akka.remote.artery import java.nio.ByteBuffer import java.util.concurrent.TimeUnit diff --git a/akka-bench-jmh/src/main/scala/akka/aeron/AeronSource.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala similarity index 99% rename from akka-bench-jmh/src/main/scala/akka/aeron/AeronSource.scala rename to akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala index 0a7ff3f6e1..a2bdaee683 100644 --- a/akka-bench-jmh/src/main/scala/akka/aeron/AeronSource.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala @@ -1,4 +1,4 @@ -package akka.aeron +package akka.remote.artery import java.nio.ByteBuffer import java.util.concurrent.TimeUnit diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index a2a08c475a..f0594b982e 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -107,7 +107,7 @@ object AkkaBuild extends Build { dependencies = Seq( actor, http, stream, streamTests, - persistence, distributedData, + remote, persistence, distributedData, testkit ).map(_ % "compile;compile->test;provided->provided") ).disablePlugins(ValidatePullRequest) diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 65438b552c..b55dc7d954 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -129,7 +129,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) val remoteTests = l ++= Seq(Test.junit, Test.scalatest.value, Test.scalaXml) @@ -165,7 +165,7 @@ object Dependencies { val contrib = l ++= Seq(Test.junitIntf, Test.commonsIo) - val benchJmh = l ++= Seq(Provided.levelDB, Provided.levelDBNative, aeronDriver, aeronClient, hdrHistogram) + val benchJmh = l ++= Seq(Provided.levelDB, Provided.levelDBNative, hdrHistogram) // akka stream & http From 5088142307118768493f57a12cbf27e87aa30fac Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 14 Apr 2016 10:32:01 +0200 Subject: [PATCH 005/186] add Aeron to Artery --- akka-remote/src/main/resources/reference.conf | 3 + .../scala/akka/remote/RemoteSettings.scala | 2 + .../scala/akka/remote/artery/AeronSink.scala | 6 +- .../akka/remote/artery/AeronSource.scala | 8 +- .../akka/remote/artery/ArterySubsystem.scala | 42 +++--- .../scala/akka/remote/artery/Transport.scala | 129 +++++++++++++++--- .../akka/remote/artery/ArterySmokeTest.scala | 6 +- 7 files changed, 155 insertions(+), 41 deletions(-) diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index f23382ba44..6765b5f9ec 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -75,6 +75,9 @@ akka { artery { enabled = off port = 20200 + hostname = localhost + # tcp, aeron-udp + transport = tcp } ### General settings diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index 406ced337e..7cf64af82d 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -22,6 +22,8 @@ final class RemoteSettings(val config: Config) { val EnableArtery: Boolean = getBoolean("akka.remote.artery.enabled") val ArteryPort: Int = getInt("akka.remote.artery.port") + val ArteryHostname: String = getString("akka.remote.artery.hostname") + val ArteryTransport: String = getString("akka.remote.artery.transport") val LogReceive: Boolean = getBoolean("akka.remote.log-received-messages") diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala index c002695696..637bd8e7e6 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala @@ -25,7 +25,7 @@ object AeronSink { /** * @param channel eg. "aeron:udp?endpoint=localhost:40123" */ -class AeronSink(channel: String, aeron: () => Aeron) extends GraphStage[SinkShape[AeronSink.Bytes]] { +class AeronSink(channel: String, aeron: () ⇒ Aeron) extends GraphStage[SinkShape[AeronSink.Bytes]] { import AeronSink._ val in: Inlet[Bytes] = Inlet("AeronSink") @@ -90,8 +90,8 @@ class AeronSink(channel: String, aeron: () => Aeron) extends GraphStage[SinkShap override protected def onTimer(timerKey: Any): Unit = { timerKey match { - case Backoff => publish() - case msg => super.onTimer(msg) + case Backoff ⇒ publish() + case msg ⇒ super.onTimer(msg) } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala index a2bdaee683..cdc2333cfa 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala @@ -29,7 +29,7 @@ object AeronSource { /** * @param channel eg. "aeron:udp?endpoint=localhost:40123" */ -class AeronSource(channel: String, aeron: () => Aeron) extends GraphStage[SourceShape[AeronSource.Bytes]] { +class AeronSource(channel: String, aeron: () ⇒ Aeron) extends GraphStage[SourceShape[AeronSource.Bytes]] { import AeronSource._ val out: Outlet[Bytes] = Outlet("AeronSource") @@ -47,7 +47,7 @@ class AeronSource(channel: String, aeron: () => Aeron) extends GraphStage[Source private val retries = 115 private var backoffCount = retries - val receiveMessage = getAsyncCallback[Bytes] { data => + val receiveMessage = getAsyncCallback[Bytes] { data ⇒ push(out, data) } @@ -96,8 +96,8 @@ class AeronSource(channel: String, aeron: () => Aeron) extends GraphStage[Source override protected def onTimer(timerKey: Any): Unit = { timerKey match { - case Backoff => subscriberLoop() - case msg => super.onTimer(msg) + case Backoff ⇒ subscriberLoop() + case msg ⇒ super.onTimer(msg) } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala index 0b007d6415..77f85bf093 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala @@ -22,9 +22,11 @@ import scala.concurrent.{ Await, Future } * INTERNAL API */ private[remote] class ArterySubsystem(_system: ExtendedActorSystem, _provider: RemoteActorRefProvider) extends RemoteTransport(_system, _provider) { + import provider.remoteSettings + @volatile private[this] var address: Address = _ @volatile private[this] var transport: Transport = _ - @volatile private[this] var binding: Tcp.ServerBinding = _ + @volatile private[this] var tcpBinding: Option[Tcp.ServerBinding] = None @volatile private[this] var materializer: Materializer = _ override val log: LoggingAdapter = Logging(system.eventStream, getClass.getName) @@ -38,27 +40,35 @@ private[remote] class ArterySubsystem(_system: ExtendedActorSystem, _provider: R override def start(): Unit = { // TODO: Configure materializer properly // TODO: Have a supervisor actor - address = Address("akka.artery", system.name, "localhost", provider.remoteSettings.ArteryPort) + address = Address("akka.artery", system.name, remoteSettings.ArteryHostname, remoteSettings.ArteryPort) materializer = ActorMaterializer()(system) - transport = new Transport( - address, - system, - provider, - AkkaPduProtobufCodec, - new DefaultMessageDispatcher(system, provider, log)) - binding = Await.result( - Tcp(system).bindAndHandle(transport.inboundFlow, address.host.get, address.port.get)(materializer), - 3.seconds) + transport = remoteSettings.ArteryTransport match { + case "tcp" ⇒ + new TcpTransport( + address, + system, + materializer, + provider, + AkkaPduProtobufCodec, + new DefaultMessageDispatcher(system, provider, log)) + case "aeron-udp" ⇒ + new AeronTransport( + address, + system, + materializer, + provider, + AkkaPduProtobufCodec, + new DefaultMessageDispatcher(system, provider, log)) + case unknown ⇒ throw new IllegalArgumentException(s"Unknown transport $unknown") + } - log.info("Artery started up with address {}", binding.localAddress) + transport.start() } override def shutdown(): Future[Done] = { - import system.dispatcher - binding.unbind().map(_ ⇒ Done).andThen { - case _ ⇒ transport.killSwitch.abort(new Exception("System shut down")) - } + if (transport != null) transport.shutdown() + else Future.successful(Done) } override def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = { diff --git a/akka-remote/src/main/scala/akka/remote/artery/Transport.scala b/akka-remote/src/main/scala/akka/remote/artery/Transport.scala index 9957ce5d0d..9c52f4901f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Transport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Transport.scala @@ -4,9 +4,9 @@ package akka.remote.artery +import scala.concurrent.duration._ import java.net.InetSocketAddress import java.nio.ByteOrder - import akka.NotUsed import akka.actor.{ Address, ExtendedActorSystem } import akka.remote.EndpointManager.Send @@ -16,30 +16,29 @@ import akka.serialization.Serialization import akka.stream.{ KillSwitches, SharedKillSwitch } import akka.stream.scaladsl.{ Flow, Framing, Sink, Source, Tcp } import akka.util.{ ByteString, ByteStringBuilder } +import scala.concurrent.Future +import akka.Done +import akka.stream.Materializer +import scala.concurrent.Await +import akka.event.LoggingAdapter +import akka.event.Logging +import io.aeron.driver.MediaDriver +import io.aeron.Aeron /** * INTERNAL API */ // FIXME: Replace the codec with a custom made, hi-perf one -private[remote] class Transport(val localAddress: Address, - val system: ExtendedActorSystem, - val provider: RemoteActorRefProvider, - val codec: AkkaPduCodec, - val inboundDispatcher: InboundMessageDispatcher) { +private[remote] abstract class Transport(val localAddress: Address, + val system: ExtendedActorSystem, + val provider: RemoteActorRefProvider, + val codec: AkkaPduCodec, + val inboundDispatcher: InboundMessageDispatcher) { + + val log: LoggingAdapter = Logging(system.eventStream, getClass.getName) val killSwitch: SharedKillSwitch = KillSwitches.shared("transportKillSwitch") - def outbound(remoteAddress: Address): Sink[Send, Any] = { - val remoteInetSocketAddress = new InetSocketAddress( - remoteAddress.host.get, - remoteAddress.port.get) - - Flow.fromGraph(killSwitch.flow[Send]) - .via(encoder) - .via(Tcp(system).outgoingConnection(remoteInetSocketAddress, halfClose = false)) - .to(Sink.ignore) - } - // TODO: Try out parallelized serialization (mapAsync) for performance val encoder: Flow[Send, ByteString, NotUsed] = Flow[Send].map { sendEnvelope ⇒ val pdu: ByteString = codec.constructMessage( @@ -75,4 +74,100 @@ private[remote] class Transport(val localAddress: Address, Source.maybe[ByteString].via(killSwitch.flow)) } + def start(): Unit + + def shutdown(): Future[Done] + + def outbound(remoteAddress: Address): Sink[Send, Any] +} + +/** + * INTERNAL API + */ +private[remote] class TcpTransport( + localAddress: Address, + system: ExtendedActorSystem, + materializer: Materializer, + provider: RemoteActorRefProvider, + codec: AkkaPduCodec, + inboundDispatcher: InboundMessageDispatcher) + extends Transport(localAddress, system, provider, codec, inboundDispatcher) { + + @volatile private[this] var binding: Tcp.ServerBinding = _ + + override def start(): Unit = { + binding = Await.result( + Tcp(system).bindAndHandle(inboundFlow, localAddress.host.get, localAddress.port.get)(materializer), + 3.seconds) + log.info("Artery TCP started up with address {}", binding.localAddress) + } + + override def shutdown(): Future[Done] = { + import system.dispatcher + if (binding != null) { + binding.unbind().map(_ ⇒ Done).andThen { + case _ ⇒ killSwitch.abort(new Exception("System shut down")) + } + } else + Future.successful(Done) + } + + override def outbound(remoteAddress: Address): Sink[Send, Any] = { + val remoteInetSocketAddress = new InetSocketAddress( + remoteAddress.host.get, + remoteAddress.port.get) + + Flow.fromGraph(killSwitch.flow[Send]) + .via(encoder) + .via(Tcp(system).outgoingConnection(remoteInetSocketAddress, halfClose = false)) + .to(Sink.ignore) + } +} + +/** + * INTERNAL API + */ +private[remote] class AeronTransport( + localAddress: Address, + system: ExtendedActorSystem, + materializer: Materializer, + provider: RemoteActorRefProvider, + codec: AkkaPduCodec, + inboundDispatcher: InboundMessageDispatcher) + extends Transport(localAddress, system, provider, codec, inboundDispatcher) { + + private implicit val mat = materializer + // TODO support port 0 + private val inboundChannel = s"aeron:udp?endpoint=${localAddress.host.get}:${localAddress.port.get}" + + private val aeron = { + val ctx = new Aeron.Context + // TODO also support external media driver + val driver = MediaDriver.launchEmbedded() + ctx.aeronDirectoryName(driver.aeronDirectoryName) + Aeron.connect(ctx) + } + + override def start(): Unit = { + Source.fromGraph(new AeronSource(inboundChannel, () ⇒ aeron)) + .map(ByteString.apply) // TODO we should use ByteString all the way + .via(inboundFlow) + .runWith(Sink.ignore) + } + + override def shutdown(): Future[Done] = { + // FIXME stop the AeronSource first? + aeron.close() + Future.successful(Done) + } + + override def outbound(remoteAddress: Address): Sink[Send, Any] = { + val outboundChannel = s"aeron:udp?endpoint=${remoteAddress.host.get}:${remoteAddress.port.get}" + Flow.fromGraph(killSwitch.flow[Send]) + .via(encoder) + .map(_.toArray) // TODO we should use ByteString all the way + .to(new AeronSink(outboundChannel, () ⇒ aeron)) + } + + // FIXME we don't need Framing for Aeron, since it has fragmentation } diff --git a/akka-remote/src/test/scala/akka/remote/artery/ArterySmokeTest.scala b/akka-remote/src/test/scala/akka/remote/artery/ArterySmokeTest.scala index 9c78971bdd..5c1bed24f5 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/ArterySmokeTest.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/ArterySmokeTest.scala @@ -11,7 +11,11 @@ object ArterySmokeTest { val commonConfig = """ akka { actor.provider = "akka.remote.RemoteActorRefProvider" - remote.artery.enabled = on + remote.artery { + enabled = on + #transport = tcp + transport = aeron-udp + } } """ From 49168f6111485096c7af8cc5ea6cb1d01e920880 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 14 Apr 2016 11:01:53 +0200 Subject: [PATCH 006/186] remove TcpTransport --- akka-remote/src/main/resources/reference.conf | 2 - .../scala/akka/remote/RemoteSettings.scala | 1 - .../akka/remote/artery/ArterySubsystem.scala | 28 +--- .../scala/akka/remote/artery/Transport.scala | 143 +++++------------- .../akka/remote/artery/ArterySmokeTest.scala | 6 +- 5 files changed, 50 insertions(+), 130 deletions(-) diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 6765b5f9ec..e2aa5d8538 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -76,8 +76,6 @@ akka { enabled = off port = 20200 hostname = localhost - # tcp, aeron-udp - transport = tcp } ### General settings diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index 7cf64af82d..97806381ff 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -23,7 +23,6 @@ final class RemoteSettings(val config: Config) { val EnableArtery: Boolean = getBoolean("akka.remote.artery.enabled") val ArteryPort: Int = getInt("akka.remote.artery.port") val ArteryHostname: String = getString("akka.remote.artery.hostname") - val ArteryTransport: String = getString("akka.remote.artery.transport") val LogReceive: Boolean = getBoolean("akka.remote.log-received-messages") diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala index 77f85bf093..d9bf776709 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala @@ -43,26 +43,14 @@ private[remote] class ArterySubsystem(_system: ExtendedActorSystem, _provider: R address = Address("akka.artery", system.name, remoteSettings.ArteryHostname, remoteSettings.ArteryPort) materializer = ActorMaterializer()(system) - transport = remoteSettings.ArteryTransport match { - case "tcp" ⇒ - new TcpTransport( - address, - system, - materializer, - provider, - AkkaPduProtobufCodec, - new DefaultMessageDispatcher(system, provider, log)) - case "aeron-udp" ⇒ - new AeronTransport( - address, - system, - materializer, - provider, - AkkaPduProtobufCodec, - new DefaultMessageDispatcher(system, provider, log)) - case unknown ⇒ throw new IllegalArgumentException(s"Unknown transport $unknown") - } - + transport = + new Transport( + address, + system, + materializer, + provider, + AkkaPduProtobufCodec, + new DefaultMessageDispatcher(system, provider, log)) transport.start() } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Transport.scala b/akka-remote/src/main/scala/akka/remote/artery/Transport.scala index 9c52f4901f..88c55b3ff4 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Transport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Transport.scala @@ -29,16 +29,51 @@ import io.aeron.Aeron * INTERNAL API */ // FIXME: Replace the codec with a custom made, hi-perf one -private[remote] abstract class Transport(val localAddress: Address, - val system: ExtendedActorSystem, - val provider: RemoteActorRefProvider, - val codec: AkkaPduCodec, - val inboundDispatcher: InboundMessageDispatcher) { +private[remote] class Transport( + val localAddress: Address, + val system: ExtendedActorSystem, + val materializer: Materializer, + val provider: RemoteActorRefProvider, + val codec: AkkaPduCodec, + val inboundDispatcher: InboundMessageDispatcher) { - val log: LoggingAdapter = Logging(system.eventStream, getClass.getName) + private val log: LoggingAdapter = Logging(system.eventStream, getClass.getName) + + private implicit val mat = materializer + // TODO support port 0 + private val inboundChannel = s"aeron:udp?endpoint=${localAddress.host.get}:${localAddress.port.get}" + + private val aeron = { + val ctx = new Aeron.Context + // TODO also support external media driver + val driver = MediaDriver.launchEmbedded() + ctx.aeronDirectoryName(driver.aeronDirectoryName) + Aeron.connect(ctx) + } + + def start(): Unit = { + Source.fromGraph(new AeronSource(inboundChannel, () ⇒ aeron)) + .map(ByteString.apply) // TODO we should use ByteString all the way + .via(inboundFlow) + .runWith(Sink.ignore) + } + + def shutdown(): Future[Done] = { + // FIXME stop the AeronSource first? + aeron.close() + Future.successful(Done) + } val killSwitch: SharedKillSwitch = KillSwitches.shared("transportKillSwitch") + def outbound(remoteAddress: Address): Sink[Send, Any] = { + val outboundChannel = s"aeron:udp?endpoint=${remoteAddress.host.get}:${remoteAddress.port.get}" + Flow.fromGraph(killSwitch.flow[Send]) + .via(encoder) + .map(_.toArray) // TODO we should use ByteString all the way + .to(new AeronSink(outboundChannel, () ⇒ aeron)) + } + // TODO: Try out parallelized serialization (mapAsync) for performance val encoder: Flow[Send, ByteString, NotUsed] = Flow[Send].map { sendEnvelope ⇒ val pdu: ByteString = codec.constructMessage( @@ -74,100 +109,4 @@ private[remote] abstract class Transport(val localAddress: Address, Source.maybe[ByteString].via(killSwitch.flow)) } - def start(): Unit - - def shutdown(): Future[Done] - - def outbound(remoteAddress: Address): Sink[Send, Any] -} - -/** - * INTERNAL API - */ -private[remote] class TcpTransport( - localAddress: Address, - system: ExtendedActorSystem, - materializer: Materializer, - provider: RemoteActorRefProvider, - codec: AkkaPduCodec, - inboundDispatcher: InboundMessageDispatcher) - extends Transport(localAddress, system, provider, codec, inboundDispatcher) { - - @volatile private[this] var binding: Tcp.ServerBinding = _ - - override def start(): Unit = { - binding = Await.result( - Tcp(system).bindAndHandle(inboundFlow, localAddress.host.get, localAddress.port.get)(materializer), - 3.seconds) - log.info("Artery TCP started up with address {}", binding.localAddress) - } - - override def shutdown(): Future[Done] = { - import system.dispatcher - if (binding != null) { - binding.unbind().map(_ ⇒ Done).andThen { - case _ ⇒ killSwitch.abort(new Exception("System shut down")) - } - } else - Future.successful(Done) - } - - override def outbound(remoteAddress: Address): Sink[Send, Any] = { - val remoteInetSocketAddress = new InetSocketAddress( - remoteAddress.host.get, - remoteAddress.port.get) - - Flow.fromGraph(killSwitch.flow[Send]) - .via(encoder) - .via(Tcp(system).outgoingConnection(remoteInetSocketAddress, halfClose = false)) - .to(Sink.ignore) - } -} - -/** - * INTERNAL API - */ -private[remote] class AeronTransport( - localAddress: Address, - system: ExtendedActorSystem, - materializer: Materializer, - provider: RemoteActorRefProvider, - codec: AkkaPduCodec, - inboundDispatcher: InboundMessageDispatcher) - extends Transport(localAddress, system, provider, codec, inboundDispatcher) { - - private implicit val mat = materializer - // TODO support port 0 - private val inboundChannel = s"aeron:udp?endpoint=${localAddress.host.get}:${localAddress.port.get}" - - private val aeron = { - val ctx = new Aeron.Context - // TODO also support external media driver - val driver = MediaDriver.launchEmbedded() - ctx.aeronDirectoryName(driver.aeronDirectoryName) - Aeron.connect(ctx) - } - - override def start(): Unit = { - Source.fromGraph(new AeronSource(inboundChannel, () ⇒ aeron)) - .map(ByteString.apply) // TODO we should use ByteString all the way - .via(inboundFlow) - .runWith(Sink.ignore) - } - - override def shutdown(): Future[Done] = { - // FIXME stop the AeronSource first? - aeron.close() - Future.successful(Done) - } - - override def outbound(remoteAddress: Address): Sink[Send, Any] = { - val outboundChannel = s"aeron:udp?endpoint=${remoteAddress.host.get}:${remoteAddress.port.get}" - Flow.fromGraph(killSwitch.flow[Send]) - .via(encoder) - .map(_.toArray) // TODO we should use ByteString all the way - .to(new AeronSink(outboundChannel, () ⇒ aeron)) - } - - // FIXME we don't need Framing for Aeron, since it has fragmentation } diff --git a/akka-remote/src/test/scala/akka/remote/artery/ArterySmokeTest.scala b/akka-remote/src/test/scala/akka/remote/artery/ArterySmokeTest.scala index 5c1bed24f5..9c78971bdd 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/ArterySmokeTest.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/ArterySmokeTest.scala @@ -11,11 +11,7 @@ object ArterySmokeTest { val commonConfig = """ akka { actor.provider = "akka.remote.RemoteActorRefProvider" - remote.artery { - enabled = on - #transport = tcp - transport = aeron-udp - } + remote.artery.enabled = on } """ From 9cd8c267f36b1e1e1e2b05b70b3fd5684f08a7eb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Endre=20S=C3=A1ndor=20Varga?= Date: Thu, 14 Apr 2016 11:42:28 +0200 Subject: [PATCH 007/186] #20311: Fix association race condition and add reproducer --- .../akka/remote/artery/ArterySubsystem.scala | 23 ++--- .../akka/remote/artery/ArterySmokeTest.scala | 57 ----------- .../artery/RemoteSendConsistencySpec.scala | 97 +++++++++++++++++++ 3 files changed, 107 insertions(+), 70 deletions(-) delete mode 100644 akka-remote/src/test/scala/akka/remote/artery/ArterySmokeTest.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala index 0b007d6415..0a780d36c4 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala @@ -67,30 +67,27 @@ private[remote] class ArterySubsystem(_system: ExtendedActorSystem, _provider: R val association = if (cached ne null) cached - else { - val association = getAssociation(remoteAddress) - association.associate() - recipient.cachedAssociation = association - association - } + else associate(remoteAddress) association.send(message, senderOption, recipient) } - private def getAssociation(remoteAddress: Address): Association = { + private def associate(remoteAddress: Address): Association = { val current = associations.get(remoteAddress) if (current ne null) current else { - val newAssociation = new Association(materializer, remoteAddress, transport) - val currentAssociation = associations.putIfAbsent(remoteAddress, newAssociation) - if (currentAssociation eq null) { - newAssociation - } else currentAssociation + associations.computeIfAbsent(remoteAddress, new java.util.function.Function[Address, Association] { + override def apply(remoteAddress: Address): Association = { + val newAssociation = new Association(materializer, remoteAddress, transport) + newAssociation.associate() // This is a bit costly for this blocking method :( + newAssociation + } + }) } } override def quarantine(remoteAddress: Address, uid: Option[Int]): Unit = { - getAssociation(remoteAddress).quarantine(uid) + ??? } } diff --git a/akka-remote/src/test/scala/akka/remote/artery/ArterySmokeTest.scala b/akka-remote/src/test/scala/akka/remote/artery/ArterySmokeTest.scala deleted file mode 100644 index 9c78971bdd..0000000000 --- a/akka-remote/src/test/scala/akka/remote/artery/ArterySmokeTest.scala +++ /dev/null @@ -1,57 +0,0 @@ -package akka.remote.artery - -import akka.actor.{ Actor, ActorIdentity, ActorSystem, ExtendedActorSystem, Identify, Props, RootActorPath } -import akka.testkit.{ AkkaSpec, ImplicitSender } -import com.typesafe.config.ConfigFactory - -import ArterySmokeTest._ - -object ArterySmokeTest { - - val commonConfig = """ - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.artery.enabled = on - } - """ - -} - -class ArterySmokeTest extends AkkaSpec(commonConfig) with ImplicitSender { - - val configB = ConfigFactory.parseString("akka.remote.artery.port = 20201") - val systemB = ActorSystem("systemB", configB.withFallback(system.settings.config)) - val addressB = systemB.asInstanceOf[ExtendedActorSystem].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].ref.get - } - - remoteRef ! "ping" - expectMsg("pong") - - remoteRef ! "ping" - expectMsg("pong") - - remoteRef ! "ping" - expectMsg("pong") - - } - - } - - override def afterTermination(): Unit = shutdown(systemB) - -} 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..34013720c6 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala @@ -0,0 +1,97 @@ +package akka.remote.artery + +import akka.actor.{ Actor, ActorIdentity, ActorSystem, Deploy, ExtendedActorSystem, Identify, Props, RootActorPath } +import akka.testkit.{ AkkaSpec, ImplicitSender } +import com.typesafe.config.ConfigFactory +import RemoteSendConsistencySpec._ +import akka.actor.Actor.Receive + +object RemoteSendConsistencySpec { + + val commonConfig = """ + akka { + actor.provider = "akka.remote.RemoteActorRefProvider" + remote.artery.enabled = on + } + """ + +} + +class RemoteSendConsistencySpec extends AkkaSpec(commonConfig) with ImplicitSender { + + val configB = ConfigFactory.parseString("akka.remote.artery.port = 20201") + val systemB = ActorSystem("systemB", configB.withFallback(system.settings.config)) + val addressB = systemB.asInstanceOf[ExtendedActorSystem].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].ref.get + } + + remoteRef ! "ping" + expectMsg("pong") + + remoteRef ! "ping" + expectMsg("pong") + + remoteRef ! "ping" + expectMsg("pong") + + } + + "be able to send messages concurrently preserving order" in { + val actorOnSystemB = systemB.actorOf(Props(new Actor { + def receive = { + case i: Int ⇒ sender() ! i + } + }), "echo2") + + val remoteRef = { + system.actorSelection(rootB / "user" / "echo2") ! Identify(None) + expectMsgType[ActorIdentity].ref.get + } + + val senderProps = Props(new Actor { + var counter = 1000 + remoteRef ! 1000 + + 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) + system.actorOf(senderProps) + system.actorOf(senderProps) + system.actorOf(senderProps) + + expectMsg("success") + expectMsg("success") + expectMsg("success") + expectMsg("success") + } + + } + + override def afterTermination(): Unit = shutdown(systemB) + +} From 777bd1b85c23bb292512af35c790965072e8765b Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 14 Apr 2016 17:07:09 +0200 Subject: [PATCH 008/186] disable some PR validation, during Artery prototyping --- project/AkkaBuild.scala | 71 +++++++++++++++++++++-------------------- 1 file changed, 36 insertions(+), 35 deletions(-) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index f0594b982e..1530ee7d50 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -58,7 +58,7 @@ object AkkaBuild extends Build { aggregate = Seq(actor, testkit, actorTests, remote, remoteTests, camel, cluster, clusterMetrics, clusterTools, clusterSharding, distributedData, slf4j, agent, persistence, persistenceQuery, persistenceTck, persistenceShared, - kernel, osgi, docs, contrib, samples, multiNodeTestkit, benchJmh, typed, protobuf, + kernel, osgi, contrib, multiNodeTestkit, benchJmh, typed, protobuf, stream, streamTestkit, streamTests, streamTestsTck, parsing, httpCore, http, httpSprayJson, httpXml, httpJackson, httpTests, httpTestkit ) @@ -81,25 +81,25 @@ object AkkaBuild extends Build { lazy val actor = Project( id = "akka-actor", base = file("akka-actor") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val testkit = Project( id = "akka-testkit", base = file("akka-testkit"), dependencies = Seq(actor) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val typed = Project( id = "akka-typed-experimental", base = file("akka-typed"), dependencies = Seq(testkit % "compile;test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val actorTests = Project( id = "akka-actor-tests", base = file("akka-actor-tests"), dependencies = Seq(testkit % "compile;test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val benchJmh = Project( id = "akka-bench-jmh", @@ -115,7 +115,7 @@ object AkkaBuild extends Build { lazy val protobuf = Project( id = "akka-protobuf", base = file("akka-protobuf") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val remote = Project( id = "akka-remote", @@ -127,7 +127,7 @@ object AkkaBuild extends Build { id = "akka-multi-node-testkit", base = file("akka-multi-node-testkit"), dependencies = Seq(remote, testkit) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val remoteTests = Project( id = "akka-remote-tests", @@ -139,19 +139,19 @@ object AkkaBuild extends Build { id = "akka-cluster", base = file("akka-cluster"), dependencies = Seq(remote, remoteTests % "test->test" , testkit % "test->test") - ) configs (MultiJvm) + ).disablePlugins(ValidatePullRequest, MimaPlugin) configs (MultiJvm) lazy val clusterMetrics = Project( id = "akka-cluster-metrics", base = file("akka-cluster-metrics"), dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm", slf4j % "test->compile") - ) configs (MultiJvm) + ).disablePlugins(ValidatePullRequest, MimaPlugin) configs (MultiJvm) lazy val clusterTools = Project( id = "akka-cluster-tools", base = file("akka-cluster-tools"), dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm") - ) configs (MultiJvm) + ).disablePlugins(ValidatePullRequest, MimaPlugin) configs (MultiJvm) lazy val clusterSharding = Project( id = "akka-cluster-sharding", @@ -162,31 +162,31 @@ object AkkaBuild extends Build { // provided. dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm", persistence % "compile;test->provided", distributedData % "provided;test", clusterTools) - ) configs (MultiJvm) + ).disablePlugins(ValidatePullRequest, MimaPlugin) configs (MultiJvm) lazy val distributedData = Project( id = "akka-distributed-data-experimental", base = file("akka-distributed-data"), dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm") - ) configs (MultiJvm) + ).disablePlugins(ValidatePullRequest, MimaPlugin) configs (MultiJvm) lazy val slf4j = Project( id = "akka-slf4j", base = file("akka-slf4j"), dependencies = Seq(actor, testkit % "test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val agent = Project( id = "akka-agent", base = file("akka-agent"), dependencies = Seq(actor, testkit % "test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val persistence = Project( id = "akka-persistence", base = file("akka-persistence"), dependencies = Seq(actor, testkit % "test->test", protobuf) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val persistenceQuery = Project( id = "akka-persistence-query-experimental", @@ -196,48 +196,48 @@ object AkkaBuild extends Build { persistence % "compile;provided->provided;test->test", testkit % "compile;test->test", streamTestkit % "compile;test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val persistenceTck = Project( id = "akka-persistence-tck", base = file("akka-persistence-tck"), dependencies = Seq(persistence % "compile;provided->provided;test->test", testkit % "compile;test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val persistenceShared = Project( id = "akka-persistence-shared", base = file("akka-persistence-shared"), dependencies = Seq(persistence % "test->test", testkit % "test->test", remote % "test", protobuf) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val httpCore = Project( id = "akka-http-core", base = file("akka-http-core"), dependencies = Seq(stream, parsing, streamTestkit % "test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val http = Project( id = "akka-http-experimental", base = file("akka-http"), dependencies = Seq(httpCore) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val httpTestkit = Project( id = "akka-http-testkit", base = file("akka-http-testkit"), dependencies = Seq(http, streamTestkit) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val httpTests = Project( id = "akka-http-tests", base = file("akka-http-tests"), dependencies = Seq(httpTestkit % "test", testkit % "test->test", httpSprayJson, httpXml, httpJackson) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val httpMarshallersScala = Project( id = "akka-http-marshallers-scala-experimental", base = file("akka-http-marshallers-scala") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) .settings(parentSettings: _*) .aggregate(httpSprayJson, httpXml) @@ -250,7 +250,7 @@ object AkkaBuild extends Build { lazy val httpMarshallersJava = Project( id = "akka-http-marshallers-java-experimental", base = file("akka-http-marshallers-java") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) .settings(parentSettings: _*) .aggregate(httpJackson) @@ -262,61 +262,61 @@ object AkkaBuild extends Build { id = s"akka-http-$name-experimental", base = file(s"akka-http-marshallers-scala/akka-http-$name"), dependencies = Seq(http) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) def httpMarshallersJavaSubproject(name: String) = Project( id = s"akka-http-$name-experimental", base = file(s"akka-http-marshallers-java/akka-http-$name"), dependencies = Seq(http) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val parsing = Project( id = "akka-parsing", base = file("akka-parsing") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val stream = Project( id = "akka-stream", base = file("akka-stream"), dependencies = Seq(actor) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val streamTestkit = Project( id = "akka-stream-testkit", base = file("akka-stream-testkit"), dependencies = Seq(stream, testkit % "compile;test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val streamTests = Project( id = "akka-stream-tests", base = file("akka-stream-tests"), dependencies = Seq(streamTestkit % "test->test", stream) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val streamTestsTck = Project( id = "akka-stream-tests-tck", base = file("akka-stream-tests-tck"), dependencies = Seq(streamTestkit % "test->test", stream) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val kernel = Project( id = "akka-kernel", base = file("akka-kernel"), dependencies = Seq(actor, testkit % "test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val camel = Project( id = "akka-camel", base = file("akka-camel"), dependencies = Seq(actor, slf4j, testkit % "test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val osgi = Project( id = "akka-osgi", base = file("akka-osgi"), dependencies = Seq(actor) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val docs = Project( id = "akka-docs", @@ -337,7 +337,7 @@ object AkkaBuild extends Build { id = "akka-contrib", base = file("akka-contrib"), dependencies = Seq(remote, remoteTests % "test->test", cluster, clusterTools, persistence % "compile;test->provided") - ) configs (MultiJvm) + ).disablePlugins(ValidatePullRequest, MimaPlugin) configs (MultiJvm) lazy val samplesSettings = parentSettings ++ ActivatorDist.settings @@ -401,6 +401,7 @@ object AkkaBuild extends Build { executeMvnCommands("Osgi sample Dining hakkers test failed", "clean", "install") }} ) + .disablePlugins(ValidatePullRequest, MimaPlugin) .settings(dontPublishSettings: _*) val dontPublishSettings = Seq( From 4eb76df3b78353c6bab834d50b458876297ff5dd Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 19 Apr 2016 17:36:28 +0200 Subject: [PATCH 009/186] improve Aeron source and sink * use FragmentAssembler for large messages * no need for aeron factory * some logging/listeners of aeron events and exceptions --- .../scala/akka/remote/artery/AeronSink.scala | 13 +++-- .../akka/remote/artery/AeronSource.scala | 57 +++++++++++-------- .../akka/remote/artery/ArterySubsystem.scala | 5 +- .../scala/akka/remote/artery/Transport.scala | 46 +++++++++++++-- 4 files changed, 84 insertions(+), 37 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala index 637bd8e7e6..06296c6e78 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala @@ -25,7 +25,7 @@ object AeronSink { /** * @param channel eg. "aeron:udp?endpoint=localhost:40123" */ -class AeronSink(channel: String, aeron: () ⇒ Aeron) extends GraphStage[SinkShape[AeronSink.Bytes]] { +class AeronSink(channel: String, aeron: Aeron) extends GraphStage[SinkShape[AeronSink.Bytes]] { import AeronSink._ val in: Inlet[Bytes] = Inlet("AeronSink") @@ -36,10 +36,10 @@ class AeronSink(channel: String, aeron: () ⇒ Aeron) extends GraphStage[SinkSha private val buffer = new UnsafeBuffer(ByteBuffer.allocateDirect(128 * 1024)) private val streamId = 10 - private val pub = aeron().addPublication(channel, streamId) + private val pub = aeron.addPublication(channel, streamId) private val idleStrategy = new BackoffIdleStrategy( 100, 10, TimeUnit.MICROSECONDS.toNanos(1), TimeUnit.MICROSECONDS.toNanos(100)) - private val retries = 120 + private val retries = 130 private var backoffCount = retries private var lastMsgSize = 0 @@ -68,13 +68,14 @@ class AeronSink(channel: String, aeron: () ⇒ Aeron) extends GraphStage[SinkSha if (backoffCount == 1) { println(s"# drop") // FIXME pull(in) // drop it - } else if (backoffCount <= 5) { + } else if (backoffCount <= 15) { // TODO Instead of using the scheduler we should handoff the task of // retrying/polling to a separate thread that performs the polling for // all sources/sinks and notifies back when there is some news. - // println(s"# scheduled backoff ${6 - backoffCount}") // FIXME backoffCount -= 1 - if (backoffCount <= 2) + if (backoffCount <= 10) + println(s"# snd backoff $backoffCount") // FIXME + if (backoffCount <= 10) scheduleOnce(Backoff, 50.millis) else scheduleOnce(Backoff, 1.millis) diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala index cdc2333cfa..e1c90b4b56 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala @@ -15,6 +15,7 @@ import akka.stream.stage.GraphStageLogic import akka.stream.stage.OutHandler import akka.stream.stage.TimerGraphStageLogic import io.aeron.Aeron +import io.aeron.FragmentAssembler import io.aeron.logbuffer.FragmentHandler import io.aeron.logbuffer.Header import org.agrona.DirectBuffer @@ -29,7 +30,7 @@ object AeronSource { /** * @param channel eg. "aeron:udp?endpoint=localhost:40123" */ -class AeronSource(channel: String, aeron: () ⇒ Aeron) extends GraphStage[SourceShape[AeronSource.Bytes]] { +class AeronSource(channel: String, aeron: Aeron) extends GraphStage[SourceShape[AeronSource.Bytes]] { import AeronSource._ val out: Outlet[Bytes] = Outlet("AeronSource") @@ -38,26 +39,30 @@ class AeronSource(channel: String, aeron: () ⇒ Aeron) extends GraphStage[Sourc override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new TimerGraphStageLogic(shape) with OutHandler { - private val buffer = new UnsafeBuffer(ByteBuffer.allocateDirect(256)) + private val buffer = new UnsafeBuffer(ByteBuffer.allocateDirect(128 * 1024)) private val streamId = 10 - private val sub = aeron().addSubscription(channel, streamId) + private val sub = aeron.addSubscription(channel, streamId) private val running = new AtomicBoolean(true) + private val spinning = 20000 + private val yielding = 0 + private val parking = 50 private val idleStrategy = new BackoffIdleStrategy( - 100, 10, TimeUnit.MICROSECONDS.toNanos(1), TimeUnit.MICROSECONDS.toNanos(100)) - private val retries = 115 - private var backoffCount = retries + spinning, yielding, TimeUnit.MICROSECONDS.toNanos(1), TimeUnit.MICROSECONDS.toNanos(100)) + private val idleStrategyRetries = spinning + yielding + parking + private var backoffCount = idleStrategyRetries + private val backoffDuration1 = 1.millis + private val backoffDuration2 = 50.millis + private var messageReceived = false - val receiveMessage = getAsyncCallback[Bytes] { data ⇒ - push(out, data) - } - - val fragmentHandler: FragmentHandler = new FragmentHandler { + // the fragmentHandler is called from `poll` in same thread, i.e. no async callback is needed + val fragmentHandler = new FragmentAssembler(new FragmentHandler { override def onFragment(buffer: DirectBuffer, offset: Int, length: Int, header: Header): Unit = { + messageReceived = true val data = Array.ofDim[Byte](length) buffer.getBytes(offset, data); - receiveMessage.invoke(data) + push(out, data) } - } + }) override def postStop(): Unit = { running.set(false) @@ -67,29 +72,31 @@ class AeronSource(channel: String, aeron: () ⇒ Aeron) extends GraphStage[Sourc // OutHandler override def onPull(): Unit = { idleStrategy.reset() - backoffCount = retries + backoffCount = idleStrategyRetries subscriberLoop() } @tailrec private def subscriberLoop(): Unit = if (running.get) { + messageReceived = false // will be set by the fragmentHandler if got full msg + // we only poll 1 fragment, otherwise we would have to use another buffer for + // received messages that can't be pushed val fragmentsRead = sub.poll(fragmentHandler, 1) - if (fragmentsRead <= 0) { + if (fragmentsRead > 0 && !messageReceived) + subscriberLoop() // recursive, read more fragments + else if (fragmentsRead <= 0) { // TODO the backoff strategy should be measured and tuned - if (backoffCount <= 0) { + backoffCount -= 1 + if (backoffCount > 0) { + idleStrategy.idle() + subscriberLoop() // recursive + } else if (backoffCount > -1000) { // TODO Instead of using the scheduler we should handoff the task of // retrying/polling to a separate thread that performs the polling for // all sources/sinks and notifies back when there is some news. - // println(s"# scheduled backoff ${0 - backoffCount + 1}") // FIXME - backoffCount -= 1 - if (backoffCount <= -5) - scheduleOnce(Backoff, 50.millis) - else - scheduleOnce(Backoff, 1.millis) + scheduleOnce(Backoff, backoffDuration1) } else { - idleStrategy.idle() - backoffCount -= 1 - subscriberLoop() // recursive + scheduleOnce(Backoff, backoffDuration2) } } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala index 714ea0c51e..bf950c5591 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala @@ -96,7 +96,7 @@ private[remote] class ArterySubsystem(_system: ExtendedActorSystem, _provider: R * Thread-safe, mutable holder for association state. Main entry point for remote destined message to a specific * remote address. */ -private[remote] class Association( +private[akka] class Association( val materializer: Materializer, val remoteAddress: Address, val transport: Transport) { @@ -113,7 +113,8 @@ private[remote] class Association( // Idempotent def associate(): Unit = { - queue = Source.queue(256, OverflowStrategy.dropBuffer).to(sink).run()(materializer) + if (queue eq null) + queue = Source.queue(256, OverflowStrategy.dropBuffer).to(sink).run()(materializer) } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Transport.scala b/akka-remote/src/main/scala/akka/remote/artery/Transport.scala index 88c55b3ff4..1b0302b126 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Transport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Transport.scala @@ -24,12 +24,17 @@ import akka.event.LoggingAdapter import akka.event.Logging import io.aeron.driver.MediaDriver import io.aeron.Aeron +import org.agrona.ErrorHandler +import io.aeron.AvailableImageHandler +import io.aeron.Image +import io.aeron.UnavailableImageHandler +import io.aeron.exceptions.ConductorServiceTimeoutException /** * INTERNAL API */ // FIXME: Replace the codec with a custom made, hi-perf one -private[remote] class Transport( +private[akka] class Transport( val localAddress: Address, val system: ExtendedActorSystem, val materializer: Materializer, @@ -45,14 +50,47 @@ private[remote] class Transport( private val aeron = { val ctx = new Aeron.Context + + 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}") + // FIXME we should call FragmentAssembler.freeSessionBuffer when image is unavailable + } + }) + ctx.errorHandler(new ErrorHandler { + override def onError(cause: Throwable): Unit = { + cause match { + case e: ConductorServiceTimeoutException ⇒ + // Timeout between service calls + log.error(cause, s"Aeron ServiceTimeoutException, ${cause.getMessage}") + + case _ ⇒ + log.error(cause, s"Aeron error, ${cause.getMessage}") + } + } + }) // TODO also support external media driver - val driver = MediaDriver.launchEmbedded() + val driverContext = new MediaDriver.Context + // FIXME settings from config + driverContext.clientLivenessTimeoutNs(SECONDS.toNanos(10)) + driverContext.imageLivenessTimeoutNs(SECONDS.toNanos(10)) + driverContext.driverTimeoutMs(SECONDS.toNanos(10)) + val driver = MediaDriver.launchEmbedded(driverContext) + ctx.aeronDirectoryName(driver.aeronDirectoryName) Aeron.connect(ctx) } def start(): Unit = { - Source.fromGraph(new AeronSource(inboundChannel, () ⇒ aeron)) + Source.fromGraph(new AeronSource(inboundChannel, aeron)) + .async // FIXME use dedicated dispatcher for AeronSource .map(ByteString.apply) // TODO we should use ByteString all the way .via(inboundFlow) .runWith(Sink.ignore) @@ -71,7 +109,7 @@ private[remote] class Transport( Flow.fromGraph(killSwitch.flow[Send]) .via(encoder) .map(_.toArray) // TODO we should use ByteString all the way - .to(new AeronSink(outboundChannel, () ⇒ aeron)) + .to(new AeronSink(outboundChannel, aeron)) } // TODO: Try out parallelized serialization (mapAsync) for performance From 263e3a3b9976c2220124800d20c25e769f41ff05 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 19 Apr 2016 17:38:26 +0200 Subject: [PATCH 010/186] add Artery throughput and latency tests * as multi-node tests * adjust multi-node-testkit to use the hostname for Artery * update HdrHistogram version --- .../akka/remote/testkit/MultiNodeSpec.scala | 1 + .../artery/AeronStreamLatencySpec.scala | 212 ++++++++++++++ .../artery/AeronStreamMaxThroughputSpec.scala | 194 ++++++++++++ .../akka/remote/artery/LatencySpec.scala | 270 +++++++++++++++++ .../remote/artery/MaxThroughputSpec.scala | 277 ++++++++++++++++++ akka-remote/src/main/resources/reference.conf | 8 +- .../scala/akka/remote/RemoteSettings.scala | 7 +- .../scala/akka/remote/artery/AeronSink.scala | 2 - .../akka/remote/artery}/RateReporter.java | 2 +- .../akka/remote/artery/AeronStreamsApp.scala | 77 +++-- .../artery/RemoteSendConsistencySpec.scala | 16 +- .../akka/testkit/metrics/HdrHistogram.scala | 4 +- project/AkkaBuild.scala | 2 +- project/Dependencies.scala | 5 +- 14 files changed, 1033 insertions(+), 44 deletions(-) create mode 100644 akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/AeronStreamLatencySpec.scala create mode 100644 akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/AeronStreamMaxThroughputSpec.scala create mode 100644 akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/LatencySpec.scala create mode 100644 akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/MaxThroughputSpec.scala rename {akka-bench-jmh/src/main/java/akka/aeron => akka-remote/src/test/java/akka/remote/artery}/RateReporter.java (99%) rename akka-bench-jmh/src/main/scala/akka/aeron/AeronStreams.scala => akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala (75%) 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 aae55a1db7..b8969bfc6a 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 @@ -196,6 +196,7 @@ object MultiNodeSpec { private[testkit] val nodeConfig = mapToConfig(Map( "akka.actor.provider" -> "akka.remote.RemoteActorRefProvider", + "akka.remote.artery.hostname" -> selfName, "akka.remote.netty.tcp.hostname" -> selfName, "akka.remote.netty.tcp.port" -> selfPort)) 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..c6cd4a7fe1 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/AeronStreamLatencySpec.scala @@ -0,0 +1,212 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.concurrent.CyclicBarrier +import java.util.concurrent.Executors +import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.atomic.AtomicLongArray + +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.ThrottleMode +import akka.stream.scaladsl.Source +import akka.testkit._ +import com.typesafe.config.ConfigFactory +import io.aeron.Aeron +import io.aeron.driver.MediaDriver +import org.HdrHistogram.Histogram + +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 = "akka.remote.RemoteActorRefProvider" + serialize-creators = false + serialize-messages = false + } + remote.artery.enabled = off + } + """))) + + def aeronPort(roleName: RoleName): Int = + roleName match { + case `first` ⇒ 20521 // TODO yeah, we should have support for dynamic port assignment + case `second` ⇒ 20522 + } + + 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") + + val aeron = { + val ctx = new Aeron.Context + val driver = MediaDriver.launchEmbedded() + ctx.aeronDirectoryName(driver.aeronDirectoryName) + Aeron.connect(ctx) + } + + 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}:${aeronPort(roleName)}" + } + + lazy val reporterExecutor = Executors.newFixedThreadPool(1) + def reporter(name: String): RateReporter = { + val r = new RateReporter(SECONDS.toNanos(1), new RateReporter.Reporter { + override def onReport(messagesPerSec: Double, bytesPerSec: Double, totalMessages: Long, totalBytes: Long): Unit = { + println(name + ": %.03g msgs/sec, %.03g bytes/sec, totals %d messages %d MB".format( + messagesPerSec, bytesPerSec, totalMessages, totalBytes / (1024 * 1024))) + } + }) + reporterExecutor.execute(r) + r + } + + override def afterAll(): Unit = { + reporterExecutor.shutdown() + super.afterAll() + } + + def printTotal(testName: String, payloadSize: Long, histogram: Histogram): 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) + + 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, ") + println("Histogram of RTT latencies in microseconds.") + histogram.outputPercentileDistribution(System.out, 1000.0) + } + + 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-1000-size-1k", + messageRate = 1000, + payloadSize = 1000, + repeat = repeatCount)) + + 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 barrier = new CyclicBarrier(2) + val count = new AtomicInteger + Source.fromGraph(new AeronSource(channel(first), aeron)) + .runForeach { bytes ⇒ + 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) { + printTotal(testName, bytes.length, histogram) + barrier.await() // this is always the last party + } + } + + for (n ← 1 to repeat) { + histogram.reset() + count.set(0) + + Source(1 to totalMessages) + .throttle(messageRate, 1.second, math.max(messageRate / 10, 1), ThrottleMode.Shaping) + .map { n ⇒ + sendTimes.set(n - 1, System.nanoTime()) + payload + } + .runWith(new AeronSink(channel(second), aeron)) + + barrier.await((totalMessages / messageRate) + 10, SECONDS) + } + + rep.halt() + } + + enterBarrier("after-" + testName) + } + + "Latency of Aeron Streams" must { + + "start echo" in { + runOn(second) { + // just echo back + Source.fromGraph(new AeronSource(channel(second), aeron)) + .runWith(new AeronSink(channel(first), aeron)) + } + 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..22b42bbbd0 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/AeronStreamMaxThroughputSpec.scala @@ -0,0 +1,194 @@ +/** + * 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 + +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 = "akka.remote.RemoteActorRefProvider" + serialize-creators = false + serialize-messages = false + } + remote.artery.enabled = off + } + """))) + + def aeronPort(roleName: RoleName): Int = + roleName match { + case `first` ⇒ 20511 // TODO yeah, we should have support for dynamic port assignment + case `second` ⇒ 20512 + } + + 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") + + val aeron = { + val ctx = new Aeron.Context + val driver = MediaDriver.launchEmbedded() + ctx.aeronDirectoryName(driver.aeronDirectoryName) + Aeron.connect(ctx) + } + + 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}:${aeronPort(roleName)}" + } + + lazy val reporterExecutor = Executors.newFixedThreadPool(1) + def reporter(name: String): RateReporter = { + val r = new RateReporter(SECONDS.toNanos(1), new RateReporter.Reporter { + override def onReport(messagesPerSec: Double, bytesPerSec: Double, totalMessages: Long, totalBytes: Long): Unit = { + println(name + ": %.03g msgs/sec, %.03g bytes/sec, totals %d messages %d MB".format( + messagesPerSec, bytesPerSec, totalMessages, totalBytes / (1024 * 1024))) + } + }) + reporterExecutor.execute(r) + r + } + + override def afterAll(): Unit = { + reporterExecutor.shutdown() + super.afterAll() + } + + def printTotal(testName: String, total: Long, startTime: Long, payloadSize: Long): Unit = { + val d = (System.nanoTime - startTime).nanos.toMillis + println(f"=== AeronStreamMaxThroughput $testName: " + + f"${1000.0 * total / d}%.03g msg/s, ${1000.0 * total * payloadSize / d}%.03g bytes/s, " + + s"payload size $payloadSize, " + + s"$d ms to deliver $total messages") + } + + 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) + Source.fromGraph(new AeronSource(channel(second), aeron)) + .runForeach { bytes ⇒ + rep.onMessage(1, bytes.length) + count += 1 + if (count == 1) { + t0 = System.nanoTime() + } else if (count == totalMessages) { + printTotal(testName, totalMessages, t0, payloadSize) + done.countDown() + } + }.onFailure { + case e ⇒ + e.printStackTrace + } + + enterBarrier(receiverName + "-started") + Await.ready(done, barrierTimeout) + rep.halt() + 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 ⇒ payload } + .runWith(new AeronSink(channel(second), aeron)) + + 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/LatencySpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/LatencySpec.scala new file mode 100644 index 0000000000..3cc9fca890 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/LatencySpec.scala @@ -0,0 +1,270 @@ +/** + * 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 java.util.concurrent.CyclicBarrier +import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.atomic.AtomicLongArray +import org.HdrHistogram.Histogram +import akka.stream.ThrottleMode +import java.io.StringWriter +import java.io.PrintStream +import java.io.OutputStreamWriter +import java.io.BufferedOutputStream +import java.io.ByteArrayOutputStream + +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 (10) and repeatCount (3) + akka.test.LatencySpec.totalMessagesFactor = 1.0 + akka.test.LatencySpec.repeatCount = 1 + akka { + loglevel = ERROR + testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s + actor { + provider = "akka.remote.RemoteActorRefProvider" + serialize-creators = false + serialize-messages = false + } + remote.artery { + enabled = on + } + } + """))) + + def aeronPort(roleName: RoleName): Int = + roleName match { + case `first` ⇒ 20501 // TODO yeah, we should have support for dynamic port assignment + case `second` ⇒ 20502 + } + + nodeConfig(first) { + ConfigFactory.parseString(s""" + akka.remote.artery.port = ${aeronPort(first)} + """) + } + + nodeConfig(second) { + ConfigFactory.parseString(s""" + akka.remote.artery.port = ${aeronPort(second)} + """) + } + + 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): Props = + Props(new Receiver(reporter, settings, totalMessages, sendTimes, histogram)) + + class Receiver(reporter: RateReporter, settings: TestSettings, totalMessages: Int, + sendTimes: AtomicLongArray, histogram: Histogram) extends Actor { + import settings._ + + var count = 0 + + def receive = { + case bytes: Array[Byte] ⇒ + if (bytes.length != payloadSize) throw new IllegalArgumentException("Invalid message") + reporter.onMessage(1, payloadSize) + count += 1 + val d = System.nanoTime() - sendTimes.get(count - 1) + histogram.recordValue(d) + if (count == totalMessages) { + printTotal(testName, bytes.length, histogram) + context.stop(self) + } + } + + def printTotal(testName: String, payloadSize: Long, histogram: Histogram): 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) + + 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, ") + println("Histogram of RTT latencies in microseconds.") + histogram.outputPercentileDistribution(System.out, 1000.0) + } + } + + final case class TestSettings( + testName: String, + messageRate: Int, // msg/s + payloadSize: Int, + repeat: Int) + +} + +class LatencySpecMultiJvmNode1 extends LatencySpec +class LatencySpecMultiJvmNode2 extends LatencySpec + +abstract class LatencySpec + extends MultiNodeSpec(LatencySpec) + with STMultiNodeSpec with ImplicitSender { + + import LatencySpec._ + + val totalMessagesFactor = system.settings.config.getDouble("akka.test.LatencySpec.totalMessagesFactor") + val repeatCount = system.settings.config.getInt("akka.test.LatencySpec.repeatCount") + + val aeron = { + val ctx = new Aeron.Context + val driver = MediaDriver.launchEmbedded() + ctx.aeronDirectoryName(driver.aeronDirectoryName) + Aeron.connect(ctx) + } + + 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}:${aeronPort(roleName)}" + } + + lazy val reporterExecutor = Executors.newFixedThreadPool(1) + def reporter(name: String): RateReporter = { + val r = new RateReporter(SECONDS.toNanos(1), new RateReporter.Reporter { + override def onReport(messagesPerSec: Double, bytesPerSec: Double, totalMessages: Long, totalBytes: Long): Unit = { + println(name + ": %.03g msgs/sec, %.03g bytes/sec, totals %d messages %d MB".format( + messagesPerSec, bytesPerSec, totalMessages, totalBytes / (1024 * 1024))) + } + }) + reporterExecutor.execute(r) + r + } + + override def afterAll(): Unit = { + reporterExecutor.shutdown() + 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 = "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-1000-size-1k", + messageRate = 1000, + payloadSize = 1000, + repeat = repeatCount)) + + 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() + + for (n ← 1 to repeat) { + echo ! Reset + expectMsg(Reset) + histogram.reset() + val receiver = system.actorOf(receiverProps(rep, testSettings, totalMessages, sendTimes, histogram)) + + Source(1 to totalMessages) + .throttle(messageRate, 1.second, math.max(messageRate / 10, 1), ThrottleMode.Shaping) + .runForeach { n ⇒ + sendTimes.set(n - 1, System.nanoTime()) + echo.tell(payload, receiver) + } + + watch(receiver) + expectTerminated(receiver, ((totalMessages / messageRate) + 10).seconds) + } + + 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..e56a6f1ec2 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/MaxThroughputSpec.scala @@ -0,0 +1,277 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.concurrent.Executors +import java.util.concurrent.TimeUnit.NANOSECONDS +import scala.concurrent.duration._ +import akka.actor._ +import akka.remote.RemoteActorRefProvider +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 +import java.net.InetAddress + +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 { + loglevel = ERROR + testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s + actor { + provider = "akka.remote.RemoteActorRefProvider" + serialize-creators = false + serialize-messages = false + } + remote.artery { + enabled = on + } + } + """))) + + def aeronPort(roleName: RoleName): Int = + roleName match { + case `first` ⇒ 20501 // TODO yeah, we should have support for dynamic port assignment + case `second` ⇒ 20502 + } + + nodeConfig(first) { + ConfigFactory.parseString(s""" + akka.remote.artery.port = ${aeronPort(first)} + """) + } + + nodeConfig(second) { + ConfigFactory.parseString(s""" + akka.remote.artery.port = ${aeronPort(second)} + """) + } + + case object Run + sealed trait Echo extends DeadLetterSuppression + final case object Start extends Echo + final case object End extends Echo + final case class EndResult(totalReceived: Long) + final case class FlowControl(burstStartTime: Long) extends Echo + + def receiverProps(reporter: RateReporter, payloadSize: Int): Props = + Props(new Receiver(reporter, payloadSize)) + + class Receiver(reporter: RateReporter, payloadSize: Int) extends Actor { + var c = 0L + + def receive = { + case Start ⇒ + c = 0 + sender() ! Start + case End ⇒ + sender() ! EndResult(c) + context.stop(self) + case m: Echo ⇒ + sender() ! m + case msg: Array[Byte] ⇒ + if (msg.length != payloadSize) throw new IllegalArgumentException("Invalid message") + reporter.onMessage(1, payloadSize) + c += 1 + } + } + + def senderProps(target: ActorRef, testSettings: TestSettings): Props = + Props(new Sender(target, testSettings)) + + class Sender(target: ActorRef, testSettings: TestSettings) extends Actor { + import testSettings._ + val payload = ("0" * testSettings.payloadSize).getBytes("utf-8") + var startTime = 0L + var remaining = totalMessages + var maxRoundTripMillis = 0L + + def receive = { + case Run ⇒ + // first some warmup + sendBatch() + // then Start, which will echo back here + target ! Start + + 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 throughtput = (totalReceived * 1000.0 / took).toInt + println( + s"=== MaxThroughput ${self.path.name}: " + + s"throughtput $throughtput msg/s, " + + s"dropped ${totalMessages - totalReceived}, " + + s"max round-trip $maxRoundTripMillis ms, " + + s"burst size $burstSize, " + + s"payload size $payloadSize, " + + s"$took ms to deliver $totalReceived messages") + context.stop(self) + } + + def sendBatch(): Unit = { + val batchSize = math.min(remaining, burstSize) + var i = 0 + while (i < batchSize) { + target ! payload + 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) + +} + +class MaxThroughputSpecMultiJvmNode1 extends MaxThroughputSpec +class MaxThroughputSpecMultiJvmNode2 extends MaxThroughputSpec + +abstract class MaxThroughputSpec + extends MultiNodeSpec(MaxThroughputSpec) + with STMultiNodeSpec with ImplicitSender { + + import MaxThroughputSpec._ + + val totalMessagesFactor = system.settings.config.getDouble("akka.test.MaxThroughputSpec.totalMessagesFactor") + + 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): RateReporter = { + val r = new RateReporter(SECONDS.toNanos(1), new RateReporter.Reporter { + override def onReport(messagesPerSec: Double, bytesPerSec: Double, totalMessages: Long, totalBytes: Long): Unit = { + println(name + ": %.03g msgs/sec, %.03g bytes/sec, totals %d messages %d MB".format( + messagesPerSec, bytesPerSec, totalMessages, totalBytes / (1024 * 1024))) + } + }) + reporterExecutor.execute(r) + r + } + + override def afterAll(): Unit = { + reporterExecutor.shutdown() + 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 = "1-to-1", + totalMessages = adjustedTotalMessages(20000), + burstSize = 1000, + payloadSize = 100, + senderReceiverPairs = 1), + TestSettings( + testName = "1-to-1-size-1k", + totalMessages = adjustedTotalMessages(20000), + burstSize = 1000, + payloadSize = 1000, + senderReceiverPairs = 1), + TestSettings( + testName = "1-to-1-size-10k", + totalMessages = adjustedTotalMessages(10000), + burstSize = 1000, + payloadSize = 10000, + senderReceiverPairs = 1), + TestSettings( + testName = "5-to-5", + totalMessages = adjustedTotalMessages(20000), + burstSize = 1000, + payloadSize = 100, + senderReceiverPairs = 5)) + + def test(testSettings: TestSettings): Unit = { + import testSettings._ + val receiverName = testName + "-rcv" + + runOn(second) { + val rep = reporter(testName) + for (n ← 1 to senderReceiverPairs) { + val receiver = system.actorOf(receiverProps(rep, payloadSize), receiverName + n) + } + enterBarrier(receiverName + "-started") + enterBarrier(testName + "-done") + rep.halt() + } + + runOn(first) { + enterBarrier(receiverName + "-started") + val senders = for (n ← 1 to senderReceiverPairs) yield { + val receiver = identifyReceiver(receiverName + n) + val snd = system.actorOf(senderProps(receiver, testSettings), testName + "-snd" + n) + val p = TestProbe() + p.watch(snd) + snd ! Run + (snd, p) + } + senders.foreach { + case (snd, p) ⇒ + val t = if (snd == senders.head._1) barrierTimeout else 10.seconds + p.expectTerminated(snd, t) + } + 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) + } + + // TODO add more tests, such as 5-to-5 sender receiver pairs + + } +} diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index e2aa5d8538..94e3de17fe 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -75,7 +75,13 @@ akka { artery { enabled = off port = 20200 - hostname = localhost + + # The hostname or ip clients should connect to. + # InetAddress.getLocalHost.getHostAddress is used if empty or + # "" is specified. + # InetAddress.getLocalHost.getHostName is used if + # "" is specified. + hostname = "" } ### General settings diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index 97806381ff..6493489ba1 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -15,6 +15,7 @@ import akka.event.Logging import akka.event.Logging.LogLevel import akka.ConfigurationException import java.util.Locale +import java.net.InetAddress final class RemoteSettings(val config: Config) { import config._ @@ -22,7 +23,11 @@ final class RemoteSettings(val config: Config) { val EnableArtery: Boolean = getBoolean("akka.remote.artery.enabled") val ArteryPort: Int = getInt("akka.remote.artery.port") - val ArteryHostname: String = getString("akka.remote.artery.hostname") + val ArteryHostname: String = getString("akka.remote.artery.hostname") match { + case "" | "" ⇒ InetAddress.getLocalHost.getHostAddress + case "" ⇒ InetAddress.getLocalHost.getHostName + case other ⇒ other + } val LogReceive: Boolean = getBoolean("akka.remote.log-received-messages") diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala index 06296c6e78..d4261ea99d 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala @@ -73,8 +73,6 @@ class AeronSink(channel: String, aeron: Aeron) extends GraphStage[SinkShape[Aero // retrying/polling to a separate thread that performs the polling for // all sources/sinks and notifies back when there is some news. backoffCount -= 1 - if (backoffCount <= 10) - println(s"# snd backoff $backoffCount") // FIXME if (backoffCount <= 10) scheduleOnce(Backoff, 50.millis) else diff --git a/akka-bench-jmh/src/main/java/akka/aeron/RateReporter.java b/akka-remote/src/test/java/akka/remote/artery/RateReporter.java similarity index 99% rename from akka-bench-jmh/src/main/java/akka/aeron/RateReporter.java rename to akka-remote/src/test/java/akka/remote/artery/RateReporter.java index e042e58bab..0e455fc0ab 100644 --- a/akka-bench-jmh/src/main/java/akka/aeron/RateReporter.java +++ b/akka-remote/src/test/java/akka/remote/artery/RateReporter.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package akka.aeron; +package akka.remote.artery; import java.util.concurrent.locks.LockSupport; diff --git a/akka-bench-jmh/src/main/scala/akka/aeron/AeronStreams.scala b/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala similarity index 75% rename from akka-bench-jmh/src/main/scala/akka/aeron/AeronStreams.scala rename to akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala index 9857db6e69..ce25f329a0 100644 --- a/akka-bench-jmh/src/main/scala/akka/aeron/AeronStreams.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala @@ -1,4 +1,4 @@ -package akka.aeron +package akka.remote.artery import scala.concurrent.duration._ import akka.actor.ActorSystem @@ -17,10 +17,13 @@ import java.util.concurrent.CountDownLatch import java.util.concurrent.CyclicBarrier import java.util.concurrent.atomic.AtomicLongArray import akka.stream.ThrottleMode -import akka.remote.artery.AeronSink -import akka.remote.artery.AeronSource +import org.agrona.ErrorHandler +import io.aeron.AvailableImageHandler +import io.aeron.UnavailableImageHandler +import io.aeron.Image +import io.aeron.AvailableImageHandler -object AeronStreams { +object AeronStreamsApp { val channel1 = "aeron:udp?endpoint=localhost:40123" val channel2 = "aeron:udp?endpoint=localhost:40124" @@ -32,7 +35,27 @@ object AeronStreams { lazy val aeron = { val ctx = new Aeron.Context - val driver = MediaDriver.launchEmbedded() + ctx.errorHandler(new ErrorHandler { + override def onError(cause: Throwable) { + println(s"# Aeron onError " + cause) // FIXME + } + }) + ctx.availableImageHandler(new AvailableImageHandler { + override def onAvailableImage(img: Image): Unit = { + println(s"onAvailableImage from ${img.sourceIdentity} session ${img.sessionId}") + } + }) + ctx.unavailableImageHandler(new UnavailableImageHandler { + override def onUnavailableImage(img: Image): Unit = { + println(s"onUnavailableImage from ${img.sourceIdentity} session ${img.sessionId}") + } + }) + + val driverContext = new MediaDriver.Context + driverContext.clientLivenessTimeoutNs(SECONDS.toNanos(10)) + driverContext.imageLivenessTimeoutNs(SECONDS.toNanos(10)) + driverContext.driverTimeoutMs(SECONDS.toNanos(10)) + val driver = MediaDriver.launchEmbedded(driverContext) ctx.aeronDirectoryName(driver.aeronDirectoryName) Aeron.connect(ctx) } @@ -111,12 +134,12 @@ object AeronStreams { var t0 = System.nanoTime() var count = 0L var payloadSize = 0L - Source.fromGraph(new AeronSource(channel1, () => aeron)) - .map { bytes => + Source.fromGraph(new AeronSource(channel1, aeron)) + .map { bytes ⇒ r.onMessage(1, bytes.length) bytes } - .runForeach { bytes => + .runForeach { bytes ⇒ count += 1 if (count == 1) { t0 = System.nanoTime() @@ -126,7 +149,7 @@ object AeronStreams { printTotal(throughputN, "receive", t0, payloadSize) } }.onFailure { - case e => + case e ⇒ e.printStackTrace exit(-1) } @@ -138,30 +161,30 @@ object AeronStreams { val r = reporter val t0 = System.nanoTime() Source(1 to throughputN) - .map { n => + .map { n ⇒ if (n == throughputN) { exit(0) printTotal(throughputN, "send", t0, payload.length) } n } - .map { _ => + .map { _ ⇒ r.onMessage(1, payload.length) payload } - .runWith(new AeronSink(channel1, () => aeron)) + .runWith(new AeronSink(channel1, aeron)) } def runEchoReceiver(): Unit = { // just echo back on channel2 reporterExecutor.execute(reporter) val r = reporter - Source.fromGraph(new AeronSource(channel1, () => aeron)) - .map { bytes => + Source.fromGraph(new AeronSource(channel1, aeron)) + .map { bytes ⇒ r.onMessage(1, bytes.length) bytes } - .runWith(new AeronSink(channel2, () => aeron)) + .runWith(new AeronSink(channel2, aeron)) } def runEchoSender(): Unit = { @@ -173,12 +196,12 @@ object AeronStreams { var repeat = 3 val count = new AtomicInteger var t0 = System.nanoTime() - Source.fromGraph(new AeronSource(channel2, () => aeron)) - .map { bytes => + Source.fromGraph(new AeronSource(channel2, aeron)) + .map { bytes ⇒ r.onMessage(1, bytes.length) bytes } - .runForeach { bytes => + .runForeach { bytes ⇒ val c = count.incrementAndGet() val d = System.nanoTime() - sendTimes.get(c - 1) if (c % (latencyN / 10) == 0) @@ -189,7 +212,7 @@ object AeronStreams { barrier.await() // this is always the last party } }.onFailure { - case e => + case e ⇒ e.printStackTrace exit(-1) } @@ -202,13 +225,13 @@ object AeronStreams { Source(1 to latencyN) .throttle(latencyRate, 1.second, latencyRate / 10, ThrottleMode.Shaping) - .map { n => + .map { n ⇒ if (n % (latencyN / 10) == 0) println(s"# send offset $n") // FIXME sendTimes.set(n - 1, System.nanoTime()) payload } - .runWith(new AeronSink(channel1, () => aeron)) + .runWith(new AeronSink(channel1, aeron)) barrier.await() } @@ -218,12 +241,12 @@ object AeronStreams { def runDebugReceiver(): Unit = { import system.dispatcher - Source.fromGraph(new AeronSource(channel1, () => aeron)) - .map(bytes => new String(bytes, "utf-8")) - .runForeach { s => + Source.fromGraph(new AeronSource(channel1, aeron)) + .map(bytes ⇒ new String(bytes, "utf-8")) + .runForeach { s ⇒ println(s) }.onFailure { - case e => + case e ⇒ e.printStackTrace exit(-1) } @@ -233,12 +256,12 @@ object AeronStreams { val fill = "0000" Source(1 to 1000) .throttle(1, 1.second, 1, ThrottleMode.Shaping) - .map { n => + .map { n ⇒ val s = (fill + n.toString).takeRight(4) println(s) s.getBytes("utf-8") } - .runWith(new AeronSink(channel1, () => aeron)) + .runWith(new AeronSink(channel1, aeron)) } } diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala index 34013720c6..b6c320206c 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala @@ -1,5 +1,6 @@ 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 @@ -12,6 +13,7 @@ object RemoteSendConsistencySpec { akka { actor.provider = "akka.remote.RemoteActorRefProvider" remote.artery.enabled = on + remote.artery.hostname = localhost } """ @@ -63,8 +65,8 @@ class RemoteSendConsistencySpec extends AkkaSpec(commonConfig) with ImplicitSend } val senderProps = Props(new Actor { - var counter = 1000 - remoteRef ! 1000 + var counter = 100 // FIXME try this test with 1000, why does it take so long? + remoteRef ! counter override def receive: Receive = { case i: Int ⇒ @@ -84,10 +86,12 @@ class RemoteSendConsistencySpec extends AkkaSpec(commonConfig) with ImplicitSend system.actorOf(senderProps) system.actorOf(senderProps) - expectMsg("success") - expectMsg("success") - expectMsg("success") - expectMsg("success") + within(10.seconds) { + expectMsg("success") + expectMsg("success") + expectMsg("success") + expectMsg("success") + } } } diff --git a/akka-testkit/src/test/scala/akka/testkit/metrics/HdrHistogram.scala b/akka-testkit/src/test/scala/akka/testkit/metrics/HdrHistogram.scala index 8c7e983d97..943f020050 100644 --- a/akka-testkit/src/test/scala/akka/testkit/metrics/HdrHistogram.scala +++ b/akka-testkit/src/test/scala/akka/testkit/metrics/HdrHistogram.scala @@ -41,8 +41,8 @@ private[akka] class HdrHistogram( private def wrapHistogramOutOfBoundsException(value: Long, ex: ArrayIndexOutOfBoundsException): IllegalArgumentException = new IllegalArgumentException(s"Given value $value can not be stored in this histogram " + - s"(min: ${hist.getLowestTrackableValue}, max: ${hist.getHighestTrackableValue}})", ex) + s"(min: ${hist.getLowestDiscernibleValue}, max: ${hist.getHighestTrackableValue}})", ex) - def getData = hist.copy().getHistogramData + def getData = hist.copy() } diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 1530ee7d50..2973c04915 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -132,7 +132,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", multiNodeTestkit) ) configs (MultiJvm) lazy val cluster = Project( diff --git a/project/Dependencies.scala b/project/Dependencies.scala index b55dc7d954..7e09655211 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -69,7 +69,6 @@ object Dependencies { val aeronDriver = "io.aeron" % "aeron-driver" % "0.9.5" // ApacheV2 val aeronClient = "io.aeron" % "aeron-client" % "0.9.5" // ApacheV2 - val hdrHistogram = "org.hdrhistogram" % "HdrHistogram" % "2.1.8" // CC0 object Docs { val sprayJson = "io.spray" %% "spray-json" % "1.3.2" % "test" @@ -96,7 +95,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" % "1.1.4" % "test" // CC0 + val hdrHistogram = "org.hdrhistogram" % "HdrHistogram" % "2.1.8" % "test" // CC0 val metricsAll = Seq(metrics, metricsJvm, latencyUtils, hdrHistogram) // sigar logging @@ -165,7 +164,7 @@ object Dependencies { val contrib = l ++= Seq(Test.junitIntf, Test.commonsIo) - val benchJmh = l ++= Seq(Provided.levelDB, Provided.levelDBNative, hdrHistogram) + val benchJmh = l ++= Seq(Provided.levelDB, Provided.levelDBNative) // akka stream & http From c01b3e8fedf9efea25abff9e92c0b90382d954e3 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 22 Apr 2016 16:24:12 +0200 Subject: [PATCH 011/186] add output for trend plots * use Jenkins Plot plugin * output csv values to log and then use post build step to turn those into csv files that the plot plugin understands --- .../artery/AeronStreamLatencySpec.scala | 24 ++++++++++-- .../artery/AeronStreamMaxThroughputSpec.scala | 10 ++++- .../akka/remote/artery/LatencySpec.scala | 30 +++++++++++++-- .../remote/artery/MaxThroughputSpec.scala | 38 +++++++++++++------ .../scala/akka/remote/artery/PlotResult.scala | 23 +++++++++++ 5 files changed, 104 insertions(+), 21 deletions(-) create mode 100644 akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/PlotResult.scala 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 index c6cd4a7fe1..33556e5ec2 100644 --- 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 @@ -7,9 +7,7 @@ import java.util.concurrent.CyclicBarrier import java.util.concurrent.Executors import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicLongArray - import scala.concurrent.duration._ - import akka.actor._ import akka.remote.testconductor.RoleName import akka.remote.testkit.MultiNodeConfig @@ -23,6 +21,7 @@ import com.typesafe.config.ConfigFactory import io.aeron.Aeron import io.aeron.driver.MediaDriver import org.HdrHistogram.Histogram +import java.util.concurrent.atomic.AtomicBoolean object AeronStreamLatencySpec extends MultiNodeConfig { val first = role("first") @@ -73,6 +72,8 @@ abstract class 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 aeron = { val ctx = new Aeron.Context val driver = MediaDriver.launchEmbedded() @@ -104,10 +105,15 @@ abstract class AeronStreamLatencySpec 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 printTotal(testName: String, payloadSize: Long, histogram: Histogram): Unit = { + def printTotal(testName: String, payloadSize: Long, histogram: Histogram, lastRepeat: Boolean): Unit = { import scala.collection.JavaConverters._ val percentiles = histogram.percentiles(5) def percentile(p: Double): Double = @@ -122,6 +128,14 @@ abstract class AeronStreamLatencySpec f"99%%ile: ${percentile(99.0)}%.0f µ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))) + } } val scenarios = List( @@ -159,6 +173,7 @@ abstract class AeronStreamLatencySpec val rep = reporter(testName) val barrier = new CyclicBarrier(2) val count = new AtomicInteger + val lastRepeat = new AtomicBoolean(false) Source.fromGraph(new AeronSource(channel(first), aeron)) .runForeach { bytes ⇒ if (bytes.length != payloadSize) throw new IllegalArgumentException("Invalid message") @@ -167,7 +182,7 @@ abstract class AeronStreamLatencySpec val d = System.nanoTime() - sendTimes.get(c - 1) histogram.recordValue(d) if (c == totalMessages) { - printTotal(testName, bytes.length, histogram) + printTotal(testName, bytes.length, histogram, lastRepeat.get) barrier.await() // this is always the last party } } @@ -175,6 +190,7 @@ abstract class AeronStreamLatencySpec for (n ← 1 to repeat) { histogram.reset() count.set(0) + lastRepeat.set(n == repeat) Source(1 to totalMessages) .throttle(messageRate, 1.second, math.max(messageRate / 10, 1), ThrottleMode.Shaping) 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 index 22b42bbbd0..1115a2a552 100644 --- 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 @@ -68,6 +68,7 @@ object AeronStreamMaxThroughputSpec extends MultiNodeConfig { acc } } + } class AeronStreamMaxThroughputSpecMultiJvmNode1 extends AeronStreamMaxThroughputSpec @@ -81,6 +82,8 @@ abstract class AeronStreamMaxThroughputSpec val totalMessagesFactor = system.settings.config.getDouble("akka.test.AeronStreamMaxThroughputSpec.totalMessagesFactor") + var plot = PlotResult() + val aeron = { val ctx = new Aeron.Context val driver = MediaDriver.launchEmbedded() @@ -114,15 +117,20 @@ abstract class AeronStreamMaxThroughputSpec override def afterAll(): Unit = { reporterExecutor.shutdown() + 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"${1000.0 * total / d}%.03g msg/s, ${1000.0 * total * payloadSize / d}%.03g bytes/s, " + + f"${throughput}%.03g msg/s, ${throughput * payloadSize}%.03g bytes/s, " + s"payload size $payloadSize, " + s"$d ms to deliver $total messages") + plot = plot.add(testName, throughput * payloadSize / 1024 / 1024) } val scenarios = List( 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 index 3cc9fca890..322bc12735 100644 --- 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 @@ -93,11 +93,11 @@ object LatencySpec extends MultiNodeConfig { } def receiverProps(reporter: RateReporter, settings: TestSettings, totalMessages: Int, - sendTimes: AtomicLongArray, histogram: Histogram): Props = - Props(new Receiver(reporter, settings, totalMessages, sendTimes, histogram)) + 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) extends Actor { + sendTimes: AtomicLongArray, histogram: Histogram, plotsRef: ActorRef) extends Actor { import settings._ var count = 0 @@ -130,6 +130,12 @@ object LatencySpec extends MultiNodeConfig { f"99%%ile: ${percentile(99.0)}%.0f µs, ") println("Histogram of RTT latencies in microseconds.") histogram.outputPercentileDistribution(System.out, 1000.0) + + val plots = LatencyPlots( + PlotResult().add(testName, percentile(50.0)), + PlotResult().add(testName, percentile(90.0)), + PlotResult().add(testName, percentile(99.0))) + plotsRef ! plots } } @@ -153,6 +159,8 @@ abstract class LatencySpec val totalMessagesFactor = system.settings.config.getDouble("akka.test.LatencySpec.totalMessagesFactor") val repeatCount = system.settings.config.getInt("akka.test.LatencySpec.repeatCount") + var plots = LatencyPlots() + val aeron = { val ctx = new Aeron.Context val driver = MediaDriver.launchEmbedded() @@ -184,6 +192,11 @@ abstract class LatencySpec 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() } @@ -226,12 +239,13 @@ abstract class LatencySpec val rep = reporter(testName) val echo = identifyEcho() + val plotProbe = TestProbe() for (n ← 1 to repeat) { echo ! Reset expectMsg(Reset) histogram.reset() - val receiver = system.actorOf(receiverProps(rep, testSettings, totalMessages, sendTimes, histogram)) + val receiver = system.actorOf(receiverProps(rep, testSettings, totalMessages, sendTimes, histogram, plotProbe.ref)) Source(1 to totalMessages) .throttle(messageRate, 1.second, math.max(messageRate / 10, 1), ThrottleMode.Shaping) @@ -242,6 +256,14 @@ abstract class LatencySpec watch(receiver) expectTerminated(receiver, ((totalMessages / messageRate) + 10).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() 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 index e56a6f1ec2..2aedd70c94 100644 --- 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 @@ -87,10 +87,10 @@ object MaxThroughputSpec extends MultiNodeConfig { } } - def senderProps(target: ActorRef, testSettings: TestSettings): Props = - Props(new Sender(target, testSettings)) + def senderProps(target: ActorRef, testSettings: TestSettings, plotRef: ActorRef): Props = + Props(new Sender(target, testSettings, plotRef)) - class Sender(target: ActorRef, testSettings: TestSettings) extends Actor { + class Sender(target: ActorRef, testSettings: TestSettings, plotRef: ActorRef) extends Actor { import testSettings._ val payload = ("0" * testSettings.payloadSize).getBytes("utf-8") var startTime = 0L @@ -126,15 +126,17 @@ object MaxThroughputSpec extends MultiNodeConfig { case EndResult(totalReceived) ⇒ val took = NANOSECONDS.toMillis(System.nanoTime - startTime) - val throughtput = (totalReceived * 1000.0 / took).toInt + val throughput = (totalReceived * 1000.0 / took) println( s"=== MaxThroughput ${self.path.name}: " + - s"throughtput $throughtput msg/s, " + + f"throughput ${throughput}%.03g msg/s, " + + f"${throughput * payloadSize}%.03g bytes/s, " + s"dropped ${totalMessages - totalReceived}, " + s"max round-trip $maxRoundTripMillis ms, " + s"burst size $burstSize, " + s"payload size $payloadSize, " + s"$took ms to deliver $totalReceived messages") + plotRef ! PlotResult().add(testName, throughput * payloadSize / 1024 / 1024) context.stop(self) } @@ -176,6 +178,8 @@ abstract class MaxThroughputSpec val totalMessagesFactor = system.settings.config.getDouble("akka.test.MaxThroughputSpec.totalMessagesFactor") + var plot = PlotResult() + def adjustedTotalMessages(n: Long): Long = (n * totalMessagesFactor).toLong override def initialParticipants = roles.size @@ -196,6 +200,9 @@ abstract class MaxThroughputSpec override def afterAll(): Unit = { reporterExecutor.shutdown() + runOn(first) { + println(plot.csv(system.name)) + } super.afterAll() } @@ -246,18 +253,25 @@ abstract class MaxThroughputSpec runOn(first) { enterBarrier(receiverName + "-started") + val ignore = TestProbe() val senders = for (n ← 1 to senderReceiverPairs) yield { val receiver = identifyReceiver(receiverName + n) - val snd = system.actorOf(senderProps(receiver, testSettings), testName + "-snd" + n) - val p = TestProbe() - p.watch(snd) + val plotProbe = TestProbe() + val snd = system.actorOf(senderProps(receiver, testSettings, plotProbe.ref), + testName + "-snd" + n) + val terminationProbe = TestProbe() + terminationProbe.watch(snd) snd ! Run - (snd, p) + (snd, terminationProbe, plotProbe) } senders.foreach { - case (snd, p) ⇒ - val t = if (snd == senders.head._1) barrierTimeout else 10.seconds - p.expectTerminated(snd, t) + 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") } 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..08858e62f1 --- /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()) From 2c3ed73e525150e45ac5614c77e022ae3d4f76e7 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 3 May 2016 16:44:33 +0200 Subject: [PATCH 012/186] delegate backoff to separate/shared thread (#20407) * delegate backoff to separate/shared thread * first level of spinning backoff in the source/sink, then hand over to the shared TaskRunner thread that will spin/park backoff for all sources/sinks * improve the task queue --- .../artery/AeronStreamConcistencySpec.scala | 143 ++++++++++++++++ .../artery/AeronStreamLatencySpec.scala | 58 +++++-- .../artery/AeronStreamMaxThroughputSpec.scala | 19 ++- .../scala/akka/remote/artery/AeronSink.scala | 79 +++++---- .../akka/remote/artery/AeronSource.scala | 114 +++++++------ .../scala/akka/remote/artery/TaskRunner.scala | 160 ++++++++++++++++++ .../scala/akka/remote/artery/Transport.scala | 8 +- .../akka/remote/artery/AeronStreamsApp.scala | 23 ++- project/AkkaBuild.scala | 4 +- 9 files changed, 496 insertions(+), 112 deletions(-) create mode 100644 akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/AeronStreamConcistencySpec.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala 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..8e599f6376 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/AeronStreamConcistencySpec.scala @@ -0,0 +1,143 @@ +/** + * 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 + +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 = "akka.remote.RemoteActorRefProvider" + } + remote.artery.enabled = off + } + """))) + + def aeronPort(roleName: RoleName): Int = + roleName match { + case `first` ⇒ 20521 // TODO yeah, we should have support for dynamic port assignment + case `second` ⇒ 20522 + } + +} + +class AeronStreamConsistencySpecMultiJvmNode1 extends AeronStreamConsistencySpec +class AeronStreamConsistencySpecMultiJvmNode2 extends AeronStreamConsistencySpec + +abstract class AeronStreamConsistencySpec + extends MultiNodeSpec(AeronStreamConsistencySpec) + with STMultiNodeSpec with ImplicitSender { + + import AeronStreamConsistencySpec._ + + val aeron = { + val ctx = new Aeron.Context + val driver = MediaDriver.launchEmbedded() + ctx.aeronDirectoryName(driver.aeronDirectoryName) + Aeron.connect(ctx) + } + + val taskRunner = { + val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem]) + 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}:${aeronPort(roleName)}" + } + + override def afterAll(): Unit = { + taskRunner.stop() + aeron.close() + super.afterAll() + } + + "Message consistency of Aeron Streams" must { + + "start echo" in { + runOn(second) { + // just echo back + Source.fromGraph(new AeronSource(channel(second), aeron, taskRunner)) + .runWith(new AeronSink(channel(first), aeron, taskRunner)) + } + 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), aeron, taskRunner)) + .via(killSwitch.flow) + .runForeach { bytes ⇒ + if (bytes.length == 1 && bytes(0) == startMsg(0)) + started.ref ! Done + else { + val c = count.incrementAndGet() + val x = new String(bytes, "utf-8").toInt + if (x != c) { + throw new IllegalArgumentException(s"# wrong message $x expected $c") + } + if (c == totalMessages) + done.countDown() + } + }.onFailure { + case e ⇒ e.printStackTrace + } + + within(10.seconds) { + Source(1 to 100).map(_ ⇒ startMsg) + .throttle(1, 200.milliseconds, 1, ThrottleMode.Shaping) + .runWith(new AeronSink(channel(second), aeron, taskRunner)) + started.expectMsg(Done) + } + + Source(1 to totalMessages) + .throttle(10000, 1.second, 1000, ThrottleMode.Shaping) + .map { n ⇒ n.toString.getBytes("utf-8") } + .runWith(new AeronSink(channel(second), aeron, taskRunner)) + + 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 index 33556e5ec2..628981e0fa 100644 --- 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 @@ -22,6 +22,10 @@ import io.aeron.Aeron import io.aeron.driver.MediaDriver import org.HdrHistogram.Histogram import java.util.concurrent.atomic.AtomicBoolean +import akka.stream.KillSwitches +import akka.Done +import org.agrona.IoUtil +import java.io.File object AeronStreamLatencySpec extends MultiNodeConfig { val first = role("first") @@ -74,13 +78,20 @@ abstract class AeronStreamLatencySpec var plots = LatencyPlots() + val driver = MediaDriver.launchEmbedded() + val aeron = { val ctx = new Aeron.Context - val driver = MediaDriver.launchEmbedded() ctx.aeronDirectoryName(driver.aeronDirectoryName) Aeron.connect(ctx) } + val taskRunner = { + val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem]) + r.start() + r + } + lazy implicit val mat = ActorMaterializer()(system) import system.dispatcher @@ -105,6 +116,9 @@ abstract class AeronStreamLatencySpec override def afterAll(): Unit = { reporterExecutor.shutdown() + taskRunner.stop() + aeron.close() + IoUtil.delete(new File(driver.aeronDirectoryName), true) runOn(first) { println(plots.plot50.csv(system.name + "50")) println(plots.plot90.csv(system.name + "90")) @@ -164,7 +178,7 @@ abstract class AeronStreamLatencySpec import testSettings._ runOn(first) { - val payload = ("0" * payloadSize).getBytes("utf-8") + 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) @@ -174,19 +188,34 @@ abstract class AeronStreamLatencySpec val barrier = new CyclicBarrier(2) val count = new AtomicInteger val lastRepeat = new AtomicBoolean(false) - Source.fromGraph(new AeronSource(channel(first), aeron)) + val killSwitch = KillSwitches.shared(testName) + val started = TestProbe() + val startMsg = "0".getBytes("utf-8") + Source.fromGraph(new AeronSource(channel(first), aeron, taskRunner)) + .via(killSwitch.flow) .runForeach { bytes ⇒ - 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) { - printTotal(testName, bytes.length, histogram, lastRepeat.get) - barrier.await() // this is always the last party + 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) { + printTotal(testName, bytes.length, histogram, lastRepeat.get) + barrier.await() // this is always the last party + } } } + within(10.seconds) { + Source(1 to 50).map(_ ⇒ startMsg) + .throttle(1, 200.milliseconds, 1, ThrottleMode.Shaping) + .runWith(new AeronSink(channel(second), aeron, taskRunner)) + started.expectMsg(Done) + } + for (n ← 1 to repeat) { histogram.reset() count.set(0) @@ -198,11 +227,12 @@ abstract class AeronStreamLatencySpec sendTimes.set(n - 1, System.nanoTime()) payload } - .runWith(new AeronSink(channel(second), aeron)) + .runWith(new AeronSink(channel(second), aeron, taskRunner)) barrier.await((totalMessages / messageRate) + 10, SECONDS) } + killSwitch.shutdown() rep.halt() } @@ -214,8 +244,8 @@ abstract class AeronStreamLatencySpec "start echo" in { runOn(second) { // just echo back - Source.fromGraph(new AeronSource(channel(second), aeron)) - .runWith(new AeronSink(channel(first), aeron)) + Source.fromGraph(new AeronSource(channel(second), aeron, taskRunner)) + .runWith(new AeronSink(channel(first), aeron, taskRunner)) } enterBarrier("echo-started") } 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 index 1115a2a552..e59fd296fc 100644 --- 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 @@ -5,11 +5,9 @@ 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 @@ -19,9 +17,9 @@ 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 object AeronStreamMaxThroughputSpec extends MultiNodeConfig { val first = role("first") @@ -91,6 +89,12 @@ abstract class AeronStreamMaxThroughputSpec Aeron.connect(ctx) } + val taskRunner = { + val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem]) + r.start() + r + } + lazy implicit val mat = ActorMaterializer()(system) import system.dispatcher @@ -117,6 +121,8 @@ abstract class AeronStreamMaxThroughputSpec override def afterAll(): Unit = { reporterExecutor.shutdown() + taskRunner.stop() + aeron.close() runOn(second) { println(plot.csv(system.name)) } @@ -156,7 +162,9 @@ abstract class AeronStreamMaxThroughputSpec var t0 = System.nanoTime() var count = 0L val done = TestLatch(1) - Source.fromGraph(new AeronSource(channel(second), aeron)) + val killSwitch = KillSwitches.shared(testName) + Source.fromGraph(new AeronSource(channel(second), aeron, taskRunner)) + .via(killSwitch.flow) .runForeach { bytes ⇒ rep.onMessage(1, bytes.length) count += 1 @@ -165,6 +173,7 @@ abstract class AeronStreamMaxThroughputSpec } else if (count == totalMessages) { printTotal(testName, totalMessages, t0, payloadSize) done.countDown() + killSwitch.shutdown() } }.onFailure { case e ⇒ @@ -184,7 +193,7 @@ abstract class AeronStreamMaxThroughputSpec val t0 = System.nanoTime() Source.fromIterator(() ⇒ iterate(1, totalMessages)) .map { n ⇒ payload } - .runWith(new AeronSink(channel(second), aeron)) + .runWith(new AeronSink(channel(second), aeron, taskRunner)) enterBarrier(testName + "-done") } diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala index d4261ea99d..948f8cb856 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala @@ -1,7 +1,11 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ package akka.remote.artery import java.nio.ByteBuffer import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicInteger import scala.annotation.tailrec import scala.concurrent.duration._ @@ -9,44 +13,68 @@ import scala.concurrent.duration._ import akka.stream.Attributes import akka.stream.Inlet import akka.stream.SinkShape +import akka.stream.stage.AsyncCallback import akka.stream.stage.GraphStage import akka.stream.stage.GraphStageLogic import akka.stream.stage.InHandler -import akka.stream.stage.TimerGraphStageLogic import io.aeron.Aeron -import org.agrona.concurrent.BackoffIdleStrategy +import io.aeron.Publication import org.agrona.concurrent.UnsafeBuffer object AeronSink { type Bytes = Array[Byte] - private case object Backoff + + private def offerTask(pub: Publication, buffer: UnsafeBuffer, msgSize: AtomicInteger, onOfferSuccess: AsyncCallback[Unit]): () ⇒ Boolean = { + var n = 0L + var localMsgSize = -1 + () ⇒ + { + n += 1 + if (localMsgSize == -1) + localMsgSize = msgSize.get + val result = pub.offer(buffer, 0, localMsgSize) + if (result >= 0) { + n = 0 + localMsgSize = -1 + onOfferSuccess.invoke(()) + true + } else { + // FIXME drop after too many attempts? + if (n > 1000000 && n % 100000 == 0) + println(s"# offer not accepted after $n") // FIXME + false + } + } + } } /** * @param channel eg. "aeron:udp?endpoint=localhost:40123" */ -class AeronSink(channel: String, aeron: Aeron) extends GraphStage[SinkShape[AeronSink.Bytes]] { +class AeronSink(channel: String, aeron: Aeron, taskRunner: TaskRunner) extends GraphStage[SinkShape[AeronSink.Bytes]] { import AeronSink._ + import TaskRunner._ val in: Inlet[Bytes] = Inlet("AeronSink") override val shape: SinkShape[Bytes] = SinkShape(in) override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = - new TimerGraphStageLogic(shape) with InHandler { + new GraphStageLogic(shape) with InHandler { private val buffer = new UnsafeBuffer(ByteBuffer.allocateDirect(128 * 1024)) private val streamId = 10 private val pub = aeron.addPublication(channel, streamId) - private val idleStrategy = new BackoffIdleStrategy( - 100, 10, TimeUnit.MICROSECONDS.toNanos(1), TimeUnit.MICROSECONDS.toNanos(100)) - private val retries = 130 - private var backoffCount = retries + private val spinning = 1000 + private var backoffCount = spinning private var lastMsgSize = 0 + private var lastMsgSizeRef = new AtomicInteger // used in the external backoff task + private val addOfferTask: Add = Add(offerTask(pub, buffer, lastMsgSizeRef, getAsyncCallback(_ ⇒ onOfferSuccess()))) override def preStart(): Unit = pull(in) override def postStop(): Unit = { + taskRunner.command(Remove(addOfferTask.task)) pub.close() } @@ -54,8 +82,7 @@ class AeronSink(channel: String, aeron: Aeron) extends GraphStage[SinkShape[Aero override def onPush(): Unit = { val msg = grab(in) buffer.putBytes(0, msg); - idleStrategy.reset() - backoffCount = retries + backoffCount = spinning lastMsgSize = msg.length publish() } @@ -65,33 +92,21 @@ class AeronSink(channel: String, aeron: Aeron) extends GraphStage[SinkShape[Aero // FIXME handle Publication.CLOSED // TODO the backoff strategy should be measured and tuned if (result < 0) { - if (backoffCount == 1) { - println(s"# drop") // FIXME - pull(in) // drop it - } else if (backoffCount <= 15) { - // TODO Instead of using the scheduler we should handoff the task of - // retrying/polling to a separate thread that performs the polling for - // all sources/sinks and notifies back when there is some news. - backoffCount -= 1 - if (backoffCount <= 10) - scheduleOnce(Backoff, 50.millis) - else - scheduleOnce(Backoff, 1.millis) - } else { - idleStrategy.idle() - backoffCount -= 1 + backoffCount -= 1 + if (backoffCount > 0) { publish() // recursive + } else { + // delegate backoff to shared TaskRunner + lastMsgSizeRef.set(lastMsgSize) + taskRunner.command(addOfferTask) } } else { - pull(in) + onOfferSuccess() } } - override protected def onTimer(timerKey: Any): Unit = { - timerKey match { - case Backoff ⇒ publish() - case msg ⇒ super.onTimer(msg) - } + private def onOfferSuccess(): Unit = { + pull(in) } setHandler(in, this) diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala index e1c90b4b56..5231cc8f0f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala @@ -1,8 +1,9 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ package akka.remote.artery -import java.nio.ByteBuffer import java.util.concurrent.TimeUnit -import java.util.concurrent.atomic.AtomicBoolean import scala.annotation.tailrec import scala.concurrent.duration._ @@ -10,63 +11,83 @@ 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 akka.stream.stage.TimerGraphStageLogic 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.concurrent.UnsafeBuffer object AeronSource { type Bytes = Array[Byte] - private case object Backoff + + private def pollTask(sub: Subscription, handler: MessageHandler, onMessage: AsyncCallback[Bytes]): () ⇒ 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 { + def reset(): Unit = messageReceived = null + + var messageReceived: Bytes = null + + val fragmentsHandler = new Fragments(data ⇒ messageReceived = data) + } + + class Fragments(onMessage: Bytes ⇒ Unit) extends FragmentAssembler(new FragmentHandler { + override def onFragment(buffer: DirectBuffer, offset: Int, length: Int, header: Header): Unit = { + val data = Array.ofDim[Byte](length) + buffer.getBytes(offset, data) + onMessage(data) + } + }) } /** * @param channel eg. "aeron:udp?endpoint=localhost:40123" */ -class AeronSource(channel: String, aeron: Aeron) extends GraphStage[SourceShape[AeronSource.Bytes]] { +class AeronSource(channel: String, aeron: Aeron, taskRunner: TaskRunner) extends GraphStage[SourceShape[AeronSource.Bytes]] { import AeronSource._ + import TaskRunner._ val out: Outlet[Bytes] = Outlet("AeronSource") override val shape: SourceShape[Bytes] = SourceShape(out) override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = - new TimerGraphStageLogic(shape) with OutHandler { + new GraphStageLogic(shape) with OutHandler { - private val buffer = new UnsafeBuffer(ByteBuffer.allocateDirect(128 * 1024)) private val streamId = 10 private val sub = aeron.addSubscription(channel, streamId) - private val running = new AtomicBoolean(true) - private val spinning = 20000 + private val spinning = 1000 private val yielding = 0 - private val parking = 50 + private val parking = 0 private val idleStrategy = new BackoffIdleStrategy( spinning, yielding, TimeUnit.MICROSECONDS.toNanos(1), TimeUnit.MICROSECONDS.toNanos(100)) private val idleStrategyRetries = spinning + yielding + parking private var backoffCount = idleStrategyRetries - private val backoffDuration1 = 1.millis - private val backoffDuration2 = 50.millis - private var messageReceived = false // the fragmentHandler is called from `poll` in same thread, i.e. no async callback is needed - val fragmentHandler = new FragmentAssembler(new FragmentHandler { - override def onFragment(buffer: DirectBuffer, offset: Int, length: Int, header: Header): Unit = { - messageReceived = true - val data = Array.ofDim[Byte](length) - buffer.getBytes(offset, data); - push(out, data) - } - }) + private val messageHandler = new MessageHandler + private val addPollTask: Add = Add(pollTask(sub, messageHandler, getAsyncCallback(onMessage))) override def postStop(): Unit = { - running.set(false) sub.close() + taskRunner.command(Remove(addPollTask.task)) } // OutHandler @@ -76,36 +97,31 @@ class AeronSource(channel: String, aeron: Aeron) extends GraphStage[SourceShape[ subscriberLoop() } - @tailrec private def subscriberLoop(): Unit = - if (running.get) { - messageReceived = false // will be set by the fragmentHandler if got full msg - // we only poll 1 fragment, otherwise we would have to use another buffer for - // received messages that can't be pushed - val fragmentsRead = sub.poll(fragmentHandler, 1) - if (fragmentsRead > 0 && !messageReceived) + @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) { + if (msg ne null) + onMessage(msg) + else subscriberLoop() // recursive, read more fragments - else if (fragmentsRead <= 0) { - // TODO the backoff strategy should be measured and tuned - backoffCount -= 1 - if (backoffCount > 0) { - idleStrategy.idle() - subscriberLoop() // recursive - } else if (backoffCount > -1000) { - // TODO Instead of using the scheduler we should handoff the task of - // retrying/polling to a separate thread that performs the polling for - // all sources/sinks and notifies back when there is some news. - scheduleOnce(Backoff, backoffDuration1) - } else { - scheduleOnce(Backoff, backoffDuration2) - } + } else { + // TODO the backoff strategy should be measured and tuned + backoffCount -= 1 + if (backoffCount > 0) { + idleStrategy.idle() + subscriberLoop() // recursive + } else { + // delegate backoff to shared TaskRunner + taskRunner.command(addPollTask) } } + } - override protected def onTimer(timerKey: Any): Unit = { - timerKey match { - case Backoff ⇒ subscriberLoop() - case msg ⇒ super.onTimer(msg) - } + private def onMessage(data: Bytes): Unit = { + push(out, data) } setHandler(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..307399d23c --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala @@ -0,0 +1,160 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.concurrent.TimeUnit +import scala.util.control.NonFatal +import akka.actor.ExtendedActorSystem +import akka.dispatch.AbstractNodeQueue +import akka.event.Logging +import org.agrona.concurrent.BackoffIdleStrategy +import scala.annotation.tailrec +import scala.reflect.ClassTag + +/** + * 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) + } + + /** + * 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("[", ",", "]") + } +} + +/** + * INTERNAL API + */ +private[akka] class TaskRunner(system: ExtendedActorSystem) 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] + + // TODO the backoff strategy should be measured and tuned + private val spinning = 2000000 + private val yielding = 0 + private val idleStrategy = new BackoffIdleStrategy( + spinning, yielding, TimeUnit.MICROSECONDS.toNanos(1), TimeUnit.MICROSECONDS.toNanos(100)) + private var reset = false + + def start(): Unit = { + val thread = system.threadFactory.newThread(this) + thread.start() + } + + def stop(): Unit = { + command(Shutdown) + } + + def command(cmd: Command): Unit = { + cmdQueue.add(cmd) + } + + override def run(): Unit = { + try { + running = true + while (running) { + executeTasks() + processCommand(cmdQueue.poll()) + 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 + } + } + +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/Transport.scala b/akka-remote/src/main/scala/akka/remote/artery/Transport.scala index 1b0302b126..6dd7d9a7b4 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Transport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Transport.scala @@ -88,8 +88,11 @@ private[akka] class Transport( Aeron.connect(ctx) } + private val taskRunner = new TaskRunner(system) + def start(): Unit = { - Source.fromGraph(new AeronSource(inboundChannel, aeron)) + taskRunner.start() + Source.fromGraph(new AeronSource(inboundChannel, aeron, taskRunner)) .async // FIXME use dedicated dispatcher for AeronSource .map(ByteString.apply) // TODO we should use ByteString all the way .via(inboundFlow) @@ -98,6 +101,7 @@ private[akka] class Transport( def shutdown(): Future[Done] = { // FIXME stop the AeronSource first? + taskRunner.stop() aeron.close() Future.successful(Done) } @@ -109,7 +113,7 @@ private[akka] class Transport( Flow.fromGraph(killSwitch.flow[Send]) .via(encoder) .map(_.toArray) // TODO we should use ByteString all the way - .to(new AeronSink(outboundChannel, aeron)) + .to(new AeronSink(outboundChannel, aeron, taskRunner)) } // TODO: Try out parallelized serialization (mapAsync) for performance diff --git a/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala b/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala index ce25f329a0..d2a312cb34 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala @@ -22,6 +22,7 @@ import io.aeron.AvailableImageHandler import io.aeron.UnavailableImageHandler import io.aeron.Image import io.aeron.AvailableImageHandler +import akka.actor.ExtendedActorSystem object AeronStreamsApp { @@ -63,6 +64,12 @@ object AeronStreamsApp { lazy val system = ActorSystem("AeronStreams") lazy implicit val mat = ActorMaterializer()(system) + lazy val taskRunner = { + val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem]) + r.start() + r + } + lazy val reporter = new RateReporter(SECONDS.toNanos(1), new RateReporter.Reporter { override def onReport(messagesPerSec: Double, bytesPerSec: Double, totalMessages: Long, totalBytes: Long): Unit = { println("%.03g msgs/sec, %.03g bytes/sec, totals %d messages %d MB".format( @@ -134,7 +141,7 @@ object AeronStreamsApp { var t0 = System.nanoTime() var count = 0L var payloadSize = 0L - Source.fromGraph(new AeronSource(channel1, aeron)) + Source.fromGraph(new AeronSource(channel1, aeron, taskRunner)) .map { bytes ⇒ r.onMessage(1, bytes.length) bytes @@ -172,19 +179,19 @@ object AeronStreamsApp { r.onMessage(1, payload.length) payload } - .runWith(new AeronSink(channel1, aeron)) + .runWith(new AeronSink(channel1, aeron, taskRunner)) } def runEchoReceiver(): Unit = { // just echo back on channel2 reporterExecutor.execute(reporter) val r = reporter - Source.fromGraph(new AeronSource(channel1, aeron)) + Source.fromGraph(new AeronSource(channel1, aeron, taskRunner)) .map { bytes ⇒ r.onMessage(1, bytes.length) bytes } - .runWith(new AeronSink(channel2, aeron)) + .runWith(new AeronSink(channel2, aeron, taskRunner)) } def runEchoSender(): Unit = { @@ -196,7 +203,7 @@ object AeronStreamsApp { var repeat = 3 val count = new AtomicInteger var t0 = System.nanoTime() - Source.fromGraph(new AeronSource(channel2, aeron)) + Source.fromGraph(new AeronSource(channel2, aeron, taskRunner)) .map { bytes ⇒ r.onMessage(1, bytes.length) bytes @@ -231,7 +238,7 @@ object AeronStreamsApp { sendTimes.set(n - 1, System.nanoTime()) payload } - .runWith(new AeronSink(channel1, aeron)) + .runWith(new AeronSink(channel1, aeron, taskRunner)) barrier.await() } @@ -241,7 +248,7 @@ object AeronStreamsApp { def runDebugReceiver(): Unit = { import system.dispatcher - Source.fromGraph(new AeronSource(channel1, aeron)) + Source.fromGraph(new AeronSource(channel1, aeron, taskRunner)) .map(bytes ⇒ new String(bytes, "utf-8")) .runForeach { s ⇒ println(s) @@ -261,7 +268,7 @@ object AeronStreamsApp { println(s) s.getBytes("utf-8") } - .runWith(new AeronSink(channel1, aeron)) + .runWith(new AeronSink(channel1, aeron, taskRunner)) } } diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 2973c04915..89221bdbe4 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -120,7 +120,7 @@ object AkkaBuild extends Build { lazy val remote = Project( id = "akka-remote", base = file("akka-remote"), - dependencies = Seq(actor, stream, actorTests % "test->test", testkit % "test->test", protobuf) + dependencies = Seq(actor, stream, actorTests % "test->test", testkit % "test->test", streamTestkit % "test", protobuf) ) lazy val multiNodeTestkit = Project( @@ -132,7 +132,7 @@ object AkkaBuild extends Build { lazy val remoteTests = Project( id = "akka-remote-tests", base = file("akka-remote-tests"), - dependencies = Seq(actorTests % "test->test", remote % "test->test", multiNodeTestkit) + dependencies = Seq(actorTests % "test->test", remote % "test->test", streamTestkit % "test", multiNodeTestkit) ) configs (MultiJvm) lazy val cluster = Project( From fbfe0b0068448fc0236323a10dfa91daf0cac25e Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 3 May 2016 20:42:17 +0200 Subject: [PATCH 013/186] update to Aeron 0.9.7 --- project/Dependencies.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 7e09655211..2e1747660c 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -67,8 +67,8 @@ object Dependencies { // For Java 8 Conversions val java8Compat = "org.scala-lang.modules" %% "scala-java8-compat" % "0.7.0" // Scala License - val aeronDriver = "io.aeron" % "aeron-driver" % "0.9.5" // ApacheV2 - val aeronClient = "io.aeron" % "aeron-client" % "0.9.5" // ApacheV2 + val aeronDriver = "io.aeron" % "aeron-driver" % "0.9.7" // ApacheV2 + val aeronClient = "io.aeron" % "aeron-client" % "0.9.7" // ApacheV2 object Docs { val sprayJson = "io.spray" %% "spray-json" % "1.3.2" % "test" From 4c8260c78a282294c0599d71d0499ccf5901dd04 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 25 Apr 2016 08:40:58 +0200 Subject: [PATCH 014/186] show Aeron stats --- .../artery/AeronStreamLatencySpec.scala | 11 + .../artery/AeronStreamMaxThroughputSpec.scala | 16 +- .../java/akka/remote/artery/AeronStat.java | 273 ++++++++++++++++++ .../akka/remote/artery/AeronStreamsApp.scala | 27 +- 4 files changed, 321 insertions(+), 6 deletions(-) create mode 100644 akka-remote/src/test/java/akka/remote/artery/AeronStat.java 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 index 628981e0fa..f5f45cbe4d 100644 --- 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 @@ -26,6 +26,8 @@ import akka.stream.KillSwitches import akka.Done import org.agrona.IoUtil import java.io.File +import java.io.File +import io.aeron.CncFileDescriptor object AeronStreamLatencySpec extends MultiNodeConfig { val first = role("first") @@ -80,6 +82,9 @@ abstract class AeronStreamLatencySpec val driver = MediaDriver.launchEmbedded() + val stats = + new AeronStat(AeronStat.mapCounters(new File(driver.aeronDirectoryName, CncFileDescriptor.CNC_FILE))) + val aeron = { val ctx = new Aeron.Context ctx.aeronDirectoryName(driver.aeronDirectoryName) @@ -152,6 +157,11 @@ abstract class AeronStreamLatencySpec } } + def printStats(side: String): Unit = { + println(side + " stats:") + stats.print(System.out) + } + val scenarios = List( TestSettings( testName = "rate-100-size-100", @@ -236,6 +246,7 @@ abstract class AeronStreamLatencySpec rep.halt() } + printStats(myself.name) enterBarrier("after-" + testName) } 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 index e59fd296fc..d86b55f0e5 100644 --- 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 @@ -20,6 +20,8 @@ import com.typesafe.config.ConfigFactory import io.aeron.Aeron import io.aeron.driver.MediaDriver import akka.stream.KillSwitches +import java.io.File +import io.aeron.CncFileDescriptor object AeronStreamMaxThroughputSpec extends MultiNodeConfig { val first = role("first") @@ -82,9 +84,13 @@ abstract class AeronStreamMaxThroughputSpec var plot = PlotResult() + val driver = MediaDriver.launchEmbedded() + + val stats = + new AeronStat(AeronStat.mapCounters(new File(driver.aeronDirectoryName, CncFileDescriptor.CNC_FILE))) + val aeron = { val ctx = new Aeron.Context - val driver = MediaDriver.launchEmbedded() ctx.aeronDirectoryName(driver.aeronDirectoryName) Aeron.connect(ctx) } @@ -139,6 +145,11 @@ abstract class AeronStreamMaxThroughputSpec 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", @@ -183,6 +194,7 @@ abstract class AeronStreamMaxThroughputSpec enterBarrier(receiverName + "-started") Await.ready(done, barrierTimeout) rep.halt() + printStats("receiver") enterBarrier(testName + "-done") } @@ -195,7 +207,9 @@ abstract class AeronStreamMaxThroughputSpec .map { n ⇒ payload } .runWith(new AeronSink(channel(second), aeron, taskRunner)) + printStats("sender") enterBarrier(testName + "-done") + } enterBarrier("after-" + testName) 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..feab499287 --- /dev/null +++ b/akka-remote/src/test/java/akka/remote/artery/AeronStat.java @@ -0,0 +1,273 @@ +/* + * 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 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/scala/akka/remote/artery/AeronStreamsApp.scala b/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala index d2a312cb34..ce337bdfa3 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala @@ -23,6 +23,8 @@ import io.aeron.UnavailableImageHandler import io.aeron.Image import io.aeron.AvailableImageHandler import akka.actor.ExtendedActorSystem +import java.io.File +import io.aeron.CncFileDescriptor object AeronStreamsApp { @@ -34,6 +36,18 @@ object AeronStreamsApp { val payload = ("0" * 100).getBytes("utf-8") lazy val sendTimes = new AtomicLongArray(latencyN) + lazy val driver = { + val driverContext = new MediaDriver.Context + driverContext.clientLivenessTimeoutNs(SECONDS.toNanos(10)) + driverContext.imageLivenessTimeoutNs(SECONDS.toNanos(10)) + driverContext.driverTimeoutMs(SECONDS.toNanos(10)) + MediaDriver.launchEmbedded(driverContext) + } + + lazy val stats = { + new AeronStat(AeronStat.mapCounters(new File(driver.aeronDirectoryName, CncFileDescriptor.CNC_FILE))) + } + lazy val aeron = { val ctx = new Aeron.Context ctx.errorHandler(new ErrorHandler { @@ -52,11 +66,6 @@ object AeronStreamsApp { } }) - val driverContext = new MediaDriver.Context - driverContext.clientLivenessTimeoutNs(SECONDS.toNanos(10)) - driverContext.imageLivenessTimeoutNs(SECONDS.toNanos(10)) - driverContext.driverTimeoutMs(SECONDS.toNanos(10)) - val driver = MediaDriver.launchEmbedded(driverContext) ctx.aeronDirectoryName(driver.aeronDirectoryName) Aeron.connect(ctx) } @@ -132,6 +141,9 @@ object AeronStreamsApp { if (args(0) == "debug-sender") runDebugSender() + + if (args.length >= 2 && args(1) == "stats") + runStats() } def runReceiver(): Unit = { @@ -257,6 +269,7 @@ object AeronStreamsApp { e.printStackTrace exit(-1) } + } def runDebugSender(): Unit = { @@ -271,4 +284,8 @@ object AeronStreamsApp { .runWith(new AeronSink(channel1, aeron, taskRunner)) } + def runStats(): Unit = { + Source.tick(10.second, 10.second, "tick").runForeach { _ ⇒ stats.print(System.out) } + } + } From 4b8870ca45d30068a20cfd948242271481b90595 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 3 May 2016 21:16:30 +0200 Subject: [PATCH 015/186] more readable number format --- .../remote/artery/AeronStreamLatencySpec.scala | 9 ++------- .../artery/AeronStreamMaxThroughputSpec.scala | 11 +++-------- .../scala/akka/remote/artery/LatencySpec.scala | 9 ++------- .../akka/remote/artery/MaxThroughputSpec.scala | 13 ++++--------- .../akka/remote/artery/TestRateReporter.scala | 18 ++++++++++++++++++ 5 files changed, 29 insertions(+), 31 deletions(-) create mode 100644 akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TestRateReporter.scala 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 index f5f45cbe4d..958c5d9174 100644 --- 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 @@ -108,13 +108,8 @@ abstract class AeronStreamLatencySpec } lazy val reporterExecutor = Executors.newFixedThreadPool(1) - def reporter(name: String): RateReporter = { - val r = new RateReporter(SECONDS.toNanos(1), new RateReporter.Reporter { - override def onReport(messagesPerSec: Double, bytesPerSec: Double, totalMessages: Long, totalBytes: Long): Unit = { - println(name + ": %.03g msgs/sec, %.03g bytes/sec, totals %d messages %d MB".format( - messagesPerSec, bytesPerSec, totalMessages, totalBytes / (1024 * 1024))) - } - }) + def reporter(name: String): TestRateReporter = { + val r = new TestRateReporter(name) reporterExecutor.execute(r) r } 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 index d86b55f0e5..3e68635e7f 100644 --- 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 @@ -114,13 +114,8 @@ abstract class AeronStreamMaxThroughputSpec } lazy val reporterExecutor = Executors.newFixedThreadPool(1) - def reporter(name: String): RateReporter = { - val r = new RateReporter(SECONDS.toNanos(1), new RateReporter.Reporter { - override def onReport(messagesPerSec: Double, bytesPerSec: Double, totalMessages: Long, totalBytes: Long): Unit = { - println(name + ": %.03g msgs/sec, %.03g bytes/sec, totals %d messages %d MB".format( - messagesPerSec, bytesPerSec, totalMessages, totalBytes / (1024 * 1024))) - } - }) + def reporter(name: String): TestRateReporter = { + val r = new TestRateReporter(name) reporterExecutor.execute(r) r } @@ -139,7 +134,7 @@ abstract class AeronStreamMaxThroughputSpec val d = (System.nanoTime - startTime).nanos.toMillis val throughput = 1000.0 * total / d println(f"=== AeronStreamMaxThroughput $testName: " + - f"${throughput}%.03g msg/s, ${throughput * payloadSize}%.03g bytes/s, " + + 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) 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 index 322bc12735..102ee4cb86 100644 --- 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 @@ -179,13 +179,8 @@ abstract class LatencySpec } lazy val reporterExecutor = Executors.newFixedThreadPool(1) - def reporter(name: String): RateReporter = { - val r = new RateReporter(SECONDS.toNanos(1), new RateReporter.Reporter { - override def onReport(messagesPerSec: Double, bytesPerSec: Double, totalMessages: Long, totalBytes: Long): Unit = { - println(name + ": %.03g msgs/sec, %.03g bytes/sec, totals %d messages %d MB".format( - messagesPerSec, bytesPerSec, totalMessages, totalBytes / (1024 * 1024))) - } - }) + def reporter(name: String): TestRateReporter = { + val r = new TestRateReporter(name) reporterExecutor.execute(r) r } 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 index 2aedd70c94..2f4444adbb 100644 --- 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 @@ -129,8 +129,8 @@ object MaxThroughputSpec extends MultiNodeConfig { val throughput = (totalReceived * 1000.0 / took) println( s"=== MaxThroughput ${self.path.name}: " + - f"throughput ${throughput}%.03g msg/s, " + - f"${throughput * payloadSize}%.03g bytes/s, " + + f"throughput ${throughput}%,.0f msg/s, " + + f"${throughput * payloadSize}%,.0f bytes/s, " + s"dropped ${totalMessages - totalReceived}, " + s"max round-trip $maxRoundTripMillis ms, " + s"burst size $burstSize, " + @@ -187,13 +187,8 @@ abstract class MaxThroughputSpec def remoteSettings = system.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].remoteSettings lazy val reporterExecutor = Executors.newFixedThreadPool(1) - def reporter(name: String): RateReporter = { - val r = new RateReporter(SECONDS.toNanos(1), new RateReporter.Reporter { - override def onReport(messagesPerSec: Double, bytesPerSec: Double, totalMessages: Long, totalBytes: Long): Unit = { - println(name + ": %.03g msgs/sec, %.03g bytes/sec, totals %d messages %d MB".format( - messagesPerSec, bytesPerSec, totalMessages, totalBytes / (1024 * 1024))) - } - }) + def reporter(name: String): TestRateReporter = { + val r = new TestRateReporter(name) reporterExecutor.execute(r) r } 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..2cef0d0ec6 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TestRateReporter.scala @@ -0,0 +1,18 @@ +/** + * 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") + } + }) { + +} From b93b1a09abd1f3cf4aeb3c04fd96abb1598bbc43 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 4 May 2016 13:31:08 +0200 Subject: [PATCH 016/186] first stab at system message delivery, #20323 --- .../artery/AeronStreamConcistencySpec.scala | 16 +- .../artery/AeronStreamLatencySpec.scala | 13 +- .../artery/AeronStreamMaxThroughputSpec.scala | 7 +- .../scala/akka/remote/artery/AeronSink.scala | 3 +- .../akka/remote/artery/AeronSource.scala | 3 +- .../akka/remote/artery/ArterySubsystem.scala | 27 +- .../remote/artery/SystemMessageDelivery.scala | 310 ++++++++++++++++++ .../scala/akka/remote/artery/Transport.scala | 157 ++++++++- .../akka/remote/artery/AeronStreamsApp.scala | 20 +- .../artery/RemoteSendConsistencySpec.scala | 17 +- .../artery/SystemMessageDeliverySpec.scala | 282 ++++++++++++++++ 11 files changed, 804 insertions(+), 51 deletions(-) create mode 100644 akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala 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 index 8e599f6376..28572c8418 100644 --- 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 @@ -55,9 +55,10 @@ abstract class AeronStreamConsistencySpec import AeronStreamConsistencySpec._ + val driver = MediaDriver.launchEmbedded() + val aeron = { val ctx = new Aeron.Context - val driver = MediaDriver.launchEmbedded() ctx.aeronDirectoryName(driver.aeronDirectoryName) Aeron.connect(ctx) } @@ -78,9 +79,12 @@ abstract class AeronStreamConsistencySpec s"aeron:udp?endpoint=${a.host.get}:${aeronPort(roleName)}" } + val streamId = 1 + override def afterAll(): Unit = { taskRunner.stop() aeron.close() + driver.close() super.afterAll() } @@ -89,8 +93,8 @@ abstract class AeronStreamConsistencySpec "start echo" in { runOn(second) { // just echo back - Source.fromGraph(new AeronSource(channel(second), aeron, taskRunner)) - .runWith(new AeronSink(channel(first), aeron, taskRunner)) + Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner)) + .runWith(new AeronSink(channel(first), streamId, aeron, taskRunner)) } enterBarrier("echo-started") } @@ -103,7 +107,7 @@ abstract class AeronStreamConsistencySpec val killSwitch = KillSwitches.shared("test") val started = TestProbe() val startMsg = "0".getBytes("utf-8") - Source.fromGraph(new AeronSource(channel(first), aeron, taskRunner)) + Source.fromGraph(new AeronSource(channel(first), streamId, aeron, taskRunner)) .via(killSwitch.flow) .runForeach { bytes ⇒ if (bytes.length == 1 && bytes(0) == startMsg(0)) @@ -124,14 +128,14 @@ abstract class AeronStreamConsistencySpec within(10.seconds) { Source(1 to 100).map(_ ⇒ startMsg) .throttle(1, 200.milliseconds, 1, ThrottleMode.Shaping) - .runWith(new AeronSink(channel(second), aeron, taskRunner)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner)) started.expectMsg(Done) } Source(1 to totalMessages) .throttle(10000, 1.second, 1000, ThrottleMode.Shaping) .map { n ⇒ n.toString.getBytes("utf-8") } - .runWith(new AeronSink(channel(second), aeron, taskRunner)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner)) Await.ready(done, 20.seconds) killSwitch.shutdown() 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 index 958c5d9174..d8cb7f34fb 100644 --- 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 @@ -107,6 +107,8 @@ abstract class AeronStreamLatencySpec s"aeron:udp?endpoint=${a.host.get}:${aeronPort(roleName)}" } + val streamId = 1 + lazy val reporterExecutor = Executors.newFixedThreadPool(1) def reporter(name: String): TestRateReporter = { val r = new TestRateReporter(name) @@ -118,6 +120,7 @@ abstract class AeronStreamLatencySpec reporterExecutor.shutdown() taskRunner.stop() aeron.close() + driver.close() IoUtil.delete(new File(driver.aeronDirectoryName), true) runOn(first) { println(plots.plot50.csv(system.name + "50")) @@ -196,7 +199,7 @@ abstract class AeronStreamLatencySpec val killSwitch = KillSwitches.shared(testName) val started = TestProbe() val startMsg = "0".getBytes("utf-8") - Source.fromGraph(new AeronSource(channel(first), aeron, taskRunner)) + Source.fromGraph(new AeronSource(channel(first), streamId, aeron, taskRunner)) .via(killSwitch.flow) .runForeach { bytes ⇒ if (bytes.length == 1 && bytes(0) == startMsg(0)) @@ -217,7 +220,7 @@ abstract class AeronStreamLatencySpec within(10.seconds) { Source(1 to 50).map(_ ⇒ startMsg) .throttle(1, 200.milliseconds, 1, ThrottleMode.Shaping) - .runWith(new AeronSink(channel(second), aeron, taskRunner)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner)) started.expectMsg(Done) } @@ -232,7 +235,7 @@ abstract class AeronStreamLatencySpec sendTimes.set(n - 1, System.nanoTime()) payload } - .runWith(new AeronSink(channel(second), aeron, taskRunner)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner)) barrier.await((totalMessages / messageRate) + 10, SECONDS) } @@ -250,8 +253,8 @@ abstract class AeronStreamLatencySpec "start echo" in { runOn(second) { // just echo back - Source.fromGraph(new AeronSource(channel(second), aeron, taskRunner)) - .runWith(new AeronSink(channel(first), aeron, taskRunner)) + Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner)) + .runWith(new AeronSink(channel(first), streamId, aeron, taskRunner)) } enterBarrier("echo-started") } 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 index 3e68635e7f..f374461700 100644 --- 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 @@ -113,6 +113,8 @@ abstract class AeronStreamMaxThroughputSpec s"aeron:udp?endpoint=${a.host.get}:${aeronPort(roleName)}" } + val streamId = 1 + lazy val reporterExecutor = Executors.newFixedThreadPool(1) def reporter(name: String): TestRateReporter = { val r = new TestRateReporter(name) @@ -124,6 +126,7 @@ abstract class AeronStreamMaxThroughputSpec reporterExecutor.shutdown() taskRunner.stop() aeron.close() + driver.close() runOn(second) { println(plot.csv(system.name)) } @@ -169,7 +172,7 @@ abstract class AeronStreamMaxThroughputSpec var count = 0L val done = TestLatch(1) val killSwitch = KillSwitches.shared(testName) - Source.fromGraph(new AeronSource(channel(second), aeron, taskRunner)) + Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner)) .via(killSwitch.flow) .runForeach { bytes ⇒ rep.onMessage(1, bytes.length) @@ -200,7 +203,7 @@ abstract class AeronStreamMaxThroughputSpec val t0 = System.nanoTime() Source.fromIterator(() ⇒ iterate(1, totalMessages)) .map { n ⇒ payload } - .runWith(new AeronSink(channel(second), aeron, taskRunner)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner)) printStats("sender") enterBarrier(testName + "-done") diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala index 948f8cb856..f3427d7c92 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala @@ -51,7 +51,7 @@ object AeronSink { /** * @param channel eg. "aeron:udp?endpoint=localhost:40123" */ -class AeronSink(channel: String, aeron: Aeron, taskRunner: TaskRunner) extends GraphStage[SinkShape[AeronSink.Bytes]] { +class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRunner) extends GraphStage[SinkShape[AeronSink.Bytes]] { import AeronSink._ import TaskRunner._ @@ -62,7 +62,6 @@ class AeronSink(channel: String, aeron: Aeron, taskRunner: TaskRunner) extends G new GraphStageLogic(shape) with InHandler { private val buffer = new UnsafeBuffer(ByteBuffer.allocateDirect(128 * 1024)) - private val streamId = 10 private val pub = aeron.addPublication(channel, streamId) private val spinning = 1000 diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala index 5231cc8f0f..c03aa69dea 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala @@ -61,7 +61,7 @@ object AeronSource { /** * @param channel eg. "aeron:udp?endpoint=localhost:40123" */ -class AeronSource(channel: String, aeron: Aeron, taskRunner: TaskRunner) extends GraphStage[SourceShape[AeronSource.Bytes]] { +class AeronSource(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRunner) extends GraphStage[SourceShape[AeronSource.Bytes]] { import AeronSource._ import TaskRunner._ @@ -71,7 +71,6 @@ class AeronSource(channel: String, aeron: Aeron, taskRunner: TaskRunner) extends override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) with OutHandler { - private val streamId = 10 private val sub = aeron.addSubscription(channel, streamId) private val spinning = 1000 private val yielding = 0 diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala index bf950c5591..49bbd62e4e 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala @@ -5,7 +5,6 @@ package akka.remote.artery import java.util.concurrent.ConcurrentHashMap - import akka.actor.{ ActorRef, Address, ExtendedActorSystem } import akka.event.{ Logging, LoggingAdapter } import akka.remote.EndpointManager.Send @@ -14,9 +13,9 @@ import akka.remote.{ DefaultMessageDispatcher, RemoteActorRef, RemoteActorRefPro import akka.stream.scaladsl.{ Sink, Source, SourceQueueWithComplete, Tcp } import akka.stream.{ ActorMaterializer, Materializer, OverflowStrategy } import akka.{ Done, NotUsed } - import scala.concurrent.duration._ import scala.concurrent.{ Await, Future } +import akka.dispatch.sysmsg.SystemMessage /** * INTERNAL API @@ -49,8 +48,7 @@ private[remote] class ArterySubsystem(_system: ExtendedActorSystem, _provider: R system, materializer, provider, - AkkaPduProtobufCodec, - new DefaultMessageDispatcher(system, provider, log)) + AkkaPduProtobufCodec) transport.start() } @@ -100,13 +98,24 @@ private[akka] class Association( val materializer: Materializer, val remoteAddress: Address, val transport: Transport) { + @volatile private[this] var queue: SourceQueueWithComplete[Send] = _ - private[this] val sink: Sink[Send, Any] = transport.outbound(remoteAddress) + @volatile private[this] var systemMessageQueue: SourceQueueWithComplete[Send] = _ def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = { // TODO: lookup subchannel // FIXME: Use a different envelope than the old Send, but make sure the new is handled by deadLetters properly - queue.offer(Send(message, senderOption, recipient, None)) + message match { + case _: SystemMessage | _: SystemMessageDelivery.SystemMessageReply ⇒ + implicit val ec = materializer.executionContext + systemMessageQueue.offer(Send(message, senderOption, recipient, None)).onFailure { + case e ⇒ + // FIXME proper error handling, and quarantining + println(s"# System message dropped, due to $e") // FIXME + } + case _ ⇒ + queue.offer(Send(message, senderOption, recipient, None)) + } } def quarantine(uid: Option[Int]): Unit = () @@ -114,7 +123,11 @@ private[akka] class Association( // Idempotent def associate(): Unit = { if (queue eq null) - queue = Source.queue(256, OverflowStrategy.dropBuffer).to(sink).run()(materializer) + queue = Source.queue(256, OverflowStrategy.dropBuffer) + .to(transport.outbound(remoteAddress)).run()(materializer) + if (systemMessageQueue eq null) + systemMessageQueue = Source.queue(256, OverflowStrategy.dropBuffer) + .to(transport.outboundSystemMessage(remoteAddress)).run()(materializer) } } 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..c8e3036e10 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -0,0 +1,310 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.ArrayDeque + +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 akka.Done +import akka.actor.ActorRef +import akka.actor.Address +import akka.remote.EndpointManager.Send +import akka.remote.artery.Transport.InboundEnvelope +import akka.stream.Attributes +import akka.stream.FlowShape +import akka.stream.Inlet +import akka.stream.Outlet +import akka.stream.stage.AsyncCallback +import akka.stream.stage.GraphStage +import akka.stream.stage.GraphStageLogic +import akka.stream.stage.GraphStageWithMaterializedValue +import akka.stream.stage.InHandler +import akka.stream.stage.OutHandler +import akka.stream.stage.TimerGraphStageLogic + +/** + * INTERNAL API + */ +private[akka] object SystemMessageDelivery { + // FIXME serialization of these messages + final case class SystemMessageEnvelope(message: AnyRef, seqNo: Long, ackReplyTo: ActorRef) + sealed trait SystemMessageReply + final case class Ack(seq: Long, from: Address) extends SystemMessageReply + final case class Nack(seq: Long, from: Address) extends SystemMessageReply + + private case object ResendTick +} + +/** + * INTERNAL API + */ +private[akka] class SystemMessageDelivery( + replyJunction: SystemMessageReplyJunction.Junction, + resendInterval: FiniteDuration, + localAddress: Address, + remoteAddress: Address, + ackRecipient: ActorRef) + extends GraphStage[FlowShape[Send, Send]] { + + import SystemMessageDelivery._ + import SystemMessageReplyJunction._ + + val in: Inlet[Send] = Inlet("SystemMessageDelivery.in") + val out: Outlet[Send] = Outlet("SystemMessageDelivery.out") + override val shape: FlowShape[Send, Send] = FlowShape(in, out) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new TimerGraphStageLogic(shape) with InHandler with OutHandler { + + var registered = false + var seqNo = 0L // sequence number for the first message will be 1 + val unacknowledged = new ArrayDeque[Send] + var resending = new ArrayDeque[Send] + var resendingFromSeqNo = -1L + var stopping = false + + override def preStart(): Unit = { + this.schedulePeriodically(ResendTick, resendInterval) + def filter(env: InboundEnvelope): Boolean = + env.message match { + case Ack(_, from) if from == remoteAddress ⇒ true + case Nack(_, from) if from == remoteAddress ⇒ true + case _ ⇒ false + } + + implicit val ec = materializer.executionContext + replyJunction.addReplyInterest(filter, ackCallback).foreach { + getAsyncCallback[Done] { _ ⇒ + registered = true + if (isAvailable(out)) + pull(in) // onPull from downstream already called + }.invoke + } + + replyJunction.stopped.onComplete { + getAsyncCallback[Try[Done]] { + // FIXME quarantine + case Success(_) ⇒ completeStage() + case Failure(cause) ⇒ failStage(cause) + }.invoke + } + } + + override def postStop(): Unit = { + replyJunction.removeReplyInterest(ackCallback) + } + + override def onUpstreamFinish(): Unit = { + if (unacknowledged.isEmpty) + super.onUpstreamFinish() + else + stopping = true + } + + override protected def onTimer(timerKey: Any): Unit = + timerKey match { + case ResendTick ⇒ + if (resending.isEmpty && !unacknowledged.isEmpty) { + resending = unacknowledged.clone() + tryResend() + } + } + + val ackCallback = getAsyncCallback[SystemMessageReply] { reply ⇒ + reply match { + case Ack(n, _) ⇒ + ack(n) + case Nack(n, _) ⇒ + ack(n) + if (n > resendingFromSeqNo) + resending = unacknowledged.clone() + tryResend() + } + } + + private def ack(n: Long): Unit = { + if (n > seqNo) + throw new IllegalArgumentException(s"Unexpected ack $n, when highest sent seqNo is $seqNo") + clearUnacknowledged(n) + } + + @tailrec private def clearUnacknowledged(ackedSeqNo: Long): Unit = { + if (!unacknowledged.isEmpty && + unacknowledged.peek().message.asInstanceOf[SystemMessageEnvelope].seqNo <= ackedSeqNo) { + unacknowledged.removeFirst() + if (stopping && unacknowledged.isEmpty) + completeStage() + else + clearUnacknowledged(ackedSeqNo) + } + } + + private def tryResend(): Unit = { + if (isAvailable(out) && !resending.isEmpty) + push(out, resending.poll()) + } + + // InHandler + override def onPush(): Unit = { + grab(in) match { + case s @ Send(reply: SystemMessageReply, _, _, _) ⇒ + // pass through + if (isAvailable(out)) + push(out, s) + else { + // it's ok to drop the replies, but we can try + resending.offer(s) + } + + case s @ Send(msg: AnyRef, _, _, _) ⇒ + seqNo += 1 + val sendMsg = s.copy(message = SystemMessageEnvelope(msg, seqNo, ackRecipient)) + // FIXME quarantine if unacknowledged is full + unacknowledged.offer(sendMsg) + if (resending.isEmpty && isAvailable(out)) + push(out, sendMsg) + else { + resending.offer(sendMsg) + tryResend() + } + } + } + + // OutHandler + override def onPull(): Unit = { + if (registered) { // otherwise it will be pulled after replyJunction.addReplyInterest + if (resending.isEmpty && !hasBeenPulled(in) && !stopping) + pull(in) + else + tryResend() + } + } + + setHandlers(in, out, this) + } +} + +/** + * INTERNAL API + */ +private[akka] class SystemMessageAcker(localAddress: Address) 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 { + + var seqNo = 1L + + // InHandler + override def onPush(): Unit = { + grab(in) match { + case env @ InboundEnvelope(_, _, sysEnv @ SystemMessageEnvelope(_, n, ackReplyTo), _) ⇒ + if (n == seqNo) { + ackReplyTo.tell(Ack(n, localAddress), ActorRef.noSender) + seqNo += 1 + val unwrapped = env.copy(message = sysEnv.message) + push(out, unwrapped) + } else if (n < seqNo) { + ackReplyTo.tell(Ack(n, localAddress), ActorRef.noSender) + pull(in) + } else { + ackReplyTo.tell(Nack(seqNo - 1, localAddress), ActorRef.noSender) + pull(in) + } + case env ⇒ + // messages that don't need acking + push(out, env) + } + + } + + // OutHandler + override def onPull(): Unit = pull(in) + + setHandlers(in, out, this) + } +} + +/** + * INTERNAL API + */ +private[akka] object SystemMessageReplyJunction { + import SystemMessageDelivery._ + + trait Junction { + def addReplyInterest(filter: InboundEnvelope ⇒ Boolean, replyCallback: AsyncCallback[SystemMessageReply]): Future[Done] + def removeReplyInterest(callback: AsyncCallback[SystemMessageReply]): Unit + def stopped: Future[Done] + } +} + +/** + * INTERNAL API + */ +private[akka] class SystemMessageReplyJunction + extends GraphStageWithMaterializedValue[FlowShape[InboundEnvelope, InboundEnvelope], SystemMessageReplyJunction.Junction] { + import SystemMessageReplyJunction._ + import SystemMessageDelivery._ + + val in: Inlet[InboundEnvelope] = Inlet("SystemMessageReplyJunction.in") + val out: Outlet[InboundEnvelope] = Outlet("SystemMessageReplyJunction.out") + override val shape: FlowShape[InboundEnvelope, InboundEnvelope] = FlowShape(in, out) + + override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = { + val logic = new GraphStageLogic(shape) with InHandler with OutHandler with Junction { + + private var replyHandlers: Vector[(InboundEnvelope ⇒ Boolean, AsyncCallback[SystemMessageReply])] = Vector.empty + private val stoppedPromise = Promise[Done]() + + override def postStop(): Unit = stoppedPromise.success(Done) + + // InHandler + override def onPush(): Unit = { + grab(in) match { + case env @ InboundEnvelope(_, _, reply: SystemMessageReply, _) ⇒ + replyHandlers.foreach { + case (f, callback) ⇒ + if (f(env)) + callback.invoke(reply) + } + pull(in) + case env ⇒ + push(out, env) + } + } + + // OutHandler + override def onPull(): Unit = pull(in) + + override def addReplyInterest(filter: InboundEnvelope ⇒ Boolean, replyCallback: AsyncCallback[SystemMessageReply]): Future[Done] = { + val p = Promise[Done]() + getAsyncCallback[Unit](_ ⇒ { + replyHandlers :+= (filter -> replyCallback) + p.success(Done) + }).invoke(()) + p.future + } + + override def removeReplyInterest(callback: AsyncCallback[SystemMessageReply]): Unit = { + replyHandlers = replyHandlers.filterNot { case (_, c) ⇒ c == callback } + } + + override def stopped: Future[Done] = stoppedPromise.future + + setHandlers(in, out, this) + } + (logic, logic) + } +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/Transport.scala b/akka-remote/src/main/scala/akka/remote/artery/Transport.scala index 6dd7d9a7b4..fb9563def7 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Transport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Transport.scala @@ -4,6 +4,8 @@ package akka.remote.artery +import scala.concurrent.duration._ +import akka.actor.Props import scala.concurrent.duration._ import java.net.InetSocketAddress import java.nio.ByteOrder @@ -29,6 +31,28 @@ import io.aeron.AvailableImageHandler import io.aeron.Image import io.aeron.UnavailableImageHandler import io.aeron.exceptions.ConductorServiceTimeoutException +import akka.actor.LocalRef +import akka.actor.InternalActorRef +import akka.dispatch.sysmsg.SystemMessage +import akka.actor.PossiblyHarmful +import akka.actor.RepointableRef +import akka.actor.ActorSelectionMessage +import akka.remote.RemoteRef +import akka.actor.ActorSelection +import akka.actor.ActorRef +import akka.stream.scaladsl.Keep + +/** + * INTERNAL API + */ +private[akka] object Transport { + // FIXME avoid allocating this envelope? + final case class InboundEnvelope( + recipient: InternalActorRef, + recipientAddress: Address, + message: AnyRef, + senderOption: Option[ActorRef]) +} /** * INTERNAL API @@ -39,14 +63,37 @@ private[akka] class Transport( val system: ExtendedActorSystem, val materializer: Materializer, val provider: RemoteActorRefProvider, - val codec: AkkaPduCodec, - val inboundDispatcher: InboundMessageDispatcher) { + val codec: AkkaPduCodec) { + import Transport._ private val log: LoggingAdapter = Logging(system.eventStream, getClass.getName) + private val remoteDaemon = provider.remoteDaemon private implicit val mat = materializer // TODO support port 0 private val inboundChannel = s"aeron:udp?endpoint=${localAddress.host.get}:${localAddress.port.get}" + private def outboundChannel(a: Address) = s"aeron:udp?endpoint=${a.host.get}:${a.port.get}" + private val systemMessageStreamId = 1 + private val ordinaryStreamId = 3 + + private val systemMessageResendInterval: FiniteDuration = 1.second // FIXME config + + private var systemMessageReplyJunction: SystemMessageReplyJunction.Junction = _ + + // Need an ActorRef that is passed in the `SystemMessageEnvelope.ackReplyTo`. + // Those messages are not actually handled by this actor, but intercepted by the + // SystemMessageReplyJunction stage. + private val systemMessageReplyRecepient = system.systemActorOf(Props.empty, "systemMessageReplyTo") + + private val driver = { + // TODO also support external media driver + val driverContext = new MediaDriver.Context + // FIXME settings from config + driverContext.clientLivenessTimeoutNs(SECONDS.toNanos(10)) + driverContext.imageLivenessTimeoutNs(SECONDS.toNanos(10)) + driverContext.driverTimeoutMs(SECONDS.toNanos(10)) + MediaDriver.launchEmbedded(driverContext) + } private val aeron = { val ctx = new Aeron.Context @@ -76,13 +123,6 @@ private[akka] class Transport( } } }) - // TODO also support external media driver - val driverContext = new MediaDriver.Context - // FIXME settings from config - driverContext.clientLivenessTimeoutNs(SECONDS.toNanos(10)) - driverContext.imageLivenessTimeoutNs(SECONDS.toNanos(10)) - driverContext.driverTimeoutMs(SECONDS.toNanos(10)) - val driver = MediaDriver.launchEmbedded(driverContext) ctx.aeronDirectoryName(driver.aeronDirectoryName) Aeron.connect(ctx) @@ -92,7 +132,13 @@ private[akka] class Transport( def start(): Unit = { taskRunner.start() - Source.fromGraph(new AeronSource(inboundChannel, aeron, taskRunner)) + systemMessageReplyJunction = Source.fromGraph(new AeronSource(inboundChannel, systemMessageStreamId, aeron, taskRunner)) + .async // FIXME use dedicated dispatcher for AeronSource + .map(ByteString.apply) // TODO we should use ByteString all the way + .viaMat(inboundSystemMessageFlow)(Keep.right) + .to(Sink.ignore) + .run() + Source.fromGraph(new AeronSource(inboundChannel, ordinaryStreamId, aeron, taskRunner)) .async // FIXME use dedicated dispatcher for AeronSource .map(ByteString.apply) // TODO we should use ByteString all the way .via(inboundFlow) @@ -103,17 +149,26 @@ private[akka] class Transport( // FIXME stop the AeronSource first? taskRunner.stop() aeron.close() + driver.close() Future.successful(Done) } val killSwitch: SharedKillSwitch = KillSwitches.shared("transportKillSwitch") def outbound(remoteAddress: Address): Sink[Send, Any] = { - val outboundChannel = s"aeron:udp?endpoint=${remoteAddress.host.get}:${remoteAddress.port.get}" Flow.fromGraph(killSwitch.flow[Send]) .via(encoder) .map(_.toArray) // TODO we should use ByteString all the way - .to(new AeronSink(outboundChannel, aeron, taskRunner)) + .to(new AeronSink(outboundChannel(remoteAddress), ordinaryStreamId, aeron, taskRunner)) + } + + def outboundSystemMessage(remoteAddress: Address): Sink[Send, Any] = { + Flow.fromGraph(killSwitch.flow[Send]) + .via(new SystemMessageDelivery(systemMessageReplyJunction, systemMessageResendInterval, + localAddress, remoteAddress, systemMessageReplyRecepient)) + .via(encoder) + .map(_.toArray) // TODO we should use ByteString all the way + .to(new AeronSink(outboundChannel(remoteAddress), systemMessageStreamId, aeron, taskRunner)) } // TODO: Try out parallelized serialization (mapAsync) for performance @@ -141,14 +196,86 @@ private[akka] class Transport( pdu } - val messageDispatcher: Sink[AkkaPduCodec.Message, Any] = Sink.foreach[AkkaPduCodec.Message] { m ⇒ - inboundDispatcher.dispatch(m.recipient, m.recipientAddress, m.serializedMessage, m.senderOption) + val messageDispatcher: Sink[InboundEnvelope, Future[Done]] = Sink.foreach[InboundEnvelope] { m ⇒ + dispatchInboundMessage(m.recipient, m.recipientAddress, m.message, m.senderOption) } + val deserializer: Flow[AkkaPduCodec.Message, InboundEnvelope, NotUsed] = + Flow[AkkaPduCodec.Message].map { m ⇒ + InboundEnvelope( + m.recipient, + m.recipientAddress, + MessageSerializer.deserialize(system, m.serializedMessage), + m.senderOption) + } + val inboundFlow: Flow[ByteString, ByteString, NotUsed] = { Flow.fromSinkAndSource( - decoder.to(messageDispatcher), + decoder.via(deserializer).to(messageDispatcher), Source.maybe[ByteString].via(killSwitch.flow)) } + val inboundSystemMessageFlow: Flow[ByteString, ByteString, SystemMessageReplyJunction.Junction] = { + Flow.fromSinkAndSourceMat( + decoder.via(deserializer) + .via(new SystemMessageAcker(localAddress)) + .viaMat(new SystemMessageReplyJunction)(Keep.right) + .to(messageDispatcher), + Source.maybe[ByteString].via(killSwitch.flow))((a, b) ⇒ a) + } + + private def dispatchInboundMessage(recipient: InternalActorRef, + recipientAddress: Address, + message: AnyRef, + senderOption: Option[ActorRef]): Unit = { + + import provider.remoteSettings._ + + val sender: ActorRef = senderOption.getOrElse(system.deadLetters) + val originalReceiver = recipient.path + + def msgLog = s"RemoteMessage: [$message] to [$recipient]<+[$originalReceiver] from [$sender()]" + + recipient match { + + case `remoteDaemon` ⇒ + if (UntrustedMode) log.debug("dropping daemon message in untrusted mode") + else { + if (LogReceive) log.debug("received daemon message {}", msgLog) + remoteDaemon ! message + } + + case l @ (_: LocalRef | _: RepointableRef) if l.isLocal ⇒ + if (LogReceive) log.debug("received local message {}", msgLog) + message match { + case sel: ActorSelectionMessage ⇒ + if (UntrustedMode && (!TrustedSelectionPaths.contains(sel.elements.mkString("/", "/", "")) || + sel.msg.isInstanceOf[PossiblyHarmful] || l != provider.rootGuardian)) + 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 ⇒ + log.debug("operating in UntrustedMode, dropping inbound PossiblyHarmful message of type [{}]", msg.getClass.getName) + case msg: SystemMessage ⇒ l.sendSystemMessage(msg) + case msg ⇒ l.!(msg)(sender) + } + + case r @ (_: RemoteRef | _: RepointableRef) if !r.isLocal && !UntrustedMode ⇒ + if (LogReceive) log.debug("received remote-destined message {}", msgLog) + if (provider.transport.addresses(recipientAddress)) + // if it was originally addressed to us but is in fact remote from our point of view (i.e. remote-deployed) + r.!(message)(sender) + else + log.error("dropping message [{}] for non-local recipient [{}] arriving at [{}] inbound addresses are [{}]", + message.getClass, r, recipientAddress, provider.transport.addresses.mkString(", ")) + + case r ⇒ log.error("dropping message [{}] for unknown recipient [{}] arriving at [{}] inbound addresses are [{}]", + message.getClass, r, recipientAddress, provider.transport.addresses.mkString(", ")) + + } + } + } diff --git a/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala b/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala index ce337bdfa3..834e124a5a 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala @@ -1,3 +1,6 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ package akka.remote.artery import scala.concurrent.duration._ @@ -30,6 +33,7 @@ object AeronStreamsApp { val channel1 = "aeron:udp?endpoint=localhost:40123" val channel2 = "aeron:udp?endpoint=localhost:40124" + val streamId = 1 val throughputN = 10000000 val latencyRate = 10000 // per second val latencyN = 10 * latencyRate @@ -153,7 +157,7 @@ object AeronStreamsApp { var t0 = System.nanoTime() var count = 0L var payloadSize = 0L - Source.fromGraph(new AeronSource(channel1, aeron, taskRunner)) + Source.fromGraph(new AeronSource(channel1, streamId, aeron, taskRunner)) .map { bytes ⇒ r.onMessage(1, bytes.length) bytes @@ -191,19 +195,19 @@ object AeronStreamsApp { r.onMessage(1, payload.length) payload } - .runWith(new AeronSink(channel1, aeron, taskRunner)) + .runWith(new AeronSink(channel1, streamId, aeron, taskRunner)) } def runEchoReceiver(): Unit = { // just echo back on channel2 reporterExecutor.execute(reporter) val r = reporter - Source.fromGraph(new AeronSource(channel1, aeron, taskRunner)) + Source.fromGraph(new AeronSource(channel1, streamId, aeron, taskRunner)) .map { bytes ⇒ r.onMessage(1, bytes.length) bytes } - .runWith(new AeronSink(channel2, aeron, taskRunner)) + .runWith(new AeronSink(channel2, streamId, aeron, taskRunner)) } def runEchoSender(): Unit = { @@ -215,7 +219,7 @@ object AeronStreamsApp { var repeat = 3 val count = new AtomicInteger var t0 = System.nanoTime() - Source.fromGraph(new AeronSource(channel2, aeron, taskRunner)) + Source.fromGraph(new AeronSource(channel2, streamId, aeron, taskRunner)) .map { bytes ⇒ r.onMessage(1, bytes.length) bytes @@ -250,7 +254,7 @@ object AeronStreamsApp { sendTimes.set(n - 1, System.nanoTime()) payload } - .runWith(new AeronSink(channel1, aeron, taskRunner)) + .runWith(new AeronSink(channel1, streamId, aeron, taskRunner)) barrier.await() } @@ -260,7 +264,7 @@ object AeronStreamsApp { def runDebugReceiver(): Unit = { import system.dispatcher - Source.fromGraph(new AeronSource(channel1, aeron, taskRunner)) + Source.fromGraph(new AeronSource(channel1, streamId, aeron, taskRunner)) .map(bytes ⇒ new String(bytes, "utf-8")) .runForeach { s ⇒ println(s) @@ -281,7 +285,7 @@ object AeronStreamsApp { println(s) s.getBytes("utf-8") } - .runWith(new AeronSink(channel1, aeron, taskRunner)) + .runWith(new AeronSink(channel1, streamId, aeron, taskRunner)) } def runStats(): Unit = { diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala index b6c320206c..c94e21ad64 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala @@ -1,3 +1,6 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ package akka.remote.artery import scala.concurrent.duration._ @@ -6,23 +9,29 @@ import akka.testkit.{ AkkaSpec, ImplicitSender } import com.typesafe.config.ConfigFactory import RemoteSendConsistencySpec._ import akka.actor.Actor.Receive +import akka.testkit.SocketUtil object RemoteSendConsistencySpec { - val commonConfig = """ + val Seq(portA, portB) = SocketUtil.temporaryServerAddresses(2, "localhost", udp = true).map(_.getPort) + + val commonConfig = ConfigFactory.parseString(s""" akka { actor.provider = "akka.remote.RemoteActorRefProvider" remote.artery.enabled = on remote.artery.hostname = localhost + remote.artery.port = $portA } - """ + """) + + val configB = ConfigFactory.parseString(s"akka.remote.artery.port = $portB") + .withFallback(commonConfig) } class RemoteSendConsistencySpec extends AkkaSpec(commonConfig) with ImplicitSender { - val configB = ConfigFactory.parseString("akka.remote.artery.port = 20201") - val systemB = ActorSystem("systemB", configB.withFallback(system.settings.config)) + val systemB = ActorSystem("systemB", RemoteSendConsistencySpec.configB) val addressB = systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress println(addressB) val rootB = RootActorPath(addressB) 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..a95802d69b --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -0,0 +1,282 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.Await +import scala.concurrent.Future +import scala.concurrent.Promise +import scala.concurrent.duration._ +import scala.concurrent.forkjoin.ThreadLocalRandom + +import akka.Done +import akka.NotUsed +import akka.actor.Actor +import akka.actor.ActorIdentity +import akka.actor.ActorRef +import akka.actor.ActorSystem +import akka.actor.ExtendedActorSystem +import akka.actor.Identify +import akka.actor.InternalActorRef +import akka.actor.PoisonPill +import akka.actor.Props +import akka.actor.RootActorPath +import akka.actor.Stash +import akka.remote.EndpointManager.Send +import akka.remote.RemoteActorRef +import akka.remote.artery.SystemMessageDelivery._ +import akka.remote.artery.Transport.InboundEnvelope +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.stage.AsyncCallback +import akka.stream.testkit.TestSubscriber +import akka.stream.testkit.scaladsl.TestSink +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 SystemMessageDeliverySpec { + + val Seq(portA, portB) = SocketUtil.temporaryServerAddresses(2, "localhost", udp = true).map(_.getPort) + + val commonConfig = ConfigFactory.parseString(s""" + akka { + actor.provider = "akka.remote.RemoteActorRefProvider" + remote.artery.enabled = on + remote.artery.hostname = localhost + remote.artery.port = $portA + } + akka.actor.serialize-creators = off + akka.actor.serialize-messages = off + """) + + val configB = ConfigFactory.parseString(s"akka.remote.artery.port = $portB") + .withFallback(commonConfig) + + class TestReplyJunction(sendCallbackTo: ActorRef) extends SystemMessageReplyJunction.Junction { + + def addReplyInterest(filter: InboundEnvelope ⇒ Boolean, replyCallback: AsyncCallback[SystemMessageReply]): Future[Done] = { + sendCallbackTo ! replyCallback + Future.successful(Done) + } + + override def removeReplyInterest(callback: AsyncCallback[SystemMessageReply]): Unit = () + + override def stopped: Future[Done] = Promise[Done]().future + } + + def replyConnectorProps(dropRate: Double): Props = + Props(new ReplyConnector(dropRate)) + + class ReplyConnector(dropRate: Double) extends Actor with Stash { + override def receive = { + case callback: AsyncCallback[SystemMessageReply] @unchecked ⇒ + context.become(active(callback)) + unstashAll() + case _ ⇒ stash() + } + + def active(callback: AsyncCallback[SystemMessageReply]): Receive = { + case reply: SystemMessageReply ⇒ + if (ThreadLocalRandom.current().nextDouble() >= dropRate) + callback.invoke(reply) + } + } + +} + +class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commonConfig) with ImplicitSender { + import SystemMessageDeliverySpec._ + + val addressA = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + val systemB = ActorSystem("systemB", configB) + val addressB = systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + val rootB = RootActorPath(addressB) + val matSettings = ActorMaterializerSettings(system).withFuzzing(true) + implicit val mat = ActorMaterializer(matSettings)(system) + + override def afterTermination(): Unit = shutdown(systemB) + + def setupManualCallback(ackRecipient: ActorRef, resendInterval: FiniteDuration, + dropSeqNumbers: Vector[Long], sendCount: Int): (TestSubscriber.Probe[String], AsyncCallback[SystemMessageReply]) = { + val callbackProbe = TestProbe() + val replyJunction = new TestReplyJunction(callbackProbe.ref) + + val sink = + send(sendCount, resendInterval, replyJunction, ackRecipient) + .via(drop(dropSeqNumbers)) + .via(inbound) + .map(_.message.asInstanceOf[String]) + .runWith(TestSink.probe) + + val callback = callbackProbe.expectMsgType[AsyncCallback[SystemMessageReply]] + (sink, callback) + } + + def send(sendCount: Int, resendInterval: FiniteDuration, replyJunction: SystemMessageReplyJunction.Junction, + ackRecipient: ActorRef): Source[Send, NotUsed] = { + val remoteRef = null.asInstanceOf[RemoteActorRef] // not used + Source(1 to sendCount) + .map(n ⇒ Send("msg-" + n, None, remoteRef, None)) + .via(new SystemMessageDelivery(replyJunction, resendInterval, addressA, addressB, ackRecipient)) + } + + def inbound: Flow[Send, InboundEnvelope, NotUsed] = { + val recipient = null.asInstanceOf[InternalActorRef] // not used + Flow[Send] + .map { + case Send(sysEnv: SystemMessageEnvelope, _, _, _) ⇒ + InboundEnvelope(recipient, addressB, sysEnv, None) + } + .async + .via(new SystemMessageAcker(addressB)) + } + + def drop(dropSeqNumbers: Vector[Long]): Flow[Send, Send, NotUsed] = { + Flow[Send] + .statefulMapConcat(() ⇒ { + var dropping = dropSeqNumbers + + { + case s @ Send(SystemMessageEnvelope(_, seqNo, _), _, _, _) ⇒ + val i = dropping.indexOf(seqNo) + if (i >= 0) { + dropping = dropping.updated(i, -1L) + Nil + } else + List(s) + } + }) + } + + 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 { + val actorOnSystemB = 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 resent when some in the middle are lost" in { + val ackRecipient = TestProbe() + val (sink, replyCallback) = + setupManualCallback(ackRecipient.ref, resendInterval = 60.seconds, dropSeqNumbers = Vector(3L, 4L), sendCount = 5) + + sink.request(100) + sink.expectNext("msg-1") + sink.expectNext("msg-2") + ackRecipient.expectMsg(Ack(1L, addressB)) + ackRecipient.expectMsg(Ack(2L, addressB)) + // 3 and 4 was dropped + ackRecipient.expectMsg(Nack(2L, addressB)) + sink.expectNoMsg(100.millis) // 3 was dropped + replyCallback.invoke(Nack(2L, addressB)) + // resending 3, 4, 5 + sink.expectNext("msg-3") + ackRecipient.expectMsg(Ack(3L, addressB)) + sink.expectNext("msg-4") + ackRecipient.expectMsg(Ack(4L, addressB)) + sink.expectNext("msg-5") + ackRecipient.expectMsg(Ack(5L, addressB)) + ackRecipient.expectNoMsg(100.millis) + replyCallback.invoke(Ack(5L, addressB)) + sink.expectComplete() + } + + "be resent when first is lost" in { + val ackRecipient = TestProbe() + val (sink, replyCallback) = + setupManualCallback(ackRecipient.ref, resendInterval = 60.seconds, dropSeqNumbers = Vector(1L), sendCount = 3) + + sink.request(100) + ackRecipient.expectMsg(Nack(0L, addressB)) // from receiving 2 + ackRecipient.expectMsg(Nack(0L, addressB)) // from receiving 3 + sink.expectNoMsg(100.millis) // 1 was dropped + replyCallback.invoke(Nack(0L, addressB)) + replyCallback.invoke(Nack(0L, addressB)) + // resending 1, 2, 3 + sink.expectNext("msg-1") + ackRecipient.expectMsg(Ack(1L, addressB)) + sink.expectNext("msg-2") + ackRecipient.expectMsg(Ack(2L, addressB)) + sink.expectNext("msg-3") + ackRecipient.expectMsg(Ack(3L, addressB)) + replyCallback.invoke(Ack(3L, addressB)) + sink.expectComplete() + } + + "be resent when last is lost" in { + val ackRecipient = TestProbe() + val (sink, replyCallback) = + setupManualCallback(ackRecipient.ref, resendInterval = 1.second, dropSeqNumbers = Vector(3L), sendCount = 3) + + sink.request(100) + sink.expectNext("msg-1") + ackRecipient.expectMsg(Ack(1L, addressB)) + replyCallback.invoke(Ack(1L, addressB)) + sink.expectNext("msg-2") + ackRecipient.expectMsg(Ack(2L, addressB)) + replyCallback.invoke(Ack(2L, addressB)) + sink.expectNoMsg(200.millis) // 3 was dropped + // resending 3 due to timeout + sink.expectNext("msg-3") + ackRecipient.expectMsg(Ack(3L, addressB)) + replyCallback.invoke(Ack(3L, addressB)) + sink.expectComplete() + } + + "deliver all during stress and random dropping" in { + val N = 10000 + val dropRate = 0.1 + val replyConnector = system.actorOf(replyConnectorProps(dropRate)) + val replyJunction = new TestReplyJunction(replyConnector) + + val output = + send(N, 1.second, replyJunction, replyConnector) + .via(randomDrop(dropRate)) + .via(inbound) + .map(_.message.asInstanceOf[String]) + .runWith(Sink.seq) + + Await.result(output, 20.seconds) should ===((1 to N).map("msg-" + _).toVector) + } + + "deliver all during throttling and random dropping" in { + val N = 500 + val dropRate = 0.1 + val replyConnector = system.actorOf(replyConnectorProps(dropRate)) + val replyJunction = new TestReplyJunction(replyConnector) + + val output = + send(N, 1.second, replyJunction, replyConnector) + .throttle(200, 1.second, 10, ThrottleMode.shaping) + .via(randomDrop(dropRate)) + .via(inbound) + .map(_.message.asInstanceOf[String]) + .runWith(Sink.seq) + + Await.result(output, 20.seconds) should ===((1 to N).map("msg-" + _).toVector) + } + + } + +} From cc347db22858f7a5d51573f3b605d592ae952d4d Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 10 May 2016 18:23:21 +0200 Subject: [PATCH 017/186] flush outstanding task in AeronSink before completing --- .../scala/akka/remote/artery/AeronSink.scala | 20 +++++++++++++++++-- 1 file changed, 18 insertions(+), 2 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala index f3427d7c92..590035ea01 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala @@ -70,7 +70,12 @@ class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRu private var lastMsgSizeRef = new AtomicInteger // used in the external backoff task private val addOfferTask: Add = Add(offerTask(pub, buffer, lastMsgSizeRef, getAsyncCallback(_ ⇒ onOfferSuccess()))) - override def preStart(): Unit = pull(in) + private var offerTaskInProgress = false + + override def preStart(): Unit = { + setKeepGoing(true) + pull(in) + } override def postStop(): Unit = { taskRunner.command(Remove(addOfferTask.task)) @@ -97,6 +102,7 @@ class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRu } else { // delegate backoff to shared TaskRunner lastMsgSizeRef.set(lastMsgSize) + offerTaskInProgress = true taskRunner.command(addOfferTask) } } else { @@ -105,7 +111,17 @@ class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRu } private def onOfferSuccess(): Unit = { - pull(in) + offerTaskInProgress = false + if (isClosed(in)) + completeStage() + else + pull(in) + } + + override def onUpstreamFinish(): Unit = { + // flush outstanding offer before completing stage + if (!offerTaskInProgress) + super.onUpstreamFinish() } setHandler(in, this) From 16cf8d4ab6c666c948dd8a8b2056e5ededca5191 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 9 May 2016 07:31:41 +0200 Subject: [PATCH 018/186] first stab at handshake, #20313 * UID exchange with handshake stages * second iteration of reply side-channel, observable * InboundContext and OutboundContext to facilitate testing without real transport * collapse ArterySubsystem and Transport into ArteryTransport * incomplete HandshakeRestartReceiverSpec (origin address missing to be able to implement that part * remove embedded aeron media driver directory on shutdown --- .../src/main/scala/akka/actor/Address.scala | 12 + .../artery/AeronStreamConcistencySpec.scala | 3 + .../artery/AeronStreamMaxThroughputSpec.scala | 2 + .../artery/HandshakeRestartReceiverSpec.scala | 135 +++++++ .../akka/remote/RemoteActorRefProvider.scala | 4 +- .../scala/akka/remote/UniqueAddress.scala | 17 + .../akka/remote/artery/ArterySubsystem.scala | 133 ------- .../akka/remote/artery/ArteryTransport.scala | 353 ++++++++++++++++++ .../akka/remote/artery/Association.scala | 80 ++++ .../scala/akka/remote/artery/Handshake.scala | 182 +++++++++ .../remote/artery/MessageDispatcher.scala | 84 +++++ .../main/scala/akka/remote/artery/Reply.scala | 97 +++++ .../remote/artery/SystemMessageDelivery.scala | 169 +++------ .../scala/akka/remote/artery/Transport.scala | 281 -------------- .../artery/SystemMessageDeliverySpec.scala | 204 +++++----- .../akka/remote/artery/TestContext.scala | 79 ++++ 16 files changed, 1194 insertions(+), 641 deletions(-) create mode 100644 akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/HandshakeRestartReceiverSpec.scala create mode 100644 akka-remote/src/main/scala/akka/remote/UniqueAddress.scala delete mode 100644 akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/Association.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/Handshake.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/Reply.scala delete mode 100644 akka-remote/src/main/scala/akka/remote/artery/Transport.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/TestContext.scala 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-remote-tests/src/multi-jvm/scala/akka/remote/artery/AeronStreamConcistencySpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/AeronStreamConcistencySpec.scala index 28572c8418..429a3569cd 100644 --- 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 @@ -20,6 +20,8 @@ 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 object AeronStreamConsistencySpec extends MultiNodeConfig { val first = role("first") @@ -85,6 +87,7 @@ abstract class AeronStreamConsistencySpec taskRunner.stop() aeron.close() driver.close() + IoUtil.delete(new File(driver.aeronDirectoryName), true) super.afterAll() } 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 index f374461700..042ed8fc90 100644 --- 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 @@ -22,6 +22,7 @@ import io.aeron.driver.MediaDriver import akka.stream.KillSwitches import java.io.File import io.aeron.CncFileDescriptor +import org.agrona.IoUtil object AeronStreamMaxThroughputSpec extends MultiNodeConfig { val first = role("first") @@ -127,6 +128,7 @@ abstract class AeronStreamMaxThroughputSpec taskRunner.stop() aeron.close() driver.close() + IoUtil.delete(new File(driver.aeronDirectoryName), true) runOn(second) { println(plot.csv(system.name)) } 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..0ea2f76dfb --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/HandshakeRestartReceiverSpec.scala @@ -0,0 +1,135 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.concurrent.Executors +import java.util.concurrent.TimeUnit.NANOSECONDS +import scala.concurrent.duration._ +import akka.actor._ +import akka.remote.RemoteActorRefProvider +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 +import java.net.InetAddress +import scala.concurrent.Await +import akka.remote.RARP +import akka.remote.AddressUidExtension + +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 = "akka.remote.RemoteActorRefProvider" + remote.artery { + enabled = on + } + } + """))) + + def aeronPort(roleName: RoleName): Int = + roleName match { + case `first` ⇒ 20531 // TODO yeah, we should have support for dynamic port assignment + case `second` ⇒ 20532 + } + + nodeConfig(first) { + ConfigFactory.parseString(s""" + akka.remote.artery.port = ${aeronPort(first)} + """) + } + + nodeConfig(second) { + ConfigFactory.parseString(s""" + akka.remote.artery.port = ${aeronPort(second)} + """) + } + + class Subject extends Actor { + def receive = { + case "shutdown" ⇒ context.system.terminate() + case "identify" ⇒ sender() ! (AddressUidExtension(context.system).addressUid -> 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): (Int, ActorRef) = { + system.actorSelection(rootPath / "user" / actorName) ! "identify" + expectMsgType[(Int, 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") + + val secondAddress = node(second).address + val secondAssociation = RARP(system).provider.transport.asInstanceOf[ArteryTransport].association(secondAddress) + val secondUniqueRemoteAddress = Await.result(secondAssociation.uniqueRemoteAddress, 3.seconds) + secondUniqueRemoteAddress.address should ===(secondAddress) + secondUniqueRemoteAddress.uid should ===(secondUid) + + enterBarrier("before-shutdown") + testConductor.shutdown(second).await + + within(30.seconds) { + awaitAssert { + within(1.second) { + identifyWithUid(secondRootPath, "subject2") + } + } + } + val (secondUid2, subject2) = identifyWithUid(secondRootPath, "subject2") + secondUid2 should !==(secondUid) + // FIXME verify that UID in association was replaced (not implemented yet) + + 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.port = ${addr.port.get} + """).withFallback(system.settings.config)) + freshSystem.actorOf(Props[Subject], "subject2") + + Await.result(freshSystem.whenTerminated, 45.seconds) + } + } + + } +} diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 0d6065da83..52fe8f2ba8 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -19,7 +19,7 @@ import scala.util.control.Exception.Catcher import scala.concurrent.Future import akka.ConfigurationException import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics } -import akka.remote.artery.ArterySubsystem +import akka.remote.artery.ArteryTransport /** * INTERNAL API @@ -182,7 +182,7 @@ private[akka] class RemoteActorRefProvider( d }, serialization = SerializationExtension(system), - transport = if (remoteSettings.EnableArtery) new ArterySubsystem(system, this) else new Remoting(system, this)) + transport = if (remoteSettings.EnableArtery) new ArteryTransport(system, this) else new Remoting(system, this)) _internals = internals remotingTerminator ! internals 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..7bb76716c5 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/UniqueAddress.scala @@ -0,0 +1,17 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote + +import akka.actor.Address + +@SerialVersionUID(1L) +final case class UniqueAddress(address: Address, uid: Int) extends Ordered[UniqueAddress] { + override def 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 + } +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala deleted file mode 100644 index 49bbd62e4e..0000000000 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySubsystem.scala +++ /dev/null @@ -1,133 +0,0 @@ -/** - * Copyright (C) 2009-2016 Lightbend Inc. - */ - -package akka.remote.artery - -import java.util.concurrent.ConcurrentHashMap -import akka.actor.{ ActorRef, Address, ExtendedActorSystem } -import akka.event.{ Logging, LoggingAdapter } -import akka.remote.EndpointManager.Send -import akka.remote.transport.AkkaPduProtobufCodec -import akka.remote.{ DefaultMessageDispatcher, RemoteActorRef, RemoteActorRefProvider, RemoteTransport } -import akka.stream.scaladsl.{ Sink, Source, SourceQueueWithComplete, Tcp } -import akka.stream.{ ActorMaterializer, Materializer, OverflowStrategy } -import akka.{ Done, NotUsed } -import scala.concurrent.duration._ -import scala.concurrent.{ Await, Future } -import akka.dispatch.sysmsg.SystemMessage - -/** - * INTERNAL API - */ -private[remote] class ArterySubsystem(_system: ExtendedActorSystem, _provider: RemoteActorRefProvider) extends RemoteTransport(_system, _provider) { - import provider.remoteSettings - - @volatile private[this] var address: Address = _ - @volatile private[this] var transport: Transport = _ - @volatile private[this] var tcpBinding: Option[Tcp.ServerBinding] = None - @volatile private[this] var materializer: Materializer = _ - override val log: LoggingAdapter = Logging(system.eventStream, getClass.getName) - - override def defaultAddress: Address = address - override def addresses: Set[Address] = Set(address) - override def localAddressForRemote(remote: Address): Address = defaultAddress - - // FIXME: This does locking on putIfAbsent, we need something smarter - private[this] val associations = new ConcurrentHashMap[Address, Association]() - - override def start(): Unit = { - // TODO: Configure materializer properly - // TODO: Have a supervisor actor - address = Address("akka.artery", system.name, remoteSettings.ArteryHostname, remoteSettings.ArteryPort) - materializer = ActorMaterializer()(system) - - transport = - new Transport( - address, - system, - materializer, - provider, - AkkaPduProtobufCodec) - transport.start() - } - - override def shutdown(): Future[Done] = { - if (transport != null) transport.shutdown() - else Future.successful(Done) - } - - override def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = { - val cached = recipient.cachedAssociation - val remoteAddress = recipient.path.address - - val association = - if (cached ne null) cached - else associate(remoteAddress) - - association.send(message, senderOption, recipient) - } - - private def associate(remoteAddress: Address): Association = { - val current = associations.get(remoteAddress) - if (current ne null) current - else { - associations.computeIfAbsent(remoteAddress, new java.util.function.Function[Address, Association] { - override def apply(remoteAddress: Address): Association = { - val newAssociation = new Association(materializer, remoteAddress, transport) - newAssociation.associate() // This is a bit costly for this blocking method :( - newAssociation - } - }) - } - } - - override def quarantine(remoteAddress: Address, uid: Option[Int]): Unit = { - ??? - } - -} - -/** - * INTERNAL API - * - * Thread-safe, mutable holder for association state. Main entry point for remote destined message to a specific - * remote address. - */ -private[akka] class Association( - val materializer: Materializer, - val remoteAddress: Address, - val transport: Transport) { - - @volatile private[this] var queue: SourceQueueWithComplete[Send] = _ - @volatile private[this] var systemMessageQueue: SourceQueueWithComplete[Send] = _ - - def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = { - // TODO: lookup subchannel - // FIXME: Use a different envelope than the old Send, but make sure the new is handled by deadLetters properly - message match { - case _: SystemMessage | _: SystemMessageDelivery.SystemMessageReply ⇒ - implicit val ec = materializer.executionContext - systemMessageQueue.offer(Send(message, senderOption, recipient, None)).onFailure { - case e ⇒ - // FIXME proper error handling, and quarantining - println(s"# System message dropped, due to $e") // FIXME - } - case _ ⇒ - queue.offer(Send(message, senderOption, recipient, None)) - } - } - - def quarantine(uid: Option[Int]): Unit = () - - // Idempotent - def associate(): Unit = { - if (queue eq null) - queue = Source.queue(256, OverflowStrategy.dropBuffer) - .to(transport.outbound(remoteAddress)).run()(materializer) - if (systemMessageQueue eq null) - systemMessageQueue = Source.queue(256, OverflowStrategy.dropBuffer) - .to(transport.outboundSystemMessage(remoteAddress)).run()(materializer) - } -} - 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..6b822f9521 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -0,0 +1,353 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.nio.ByteOrder +import java.util.concurrent.ConcurrentHashMap +import java.util.function.{ Function ⇒ JFunction } +import scala.concurrent.Future +import scala.concurrent.duration._ +import akka.Done +import akka.NotUsed +import akka.actor.ActorRef +import akka.actor.Address +import akka.actor.ExtendedActorSystem +import akka.actor.InternalActorRef +import akka.event.Logging +import akka.event.LoggingAdapter +import akka.remote.AddressUidExtension +import akka.remote.EndpointManager.Send +import akka.remote.MessageSerializer +import akka.remote.RemoteActorRef +import akka.remote.RemoteActorRefProvider +import akka.remote.RemoteTransport +import akka.remote.UniqueAddress +import akka.remote.artery.ReplyJunction.ReplySubject +import akka.remote.transport.AkkaPduCodec +import akka.remote.transport.AkkaPduProtobufCodec +import akka.serialization.Serialization +import akka.stream.ActorMaterializer +import akka.stream.KillSwitches +import akka.stream.Materializer +import akka.stream.SharedKillSwitch +import akka.stream.scaladsl.Flow +import akka.stream.scaladsl.Framing +import akka.stream.scaladsl.Keep +import akka.stream.scaladsl.Sink +import akka.stream.scaladsl.Source +import akka.util.ByteString +import akka.util.ByteStringBuilder +import io.aeron.Aeron +import io.aeron.AvailableImageHandler +import io.aeron.Image +import io.aeron.UnavailableImageHandler +import io.aeron.driver.MediaDriver +import io.aeron.exceptions.ConductorServiceTimeoutException +import org.agrona.ErrorHandler +import org.agrona.IoUtil +import java.io.File + +/** + * INTERNAL API + */ +private[akka] final case class InboundEnvelope( + recipient: InternalActorRef, + recipientAddress: Address, + message: AnyRef, + senderOption: Option[ActorRef]) + +/** + * INTERNAL API + * Inbound API that is used by the stream stages. + * Separate trait to facilitate testing without real transport. + */ +private[akka] trait InboundContext { + /** + * The local inbound address. + */ + def localAddress: UniqueAddress + + /** + * An inbound stage can send reply message to the origin + * address with this method. + */ + def sendReply(to: Address, message: ControlMessage): Unit + + /** + * Lookup the outbound association for a given address. + */ + def association(remoteAddress: Address): OutboundContext +} + +/** + * INTERNAL API + * Outbound association API that is used by the stream stages. + * Separate trait to facilitate testing without real transport. + */ +private[akka] trait OutboundContext { + /** + * The local inbound address. + */ + def localAddress: UniqueAddress + + /** + * The outbound address for this association. + */ + def remoteAddress: Address + + /** + * Full outbound address with UID for this association. + * Completed when by the handshake. + */ + def uniqueRemoteAddress: Future[UniqueAddress] + + /** + * Set the outbound address with UID when the + * handshake is completed. + */ + def completeRemoteAddress(a: UniqueAddress): Unit + + /** + * An outbound stage can listen to reply messages + * via this observer subject. + */ + def replySubject: ReplySubject + + // FIXME we should be able to Send without a recipient ActorRef + def dummyRecipient: RemoteActorRef +} + +/** + * INTERNAL API + */ +private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: RemoteActorRefProvider) + extends RemoteTransport(_system, _provider) with InboundContext { + import provider.remoteSettings + + // these vars are initialized once in the start method + @volatile private[this] var _localAddress: UniqueAddress = _ + override def localAddress: UniqueAddress = _localAddress + @volatile private[this] var materializer: Materializer = _ + @volatile private[this] var replySubject: ReplySubject = _ + @volatile private[this] var messageDispatcher: MessageDispatcher = _ + @volatile private[this] var driver: MediaDriver = _ + @volatile private[this] var aeron: Aeron = _ + + override val log: LoggingAdapter = Logging(system.eventStream, getClass.getName) + override def defaultAddress: Address = localAddress.address + override def addresses: Set[Address] = Set(defaultAddress) + override def localAddressForRemote(remote: Address): Address = defaultAddress + + private val codec: AkkaPduCodec = AkkaPduProtobufCodec + private val killSwitch: SharedKillSwitch = KillSwitches.shared("transportKillSwitch") + private val systemMessageResendInterval: FiniteDuration = 1.second // FIXME config + + // TODO support port 0 + private def inboundChannel = s"aeron:udp?endpoint=${localAddress.address.host.get}:${localAddress.address.port.get}" + private def outboundChannel(a: Address) = s"aeron:udp?endpoint=${a.host.get}:${a.port.get}" + private val systemMessageStreamId = 1 + private val ordinaryStreamId = 3 + private val taskRunner = new TaskRunner(system) + + // FIXME: This does locking on putIfAbsent, we need something smarter + private[this] val associations = new ConcurrentHashMap[Address, Association]() + + override def start(): Unit = { + startMediaDriver() + startAeron() + taskRunner.start() + + // TODO: Configure materializer properly + // TODO: Have a supervisor actor + _localAddress = UniqueAddress( + Address("akka.artery", system.name, remoteSettings.ArteryHostname, remoteSettings.ArteryPort), + AddressUidExtension(system).addressUid) + materializer = ActorMaterializer()(system) + + messageDispatcher = new MessageDispatcher(system, provider) + + runInboundFlows() + } + + private def startMediaDriver(): Unit = { + // TODO also support external media driver + val driverContext = new MediaDriver.Context + // FIXME settings from config + driverContext.clientLivenessTimeoutNs(SECONDS.toNanos(10)) + driverContext.imageLivenessTimeoutNs(SECONDS.toNanos(10)) + driverContext.driverTimeoutMs(SECONDS.toNanos(10)) + driver = MediaDriver.launchEmbedded(driverContext) + } + + private def startAeron(): Unit = { + val ctx = new Aeron.Context + + 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}") + // FIXME we should call FragmentAssembler.freeSessionBuffer when image is unavailable + } + }) + ctx.errorHandler(new ErrorHandler { + override def onError(cause: Throwable): Unit = { + cause match { + case e: ConductorServiceTimeoutException ⇒ + // Timeout between service calls + log.error(cause, s"Aeron ServiceTimeoutException, ${cause.getMessage}") + + case _ ⇒ + log.error(cause, s"Aeron error, ${cause.getMessage}") + } + } + }) + + ctx.aeronDirectoryName(driver.aeronDirectoryName) + aeron = Aeron.connect(ctx) + } + + private def runInboundFlows(): Unit = { + replySubject = Source.fromGraph(new AeronSource(inboundChannel, systemMessageStreamId, aeron, taskRunner)) + .async // FIXME measure + .map(ByteString.apply) // TODO we should use ByteString all the way + .viaMat(inboundSystemMessageFlow)(Keep.right) + .to(Sink.ignore) + .run()(materializer) + + Source.fromGraph(new AeronSource(inboundChannel, ordinaryStreamId, aeron, taskRunner)) + .async // FIXME measure + .map(ByteString.apply) // TODO we should use ByteString all the way + .via(inboundFlow) + .runWith(Sink.ignore)(materializer) + } + + override def shutdown(): Future[Done] = { + killSwitch.shutdown() + if (taskRunner != null) taskRunner.stop() + if (aeron != null) aeron.close() + if (driver != null) { + driver.close() + // FIXME only delete files for embedded media driver, and it should also be configurable + IoUtil.delete(new File(driver.aeronDirectoryName), true) + } + Future.successful(Done) + } + + // InboundContext + override def sendReply(to: Address, message: ControlMessage) = { + send(message, None, association(to).dummyRecipient) + } + + override def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = { + val cached = recipient.cachedAssociation + val remoteAddress = recipient.path.address + + val a = + if (cached ne null) cached + else association(remoteAddress) + + a.send(message, senderOption, recipient) + } + + override def association(remoteAddress: Address): Association = { + val current = associations.get(remoteAddress) + if (current ne null) current + else { + associations.computeIfAbsent(remoteAddress, new JFunction[Address, Association] { + override def apply(remoteAddress: Address): Association = { + val newAssociation = new Association(ArteryTransport.this, materializer, remoteAddress, replySubject) + newAssociation.associate() // This is a bit costly for this blocking method :( + newAssociation + } + }) + } + } + + override def quarantine(remoteAddress: Address, uid: Option[Int]): Unit = { + ??? + } + + def outbound(outboundContext: OutboundContext): Sink[Send, Any] = { + Flow.fromGraph(killSwitch.flow[Send]) + .via(new OutboundHandshake(outboundContext)) + .via(encoder) + .map(_.toArray) // TODO we should use ByteString all the way + .to(new AeronSink(outboundChannel(outboundContext.remoteAddress), ordinaryStreamId, aeron, taskRunner)) + } + + def outboundSystemMessage(outboundContext: OutboundContext): Sink[Send, Any] = { + Flow.fromGraph(killSwitch.flow[Send]) + .via(new OutboundHandshake(outboundContext)) + .via(new SystemMessageDelivery(outboundContext, systemMessageResendInterval)) + .via(encoder) + .map(_.toArray) // TODO we should use ByteString all the way + .to(new AeronSink(outboundChannel(outboundContext.remoteAddress), systemMessageStreamId, aeron, taskRunner)) + } + + // TODO: Try out parallelized serialization (mapAsync) for performance + val encoder: Flow[Send, ByteString, NotUsed] = Flow[Send].map { sendEnvelope ⇒ + val pdu: ByteString = codec.constructMessage( + sendEnvelope.recipient.localAddressToUse, + sendEnvelope.recipient, + Serialization.currentTransportInformation.withValue(Serialization.Information(localAddress.address, system)) { + MessageSerializer.serialize(system, sendEnvelope.message.asInstanceOf[AnyRef]) + }, + sendEnvelope.senderOption, + seqOption = None, // FIXME: Acknowledgements will be handled differently I just reused the old codec + ackOption = None) + + // TODO: Drop unserializable messages + // TODO: Drop oversized messages + (new ByteStringBuilder).putInt(pdu.size)(ByteOrder.LITTLE_ENDIAN).result() ++ pdu + } + + val decoder: Flow[ByteString, AkkaPduCodec.Message, NotUsed] = + Framing.lengthField(4, maximumFrameLength = 256000) + .map { frame ⇒ + // TODO: Drop unserializable messages + val pdu = codec.decodeMessage(frame.drop(4), provider, localAddress.address)._2.get + pdu + } + + val messageDispatcherSink: Sink[InboundEnvelope, Future[Done]] = Sink.foreach[InboundEnvelope] { m ⇒ + messageDispatcher.dispatch(m.recipient, m.recipientAddress, m.message, m.senderOption) + } + + val deserializer: Flow[AkkaPduCodec.Message, InboundEnvelope, NotUsed] = + Flow[AkkaPduCodec.Message].map { m ⇒ + InboundEnvelope( + m.recipient, + m.recipientAddress, + MessageSerializer.deserialize(system, m.serializedMessage), + m.senderOption) + } + + val inboundFlow: Flow[ByteString, ByteString, NotUsed] = { + Flow.fromSinkAndSource( + decoder + .via(deserializer) + .via(new InboundHandshake(this)) + .to(messageDispatcherSink), + Source.maybe[ByteString].via(killSwitch.flow)) + } + + val inboundSystemMessageFlow: Flow[ByteString, ByteString, ReplySubject] = { + Flow.fromSinkAndSourceMat( + decoder + .via(deserializer) + .via(new InboundHandshake(this)) + .via(new SystemMessageAcker(this)) + .viaMat(new ReplyJunction)(Keep.right) + .to(messageDispatcherSink), + Source.maybe[ByteString].via(killSwitch.flow))((a, b) ⇒ a) + } + +} + 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..1e6b8df019 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -0,0 +1,80 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.Future +import scala.concurrent.Promise + +import akka.actor.ActorRef +import akka.actor.Address + +import akka.actor.RootActorPath +import akka.dispatch.sysmsg.SystemMessage +import akka.remote.EndpointManager.Send +import akka.remote.RemoteActorRef +import akka.remote.UniqueAddress +import akka.remote.artery.ReplyJunction.ReplySubject +import akka.stream.Materializer +import akka.stream.OverflowStrategy +import akka.stream.scaladsl.Source +import akka.stream.scaladsl.SourceQueueWithComplete + +/** + * INTERNAL API + * + * Thread-safe, mutable holder for association state. Main entry point for remote destined message to a specific + * remote address. + */ +private[akka] class Association( + val transport: ArteryTransport, + val materializer: Materializer, + override val remoteAddress: Address, + override val replySubject: ReplySubject) extends OutboundContext { + + @volatile private[this] var queue: SourceQueueWithComplete[Send] = _ + @volatile private[this] var systemMessageQueue: SourceQueueWithComplete[Send] = _ + + override def localAddress: UniqueAddress = transport.localAddress + + // FIXME we also need to be able to switch to new uid + private val _uniqueRemoteAddress = Promise[UniqueAddress]() + override def uniqueRemoteAddress: Future[UniqueAddress] = _uniqueRemoteAddress.future + override def completeRemoteAddress(a: UniqueAddress): Unit = { + require(a.address == remoteAddress, s"Wrong UniqueAddress got [$a.address], expected [$remoteAddress]") + _uniqueRemoteAddress.trySuccess(a) + } + + def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = { + // TODO: lookup subchannel + // FIXME: Use a different envelope than the old Send, but make sure the new is handled by deadLetters properly + message match { + case _: SystemMessage | _: Reply ⇒ + implicit val ec = materializer.executionContext + systemMessageQueue.offer(Send(message, senderOption, recipient, None)).onFailure { + case e ⇒ + // FIXME proper error handling, and quarantining + println(s"# System message dropped, due to $e") // FIXME + } + case _ ⇒ + queue.offer(Send(message, senderOption, recipient, None)) + } + } + + // FIXME we should be able to Send without a recipient ActorRef + override val dummyRecipient: RemoteActorRef = + transport.provider.resolveActorRef(RootActorPath(remoteAddress) / "system" / "dummy").asInstanceOf[RemoteActorRef] + + def quarantine(uid: Option[Int]): Unit = () + + // Idempotent + def associate(): Unit = { + // FIXME detect and handle stream failure, e.g. handshake timeout + if (queue eq null) + queue = Source.queue(256, OverflowStrategy.dropBuffer) + .to(transport.outbound(this)).run()(materializer) + if (systemMessageQueue eq null) + systemMessageQueue = Source.queue(256, OverflowStrategy.dropBuffer) + .to(transport.outboundSystemMessage(this)).run()(materializer) + } +} 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..6590f82967 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -0,0 +1,182 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.concurrent.TimeoutException +import scala.concurrent.duration._ +import akka.Done +import akka.remote.EndpointManager.Send +import akka.remote.UniqueAddress +import akka.remote.artery.ReplyJunction.ReplyObserver +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 + +/** + * INTERNAL API + */ +private[akka] object OutboundHandshake { + // FIXME serialization for these messages + final case class HandshakeReq(from: UniqueAddress) extends ControlMessage + final case class HandshakeRsp(from: UniqueAddress) extends Reply + + private sealed trait HandshakeState + private case object Start extends HandshakeState + private case object ReplyObserverAttached extends HandshakeState + private case object ReqInProgress extends HandshakeState + private case object Completed extends HandshakeState + + private case object HandshakeTimeout + +} + +/** + * INTERNAL API + */ +private[akka] class OutboundHandshake(outboundContext: OutboundContext) extends GraphStage[FlowShape[Send, Send]] { + val in: Inlet[Send] = Inlet("OutboundHandshake.in") + val out: Outlet[Send] = Outlet("OutboundHandshake.out") + override val shape: FlowShape[Send, Send] = FlowShape(in, out) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new TimerGraphStageLogic(shape) with InHandler with OutHandler with ReplyObserver { + import OutboundHandshake._ + + private val timeout: FiniteDuration = 10.seconds // FIXME config + private var handshakeState: HandshakeState = Start + + private def remoteAddress = outboundContext.remoteAddress + + override def preStart(): Unit = { + if (outboundContext.uniqueRemoteAddress.isCompleted) { + handshakeState = Completed + } else { + implicit val ec = materializer.executionContext + outboundContext.replySubject.attach(this).foreach { + getAsyncCallback[Done] { _ ⇒ + if (handshakeState != Completed) { + if (isAvailable(out)) + pushHandshakeReq() + else + handshakeState = ReplyObserverAttached + } + }.invoke + } + + outboundContext.uniqueRemoteAddress.foreach { + getAsyncCallback[UniqueAddress] { a ⇒ + if (handshakeState != Completed) { + handshakeCompleted() + if (isAvailable(out) && !hasBeenPulled(in)) + pull(in) + } + }.invoke + } + + scheduleOnce(HandshakeTimeout, timeout) + } + } + + override def postStop(): Unit = { + outboundContext.replySubject.detach(this) + } + + // InHandler + override def onPush(): Unit = { + if (handshakeState != Completed) + throw new IllegalStateException(s"onPush before handshake completed, was [$handshakeState]") + push(out, grab(in)) + } + + // OutHandler + override def onPull(): Unit = { + handshakeState match { + case Completed ⇒ pull(in) + case ReplyObserverAttached ⇒ + pushHandshakeReq() + case Start ⇒ // will push HandshakeReq when ReplyObserver is attached + case ReqInProgress ⇒ // will pull when handshake reply is received + } + } + + private def pushHandshakeReq(): Unit = { + handshakeState = ReqInProgress + // FIXME we should be able to Send without recipient ActorRef + push(out, Send(HandshakeReq(outboundContext.localAddress), None, outboundContext.dummyRecipient, None)) + } + + private def handshakeCompleted(): Unit = { + handshakeState = Completed + cancelTimer(HandshakeTimeout) + outboundContext.replySubject.detach(this) + } + + override protected def onTimer(timerKey: Any): Unit = + timerKey match { + case HandshakeTimeout ⇒ + failStage(new TimeoutException( + s"Handshake with [$remoteAddress] did not complete within ${timeout.toMillis} ms")) + } + + // ReplyObserver, external call + override def reply(inboundEnvelope: InboundEnvelope): Unit = { + inboundEnvelope.message match { + case rsp: HandshakeRsp ⇒ + if (rsp.from.address == remoteAddress) { + getAsyncCallback[HandshakeRsp] { reply ⇒ + if (handshakeState != Completed) { + handshakeCompleted() + outboundContext.completeRemoteAddress(reply.from) + if (isAvailable(out) && !hasBeenPulled(in)) + pull(in) + } + }.invoke(rsp) + } + case _ ⇒ // not interested + } + } + + setHandlers(in, out, this) + } + +} + +/** + * INTERNAL API + */ +private[akka] class InboundHandshake(inboundContext: InboundContext) 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 InHandler with OutHandler { + import OutboundHandshake._ + + // InHandler + override def onPush(): Unit = { + grab(in) match { + case InboundEnvelope(_, _, HandshakeReq(from), _) ⇒ + inboundContext.association(from.address).completeRemoteAddress(from) + inboundContext.sendReply(from.address, HandshakeRsp(inboundContext.localAddress)) + pull(in) + case other ⇒ + push(out, other) + } + } + + // OutHandler + override def onPull(): Unit = pull(in) + + setHandlers(in, out, this) + + } + +} 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..3286729521 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala @@ -0,0 +1,84 @@ +/** + * 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 + +/** + * INTERNAL API + */ +private[akka] class MessageDispatcher( + system: ExtendedActorSystem, + provider: RemoteActorRefProvider) { + + private val remoteDaemon = provider.remoteDaemon + private val log = Logging(system.eventStream, getClass.getName) + + def dispatch(recipient: InternalActorRef, + recipientAddress: Address, + message: AnyRef, + senderOption: Option[ActorRef]): Unit = { + + import provider.remoteSettings._ + + val sender: ActorRef = senderOption.getOrElse(system.deadLetters) + val originalReceiver = recipient.path + + def msgLog = s"RemoteMessage: [$message] to [$recipient]<+[$originalReceiver] from [$sender()]" + + recipient match { + + case `remoteDaemon` ⇒ + if (UntrustedMode) log.debug("dropping daemon message in untrusted mode") + else { + if (LogReceive) log.debug("received daemon message {}", msgLog) + remoteDaemon ! message + } + + case l @ (_: LocalRef | _: RepointableRef) if l.isLocal ⇒ + if (LogReceive) log.debug("received local message {}", msgLog) + message match { + case sel: ActorSelectionMessage ⇒ + if (UntrustedMode && (!TrustedSelectionPaths.contains(sel.elements.mkString("/", "/", "")) || + sel.msg.isInstanceOf[PossiblyHarmful] || l != provider.rootGuardian)) + 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 ⇒ + log.debug("operating in UntrustedMode, dropping inbound PossiblyHarmful message of type [{}]", msg.getClass.getName) + case msg: SystemMessage ⇒ l.sendSystemMessage(msg) + case msg ⇒ l.!(msg)(sender) + } + + case r @ (_: RemoteRef | _: RepointableRef) if !r.isLocal && !UntrustedMode ⇒ + if (LogReceive) log.debug("received remote-destined message {}", msgLog) + if (provider.transport.addresses(recipientAddress)) + // if it was originally addressed to us but is in fact remote from our point of view (i.e. remote-deployed) + r.!(message)(sender) + else + log.error("dropping message [{}] for non-local recipient [{}] arriving at [{}] inbound addresses are [{}]", + message.getClass, r, recipientAddress, provider.transport.addresses.mkString(", ")) + + case r ⇒ log.error("dropping message [{}] for unknown recipient [{}] arriving at [{}] inbound addresses are [{}]", + message.getClass, r, recipientAddress, provider.transport.addresses.mkString(", ")) + + } + } + +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/Reply.scala b/akka-remote/src/main/scala/akka/remote/artery/Reply.scala new file mode 100644 index 0000000000..676425894e --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/Reply.scala @@ -0,0 +1,97 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +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.GraphStageLogic +import akka.stream.stage.GraphStageWithMaterializedValue +import akka.stream.stage.InHandler +import akka.stream.stage.OutHandler + +/** + * Marker trait for reply messages + */ +trait Reply extends ControlMessage + +/** + * 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`. + */ +trait ControlMessage + +/** + * INTERNAL API + */ +private[akka] object ReplyJunction { + + private[akka] trait ReplySubject { + def attach(observer: ReplyObserver): Future[Done] + def detach(observer: ReplyObserver): Unit + def stopped: Future[Done] + } + + private[akka] trait ReplyObserver { + def reply(inboundEnvelope: InboundEnvelope): Unit + } +} + +/** + * INTERNAL API + */ +private[akka] class ReplyJunction + extends GraphStageWithMaterializedValue[FlowShape[InboundEnvelope, InboundEnvelope], ReplyJunction.ReplySubject] { + import ReplyJunction._ + + val in: Inlet[InboundEnvelope] = Inlet("ReplyJunction.in") + val out: Outlet[InboundEnvelope] = Outlet("ReplyJunction.out") + override val shape: FlowShape[InboundEnvelope, InboundEnvelope] = FlowShape(in, out) + + override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = { + val logic = new GraphStageLogic(shape) with InHandler with OutHandler with ReplySubject { + + private var replyObservers: Vector[ReplyObserver] = Vector.empty + private val stoppedPromise = Promise[Done]() + + override def postStop(): Unit = stoppedPromise.success(Done) + + // InHandler + override def onPush(): Unit = { + grab(in) match { + case env @ InboundEnvelope(_, _, reply: Reply, _) ⇒ + replyObservers.foreach(_.reply(env)) + pull(in) + case env ⇒ + push(out, env) + } + } + + // OutHandler + override def onPull(): Unit = pull(in) + + override def attach(observer: ReplyObserver): Future[Done] = { + val p = Promise[Done]() + getAsyncCallback[Unit](_ ⇒ { + replyObservers :+= observer + p.success(Done) + }).invoke(()) + p.future + } + + override def detach(observer: ReplyObserver): Unit = { + replyObservers = replyObservers.filterNot(_ == observer) + } + + override def stopped: Future[Done] = stoppedPromise.future + + setHandlers(in, out, this) + } + (logic, logic) + } +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala index c8e3036e10..271f4a629a 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -6,26 +6,21 @@ package akka.remote.artery import java.util.ArrayDeque 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 akka.Done -import akka.actor.ActorRef -import akka.actor.Address import akka.remote.EndpointManager.Send -import akka.remote.artery.Transport.InboundEnvelope +import akka.remote.UniqueAddress +import akka.remote.artery.ReplyJunction.ReplyObserver import akka.stream.Attributes import akka.stream.FlowShape import akka.stream.Inlet import akka.stream.Outlet -import akka.stream.stage.AsyncCallback import akka.stream.stage.GraphStage import akka.stream.stage.GraphStageLogic -import akka.stream.stage.GraphStageWithMaterializedValue import akka.stream.stage.InHandler import akka.stream.stage.OutHandler import akka.stream.stage.TimerGraphStageLogic @@ -35,10 +30,10 @@ import akka.stream.stage.TimerGraphStageLogic */ private[akka] object SystemMessageDelivery { // FIXME serialization of these messages - final case class SystemMessageEnvelope(message: AnyRef, seqNo: Long, ackReplyTo: ActorRef) - sealed trait SystemMessageReply - final case class Ack(seq: Long, from: Address) extends SystemMessageReply - final case class Nack(seq: Long, from: Address) extends SystemMessageReply + // FIXME ackReplyTo should not be needed + final case class SystemMessageEnvelope(message: AnyRef, seqNo: Long, ackReplyTo: UniqueAddress) + final case class Ack(seqNo: Long, from: UniqueAddress) extends Reply + final case class Nack(seqNo: Long, from: UniqueAddress) extends Reply private case object ResendTick } @@ -47,49 +42,42 @@ private[akka] object SystemMessageDelivery { * INTERNAL API */ private[akka] class SystemMessageDelivery( - replyJunction: SystemMessageReplyJunction.Junction, - resendInterval: FiniteDuration, - localAddress: Address, - remoteAddress: Address, - ackRecipient: ActorRef) + outboundContext: OutboundContext, + resendInterval: FiniteDuration) extends GraphStage[FlowShape[Send, Send]] { import SystemMessageDelivery._ - import SystemMessageReplyJunction._ val in: Inlet[Send] = Inlet("SystemMessageDelivery.in") val out: Outlet[Send] = Outlet("SystemMessageDelivery.out") override val shape: FlowShape[Send, Send] = FlowShape(in, out) override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = - new TimerGraphStageLogic(shape) with InHandler with OutHandler { + new TimerGraphStageLogic(shape) with InHandler with OutHandler with ReplyObserver { - var registered = false - var seqNo = 0L // sequence number for the first message will be 1 - val unacknowledged = new ArrayDeque[Send] - var resending = new ArrayDeque[Send] - var resendingFromSeqNo = -1L - var stopping = false + private var replyObserverAttached = false + private var seqNo = 0L // sequence number for the first message will be 1 + private val unacknowledged = new ArrayDeque[Send] + private var resending = new ArrayDeque[Send] + private var resendingFromSeqNo = -1L + private var stopping = false + + private def localAddress = outboundContext.localAddress + private def remoteAddress = outboundContext.remoteAddress override def preStart(): Unit = { this.schedulePeriodically(ResendTick, resendInterval) - def filter(env: InboundEnvelope): Boolean = - env.message match { - case Ack(_, from) if from == remoteAddress ⇒ true - case Nack(_, from) if from == remoteAddress ⇒ true - case _ ⇒ false - } implicit val ec = materializer.executionContext - replyJunction.addReplyInterest(filter, ackCallback).foreach { + outboundContext.replySubject.attach(this).foreach { getAsyncCallback[Done] { _ ⇒ - registered = true + replyObserverAttached = true if (isAvailable(out)) pull(in) // onPull from downstream already called }.invoke } - replyJunction.stopped.onComplete { + outboundContext.replySubject.stopped.onComplete { getAsyncCallback[Try[Done]] { // FIXME quarantine case Success(_) ⇒ completeStage() @@ -99,7 +87,7 @@ private[akka] class SystemMessageDelivery( } override def postStop(): Unit = { - replyJunction.removeReplyInterest(ackCallback) + outboundContext.replySubject.detach(this) } override def onUpstreamFinish(): Unit = { @@ -118,18 +106,26 @@ private[akka] class SystemMessageDelivery( } } - val ackCallback = getAsyncCallback[SystemMessageReply] { reply ⇒ - reply match { - case Ack(n, _) ⇒ - ack(n) - case Nack(n, _) ⇒ - ack(n) - if (n > resendingFromSeqNo) - resending = unacknowledged.clone() - tryResend() + // ReplyObserver, external call + override def reply(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 ⇒ + ack(reply.seqNo) + if (reply.seqNo > resendingFromSeqNo) + resending = unacknowledged.clone() + tryResend() + } + private def ack(n: Long): Unit = { if (n > seqNo) throw new IllegalArgumentException(s"Unexpected ack $n, when highest sent seqNo is $seqNo") @@ -155,7 +151,7 @@ private[akka] class SystemMessageDelivery( // InHandler override def onPush(): Unit = { grab(in) match { - case s @ Send(reply: SystemMessageReply, _, _, _) ⇒ + case s @ Send(reply: ControlMessage, _, _, _) ⇒ // pass through if (isAvailable(out)) push(out, s) @@ -166,7 +162,7 @@ private[akka] class SystemMessageDelivery( case s @ Send(msg: AnyRef, _, _, _) ⇒ seqNo += 1 - val sendMsg = s.copy(message = SystemMessageEnvelope(msg, seqNo, ackRecipient)) + val sendMsg = s.copy(message = SystemMessageEnvelope(msg, seqNo, localAddress)) // FIXME quarantine if unacknowledged is full unacknowledged.offer(sendMsg) if (resending.isEmpty && isAvailable(out)) @@ -180,7 +176,7 @@ private[akka] class SystemMessageDelivery( // OutHandler override def onPull(): Unit = { - if (registered) { // otherwise it will be pulled after replyJunction.addReplyInterest + if (replyObserverAttached) { // otherwise it will be pulled after attached if (resending.isEmpty && !hasBeenPulled(in) && !stopping) pull(in) else @@ -195,7 +191,7 @@ private[akka] class SystemMessageDelivery( /** * INTERNAL API */ -private[akka] class SystemMessageAcker(localAddress: Address) extends GraphStage[FlowShape[InboundEnvelope, InboundEnvelope]] { +private[akka] class SystemMessageAcker(inboundContext: InboundContext) extends GraphStage[FlowShape[InboundEnvelope, InboundEnvelope]] { import SystemMessageDelivery._ val in: Inlet[InboundEnvelope] = Inlet("SystemMessageAcker.in") @@ -207,20 +203,22 @@ private[akka] class SystemMessageAcker(localAddress: Address) extends GraphStage var seqNo = 1L + def localAddress = inboundContext.localAddress + // InHandler override def onPush(): Unit = { grab(in) match { case env @ InboundEnvelope(_, _, sysEnv @ SystemMessageEnvelope(_, n, ackReplyTo), _) ⇒ if (n == seqNo) { - ackReplyTo.tell(Ack(n, localAddress), ActorRef.noSender) + inboundContext.sendReply(ackReplyTo.address, Ack(n, localAddress)) seqNo += 1 val unwrapped = env.copy(message = sysEnv.message) push(out, unwrapped) } else if (n < seqNo) { - ackReplyTo.tell(Ack(n, localAddress), ActorRef.noSender) + inboundContext.sendReply(ackReplyTo.address, Ack(n, localAddress)) pull(in) } else { - ackReplyTo.tell(Nack(seqNo - 1, localAddress), ActorRef.noSender) + inboundContext.sendReply(ackReplyTo.address, Nack(seqNo - 1, localAddress)) pull(in) } case env ⇒ @@ -237,74 +235,3 @@ private[akka] class SystemMessageAcker(localAddress: Address) extends GraphStage } } -/** - * INTERNAL API - */ -private[akka] object SystemMessageReplyJunction { - import SystemMessageDelivery._ - - trait Junction { - def addReplyInterest(filter: InboundEnvelope ⇒ Boolean, replyCallback: AsyncCallback[SystemMessageReply]): Future[Done] - def removeReplyInterest(callback: AsyncCallback[SystemMessageReply]): Unit - def stopped: Future[Done] - } -} - -/** - * INTERNAL API - */ -private[akka] class SystemMessageReplyJunction - extends GraphStageWithMaterializedValue[FlowShape[InboundEnvelope, InboundEnvelope], SystemMessageReplyJunction.Junction] { - import SystemMessageReplyJunction._ - import SystemMessageDelivery._ - - val in: Inlet[InboundEnvelope] = Inlet("SystemMessageReplyJunction.in") - val out: Outlet[InboundEnvelope] = Outlet("SystemMessageReplyJunction.out") - override val shape: FlowShape[InboundEnvelope, InboundEnvelope] = FlowShape(in, out) - - override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = { - val logic = new GraphStageLogic(shape) with InHandler with OutHandler with Junction { - - private var replyHandlers: Vector[(InboundEnvelope ⇒ Boolean, AsyncCallback[SystemMessageReply])] = Vector.empty - private val stoppedPromise = Promise[Done]() - - override def postStop(): Unit = stoppedPromise.success(Done) - - // InHandler - override def onPush(): Unit = { - grab(in) match { - case env @ InboundEnvelope(_, _, reply: SystemMessageReply, _) ⇒ - replyHandlers.foreach { - case (f, callback) ⇒ - if (f(env)) - callback.invoke(reply) - } - pull(in) - case env ⇒ - push(out, env) - } - } - - // OutHandler - override def onPull(): Unit = pull(in) - - override def addReplyInterest(filter: InboundEnvelope ⇒ Boolean, replyCallback: AsyncCallback[SystemMessageReply]): Future[Done] = { - val p = Promise[Done]() - getAsyncCallback[Unit](_ ⇒ { - replyHandlers :+= (filter -> replyCallback) - p.success(Done) - }).invoke(()) - p.future - } - - override def removeReplyInterest(callback: AsyncCallback[SystemMessageReply]): Unit = { - replyHandlers = replyHandlers.filterNot { case (_, c) ⇒ c == callback } - } - - override def stopped: Future[Done] = stoppedPromise.future - - setHandlers(in, out, this) - } - (logic, logic) - } -} diff --git a/akka-remote/src/main/scala/akka/remote/artery/Transport.scala b/akka-remote/src/main/scala/akka/remote/artery/Transport.scala deleted file mode 100644 index fb9563def7..0000000000 --- a/akka-remote/src/main/scala/akka/remote/artery/Transport.scala +++ /dev/null @@ -1,281 +0,0 @@ -/** - * Copyright (C) 2009-2016 Lightbend Inc. - */ - -package akka.remote.artery - -import scala.concurrent.duration._ -import akka.actor.Props -import scala.concurrent.duration._ -import java.net.InetSocketAddress -import java.nio.ByteOrder -import akka.NotUsed -import akka.actor.{ Address, ExtendedActorSystem } -import akka.remote.EndpointManager.Send -import akka.remote.{ InboundMessageDispatcher, MessageSerializer, RemoteActorRefProvider } -import akka.remote.transport.AkkaPduCodec -import akka.serialization.Serialization -import akka.stream.{ KillSwitches, SharedKillSwitch } -import akka.stream.scaladsl.{ Flow, Framing, Sink, Source, Tcp } -import akka.util.{ ByteString, ByteStringBuilder } -import scala.concurrent.Future -import akka.Done -import akka.stream.Materializer -import scala.concurrent.Await -import akka.event.LoggingAdapter -import akka.event.Logging -import io.aeron.driver.MediaDriver -import io.aeron.Aeron -import org.agrona.ErrorHandler -import io.aeron.AvailableImageHandler -import io.aeron.Image -import io.aeron.UnavailableImageHandler -import io.aeron.exceptions.ConductorServiceTimeoutException -import akka.actor.LocalRef -import akka.actor.InternalActorRef -import akka.dispatch.sysmsg.SystemMessage -import akka.actor.PossiblyHarmful -import akka.actor.RepointableRef -import akka.actor.ActorSelectionMessage -import akka.remote.RemoteRef -import akka.actor.ActorSelection -import akka.actor.ActorRef -import akka.stream.scaladsl.Keep - -/** - * INTERNAL API - */ -private[akka] object Transport { - // FIXME avoid allocating this envelope? - final case class InboundEnvelope( - recipient: InternalActorRef, - recipientAddress: Address, - message: AnyRef, - senderOption: Option[ActorRef]) -} - -/** - * INTERNAL API - */ -// FIXME: Replace the codec with a custom made, hi-perf one -private[akka] class Transport( - val localAddress: Address, - val system: ExtendedActorSystem, - val materializer: Materializer, - val provider: RemoteActorRefProvider, - val codec: AkkaPduCodec) { - import Transport._ - - private val log: LoggingAdapter = Logging(system.eventStream, getClass.getName) - private val remoteDaemon = provider.remoteDaemon - - private implicit val mat = materializer - // TODO support port 0 - private val inboundChannel = s"aeron:udp?endpoint=${localAddress.host.get}:${localAddress.port.get}" - private def outboundChannel(a: Address) = s"aeron:udp?endpoint=${a.host.get}:${a.port.get}" - private val systemMessageStreamId = 1 - private val ordinaryStreamId = 3 - - private val systemMessageResendInterval: FiniteDuration = 1.second // FIXME config - - private var systemMessageReplyJunction: SystemMessageReplyJunction.Junction = _ - - // Need an ActorRef that is passed in the `SystemMessageEnvelope.ackReplyTo`. - // Those messages are not actually handled by this actor, but intercepted by the - // SystemMessageReplyJunction stage. - private val systemMessageReplyRecepient = system.systemActorOf(Props.empty, "systemMessageReplyTo") - - private val driver = { - // TODO also support external media driver - val driverContext = new MediaDriver.Context - // FIXME settings from config - driverContext.clientLivenessTimeoutNs(SECONDS.toNanos(10)) - driverContext.imageLivenessTimeoutNs(SECONDS.toNanos(10)) - driverContext.driverTimeoutMs(SECONDS.toNanos(10)) - MediaDriver.launchEmbedded(driverContext) - } - - private val aeron = { - val ctx = new Aeron.Context - - 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}") - // FIXME we should call FragmentAssembler.freeSessionBuffer when image is unavailable - } - }) - ctx.errorHandler(new ErrorHandler { - override def onError(cause: Throwable): Unit = { - cause match { - case e: ConductorServiceTimeoutException ⇒ - // Timeout between service calls - log.error(cause, s"Aeron ServiceTimeoutException, ${cause.getMessage}") - - case _ ⇒ - log.error(cause, s"Aeron error, ${cause.getMessage}") - } - } - }) - - ctx.aeronDirectoryName(driver.aeronDirectoryName) - Aeron.connect(ctx) - } - - private val taskRunner = new TaskRunner(system) - - def start(): Unit = { - taskRunner.start() - systemMessageReplyJunction = Source.fromGraph(new AeronSource(inboundChannel, systemMessageStreamId, aeron, taskRunner)) - .async // FIXME use dedicated dispatcher for AeronSource - .map(ByteString.apply) // TODO we should use ByteString all the way - .viaMat(inboundSystemMessageFlow)(Keep.right) - .to(Sink.ignore) - .run() - Source.fromGraph(new AeronSource(inboundChannel, ordinaryStreamId, aeron, taskRunner)) - .async // FIXME use dedicated dispatcher for AeronSource - .map(ByteString.apply) // TODO we should use ByteString all the way - .via(inboundFlow) - .runWith(Sink.ignore) - } - - def shutdown(): Future[Done] = { - // FIXME stop the AeronSource first? - taskRunner.stop() - aeron.close() - driver.close() - Future.successful(Done) - } - - val killSwitch: SharedKillSwitch = KillSwitches.shared("transportKillSwitch") - - def outbound(remoteAddress: Address): Sink[Send, Any] = { - Flow.fromGraph(killSwitch.flow[Send]) - .via(encoder) - .map(_.toArray) // TODO we should use ByteString all the way - .to(new AeronSink(outboundChannel(remoteAddress), ordinaryStreamId, aeron, taskRunner)) - } - - def outboundSystemMessage(remoteAddress: Address): Sink[Send, Any] = { - Flow.fromGraph(killSwitch.flow[Send]) - .via(new SystemMessageDelivery(systemMessageReplyJunction, systemMessageResendInterval, - localAddress, remoteAddress, systemMessageReplyRecepient)) - .via(encoder) - .map(_.toArray) // TODO we should use ByteString all the way - .to(new AeronSink(outboundChannel(remoteAddress), systemMessageStreamId, aeron, taskRunner)) - } - - // TODO: Try out parallelized serialization (mapAsync) for performance - val encoder: Flow[Send, ByteString, NotUsed] = Flow[Send].map { sendEnvelope ⇒ - val pdu: ByteString = codec.constructMessage( - sendEnvelope.recipient.localAddressToUse, - sendEnvelope.recipient, - Serialization.currentTransportInformation.withValue(Serialization.Information(localAddress, system)) { - MessageSerializer.serialize(system, sendEnvelope.message.asInstanceOf[AnyRef]) - }, - sendEnvelope.senderOption, - seqOption = None, // FIXME: Acknowledgements will be handled differently I just reused the old codec - ackOption = None) - - // TODO: Drop unserializable messages - // TODO: Drop oversized messages - (new ByteStringBuilder).putInt(pdu.size)(ByteOrder.LITTLE_ENDIAN).result() ++ pdu - } - - val decoder: Flow[ByteString, AkkaPduCodec.Message, NotUsed] = - Framing.lengthField(4, maximumFrameLength = 256000) - .map { frame ⇒ - // TODO: Drop unserializable messages - val pdu = codec.decodeMessage(frame.drop(4), provider, localAddress)._2.get - pdu - } - - val messageDispatcher: Sink[InboundEnvelope, Future[Done]] = Sink.foreach[InboundEnvelope] { m ⇒ - dispatchInboundMessage(m.recipient, m.recipientAddress, m.message, m.senderOption) - } - - val deserializer: Flow[AkkaPduCodec.Message, InboundEnvelope, NotUsed] = - Flow[AkkaPduCodec.Message].map { m ⇒ - InboundEnvelope( - m.recipient, - m.recipientAddress, - MessageSerializer.deserialize(system, m.serializedMessage), - m.senderOption) - } - - val inboundFlow: Flow[ByteString, ByteString, NotUsed] = { - Flow.fromSinkAndSource( - decoder.via(deserializer).to(messageDispatcher), - Source.maybe[ByteString].via(killSwitch.flow)) - } - - val inboundSystemMessageFlow: Flow[ByteString, ByteString, SystemMessageReplyJunction.Junction] = { - Flow.fromSinkAndSourceMat( - decoder.via(deserializer) - .via(new SystemMessageAcker(localAddress)) - .viaMat(new SystemMessageReplyJunction)(Keep.right) - .to(messageDispatcher), - Source.maybe[ByteString].via(killSwitch.flow))((a, b) ⇒ a) - } - - private def dispatchInboundMessage(recipient: InternalActorRef, - recipientAddress: Address, - message: AnyRef, - senderOption: Option[ActorRef]): Unit = { - - import provider.remoteSettings._ - - val sender: ActorRef = senderOption.getOrElse(system.deadLetters) - val originalReceiver = recipient.path - - def msgLog = s"RemoteMessage: [$message] to [$recipient]<+[$originalReceiver] from [$sender()]" - - recipient match { - - case `remoteDaemon` ⇒ - if (UntrustedMode) log.debug("dropping daemon message in untrusted mode") - else { - if (LogReceive) log.debug("received daemon message {}", msgLog) - remoteDaemon ! message - } - - case l @ (_: LocalRef | _: RepointableRef) if l.isLocal ⇒ - if (LogReceive) log.debug("received local message {}", msgLog) - message match { - case sel: ActorSelectionMessage ⇒ - if (UntrustedMode && (!TrustedSelectionPaths.contains(sel.elements.mkString("/", "/", "")) || - sel.msg.isInstanceOf[PossiblyHarmful] || l != provider.rootGuardian)) - 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 ⇒ - log.debug("operating in UntrustedMode, dropping inbound PossiblyHarmful message of type [{}]", msg.getClass.getName) - case msg: SystemMessage ⇒ l.sendSystemMessage(msg) - case msg ⇒ l.!(msg)(sender) - } - - case r @ (_: RemoteRef | _: RepointableRef) if !r.isLocal && !UntrustedMode ⇒ - if (LogReceive) log.debug("received remote-destined message {}", msgLog) - if (provider.transport.addresses(recipientAddress)) - // if it was originally addressed to us but is in fact remote from our point of view (i.e. remote-deployed) - r.!(message)(sender) - else - log.error("dropping message [{}] for non-local recipient [{}] arriving at [{}] inbound addresses are [{}]", - message.getClass, r, recipientAddress, provider.transport.addresses.mkString(", ")) - - case r ⇒ log.error("dropping message [{}] for unknown recipient [{}] arriving at [{}] inbound addresses are [{}]", - message.getClass, r, recipientAddress, provider.transport.addresses.mkString(", ")) - - } - } - -} diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index a95802d69b..8614aaf2b8 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -3,37 +3,32 @@ */ package akka.remote.artery -import scala.concurrent.Await -import scala.concurrent.Future -import scala.concurrent.Promise -import scala.concurrent.duration._ -import scala.concurrent.forkjoin.ThreadLocalRandom +import java.util.concurrent.ThreadLocalRandom + +import scala.concurrent.Await +import scala.concurrent.duration._ -import akka.Done import akka.NotUsed -import akka.actor.Actor import akka.actor.ActorIdentity import akka.actor.ActorRef import akka.actor.ActorSystem +import akka.actor.Address import akka.actor.ExtendedActorSystem import akka.actor.Identify import akka.actor.InternalActorRef import akka.actor.PoisonPill -import akka.actor.Props import akka.actor.RootActorPath -import akka.actor.Stash +import akka.remote.AddressUidExtension import akka.remote.EndpointManager.Send import akka.remote.RemoteActorRef +import akka.remote.UniqueAddress import akka.remote.artery.SystemMessageDelivery._ -import akka.remote.artery.Transport.InboundEnvelope 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.stage.AsyncCallback -import akka.stream.testkit.TestSubscriber import akka.stream.testkit.scaladsl.TestSink import akka.testkit.AkkaSpec import akka.testkit.ImplicitSender @@ -60,86 +55,60 @@ object SystemMessageDeliverySpec { val configB = ConfigFactory.parseString(s"akka.remote.artery.port = $portB") .withFallback(commonConfig) - class TestReplyJunction(sendCallbackTo: ActorRef) extends SystemMessageReplyJunction.Junction { + class ManualReplyInboundContext( + replyProbe: ActorRef, + localAddress: UniqueAddress, + replySubject: TestReplySubject) extends TestInboundContext(localAddress, replySubject) { - def addReplyInterest(filter: InboundEnvelope ⇒ Boolean, replyCallback: AsyncCallback[SystemMessageReply]): Future[Done] = { - sendCallbackTo ! replyCallback - Future.successful(Done) + private var lastReply: Option[(Address, ControlMessage)] = None + + override def sendReply(to: Address, message: ControlMessage) = { + lastReply = Some((to, message)) + replyProbe ! message } - override def removeReplyInterest(callback: AsyncCallback[SystemMessageReply]): Unit = () - - override def stopped: Future[Done] = Promise[Done]().future - } - - def replyConnectorProps(dropRate: Double): Props = - Props(new ReplyConnector(dropRate)) - - class ReplyConnector(dropRate: Double) extends Actor with Stash { - override def receive = { - case callback: AsyncCallback[SystemMessageReply] @unchecked ⇒ - context.become(active(callback)) - unstashAll() - case _ ⇒ stash() - } - - def active(callback: AsyncCallback[SystemMessageReply]): Receive = { - case reply: SystemMessageReply ⇒ - if (ThreadLocalRandom.current().nextDouble() >= dropRate) - callback.invoke(reply) + def deliverLastReply(): Unit = { + lastReply.foreach { case (to, message) ⇒ super.sendReply(to, message) } + lastReply = None } } - } class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commonConfig) with ImplicitSender { import SystemMessageDeliverySpec._ - val addressA = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + val addressA = UniqueAddress( + system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress, + AddressUidExtension(system).addressUid) val systemB = ActorSystem("systemB", configB) - val addressB = systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress - val rootB = RootActorPath(addressB) + val addressB = UniqueAddress( + systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress, + AddressUidExtension(systemB).addressUid) + val rootB = RootActorPath(addressB.address) val matSettings = ActorMaterializerSettings(system).withFuzzing(true) implicit val mat = ActorMaterializer(matSettings)(system) override def afterTermination(): Unit = shutdown(systemB) - def setupManualCallback(ackRecipient: ActorRef, resendInterval: FiniteDuration, - dropSeqNumbers: Vector[Long], sendCount: Int): (TestSubscriber.Probe[String], AsyncCallback[SystemMessageReply]) = { - val callbackProbe = TestProbe() - val replyJunction = new TestReplyJunction(callbackProbe.ref) - - val sink = - send(sendCount, resendInterval, replyJunction, ackRecipient) - .via(drop(dropSeqNumbers)) - .via(inbound) - .map(_.message.asInstanceOf[String]) - .runWith(TestSink.probe) - - val callback = callbackProbe.expectMsgType[AsyncCallback[SystemMessageReply]] - (sink, callback) - } - - def send(sendCount: Int, resendInterval: FiniteDuration, replyJunction: SystemMessageReplyJunction.Junction, - ackRecipient: ActorRef): Source[Send, NotUsed] = { + private def send(sendCount: Int, resendInterval: FiniteDuration, outboundContext: OutboundContext): Source[Send, NotUsed] = { val remoteRef = null.asInstanceOf[RemoteActorRef] // not used Source(1 to sendCount) .map(n ⇒ Send("msg-" + n, None, remoteRef, None)) - .via(new SystemMessageDelivery(replyJunction, resendInterval, addressA, addressB, ackRecipient)) + .via(new SystemMessageDelivery(outboundContext, resendInterval)) } - def inbound: Flow[Send, InboundEnvelope, NotUsed] = { + private def inbound(inboundContext: InboundContext): Flow[Send, InboundEnvelope, NotUsed] = { val recipient = null.asInstanceOf[InternalActorRef] // not used Flow[Send] .map { case Send(sysEnv: SystemMessageEnvelope, _, _, _) ⇒ - InboundEnvelope(recipient, addressB, sysEnv, None) + InboundEnvelope(recipient, addressB.address, sysEnv, None) } .async - .via(new SystemMessageAcker(addressB)) + .via(new SystemMessageAcker(inboundContext)) } - def drop(dropSeqNumbers: Vector[Long]): Flow[Send, Send, NotUsed] = { + private def drop(dropSeqNumbers: Vector[Long]): Flow[Send, Send, NotUsed] = { Flow[Send] .statefulMapConcat(() ⇒ { var dropping = dropSeqNumbers @@ -156,7 +125,7 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commo }) } - def randomDrop[T](dropRate: Double): Flow[T, T, NotUsed] = Flow[T].mapConcat { elem ⇒ + private def randomDrop[T](dropRate: Double): Flow[T, T, NotUsed] = Flow[T].mapConcat { elem ⇒ if (ThreadLocalRandom.current().nextDouble() < dropRate) Nil else List(elem) } @@ -177,83 +146,108 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commo } "be resent when some in the middle are lost" in { - val ackRecipient = TestProbe() - val (sink, replyCallback) = - setupManualCallback(ackRecipient.ref, resendInterval = 60.seconds, dropSeqNumbers = Vector(3L, 4L), sendCount = 5) + val replyProbe = TestProbe() + val replySubject = new TestReplySubject + val inboundContextB = new ManualReplyInboundContext(replyProbe.ref, addressB, replySubject) + val inboundContextA = new TestInboundContext(addressB, replySubject) + 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[String]) + .runWith(TestSink.probe) sink.request(100) sink.expectNext("msg-1") sink.expectNext("msg-2") - ackRecipient.expectMsg(Ack(1L, addressB)) - ackRecipient.expectMsg(Ack(2L, addressB)) + replyProbe.expectMsg(Ack(1L, addressB)) + replyProbe.expectMsg(Ack(2L, addressB)) // 3 and 4 was dropped - ackRecipient.expectMsg(Nack(2L, addressB)) + replyProbe.expectMsg(Nack(2L, addressB)) sink.expectNoMsg(100.millis) // 3 was dropped - replyCallback.invoke(Nack(2L, addressB)) + inboundContextB.deliverLastReply() // resending 3, 4, 5 sink.expectNext("msg-3") - ackRecipient.expectMsg(Ack(3L, addressB)) + replyProbe.expectMsg(Ack(3L, addressB)) sink.expectNext("msg-4") - ackRecipient.expectMsg(Ack(4L, addressB)) + replyProbe.expectMsg(Ack(4L, addressB)) sink.expectNext("msg-5") - ackRecipient.expectMsg(Ack(5L, addressB)) - ackRecipient.expectNoMsg(100.millis) - replyCallback.invoke(Ack(5L, addressB)) + replyProbe.expectMsg(Ack(5L, addressB)) + replyProbe.expectNoMsg(100.millis) + inboundContextB.deliverLastReply() sink.expectComplete() } "be resent when first is lost" in { - val ackRecipient = TestProbe() - val (sink, replyCallback) = - setupManualCallback(ackRecipient.ref, resendInterval = 60.seconds, dropSeqNumbers = Vector(1L), sendCount = 3) + val replyProbe = TestProbe() + val replySubject = new TestReplySubject + val inboundContextB = new ManualReplyInboundContext(replyProbe.ref, addressB, replySubject) + val inboundContextA = new TestInboundContext(addressB, replySubject) + 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[String]) + .runWith(TestSink.probe) sink.request(100) - ackRecipient.expectMsg(Nack(0L, addressB)) // from receiving 2 - ackRecipient.expectMsg(Nack(0L, addressB)) // from receiving 3 + replyProbe.expectMsg(Nack(0L, addressB)) // from receiving 2 + replyProbe.expectMsg(Nack(0L, addressB)) // from receiving 3 sink.expectNoMsg(100.millis) // 1 was dropped - replyCallback.invoke(Nack(0L, addressB)) - replyCallback.invoke(Nack(0L, addressB)) + inboundContextB.deliverLastReply() // it's ok to not delivery all nacks // resending 1, 2, 3 sink.expectNext("msg-1") - ackRecipient.expectMsg(Ack(1L, addressB)) + replyProbe.expectMsg(Ack(1L, addressB)) sink.expectNext("msg-2") - ackRecipient.expectMsg(Ack(2L, addressB)) + replyProbe.expectMsg(Ack(2L, addressB)) sink.expectNext("msg-3") - ackRecipient.expectMsg(Ack(3L, addressB)) - replyCallback.invoke(Ack(3L, addressB)) + replyProbe.expectMsg(Ack(3L, addressB)) + inboundContextB.deliverLastReply() sink.expectComplete() } "be resent when last is lost" in { - val ackRecipient = TestProbe() - val (sink, replyCallback) = - setupManualCallback(ackRecipient.ref, resendInterval = 1.second, dropSeqNumbers = Vector(3L), sendCount = 3) + val replyProbe = TestProbe() + val replySubject = new TestReplySubject + val inboundContextB = new ManualReplyInboundContext(replyProbe.ref, addressB, replySubject) + val inboundContextA = new TestInboundContext(addressB, replySubject) + val outboundContextA = inboundContextA.association(addressB.address) + + val sink = send(sendCount = 3, resendInterval = 1.seconds, outboundContextA) + .via(drop(dropSeqNumbers = Vector(3L))) + .via(inbound(inboundContextB)) + .map(_.message.asInstanceOf[String]) + .runWith(TestSink.probe) sink.request(100) sink.expectNext("msg-1") - ackRecipient.expectMsg(Ack(1L, addressB)) - replyCallback.invoke(Ack(1L, addressB)) + replyProbe.expectMsg(Ack(1L, addressB)) + inboundContextB.deliverLastReply() sink.expectNext("msg-2") - ackRecipient.expectMsg(Ack(2L, addressB)) - replyCallback.invoke(Ack(2L, addressB)) + replyProbe.expectMsg(Ack(2L, addressB)) + inboundContextB.deliverLastReply() sink.expectNoMsg(200.millis) // 3 was dropped // resending 3 due to timeout sink.expectNext("msg-3") - ackRecipient.expectMsg(Ack(3L, addressB)) - replyCallback.invoke(Ack(3L, addressB)) + replyProbe.expectMsg(Ack(3L, addressB)) + inboundContextB.deliverLastReply() sink.expectComplete() } "deliver all during stress and random dropping" in { val N = 10000 val dropRate = 0.1 - val replyConnector = system.actorOf(replyConnectorProps(dropRate)) - val replyJunction = new TestReplyJunction(replyConnector) + val replySubject = new TestReplySubject + val inboundContextB = new TestInboundContext(addressB, replySubject, replyDropRate = dropRate) + val inboundContextA = new TestInboundContext(addressB, replySubject) + val outboundContextA = inboundContextA.association(addressB.address) val output = - send(N, 1.second, replyJunction, replyConnector) + send(N, 1.second, outboundContextA) .via(randomDrop(dropRate)) - .via(inbound) + .via(inbound(inboundContextB)) .map(_.message.asInstanceOf[String]) .runWith(Sink.seq) @@ -263,14 +257,16 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commo "deliver all during throttling and random dropping" in { val N = 500 val dropRate = 0.1 - val replyConnector = system.actorOf(replyConnectorProps(dropRate)) - val replyJunction = new TestReplyJunction(replyConnector) + val replySubject = new TestReplySubject + val inboundContextB = new TestInboundContext(addressB, replySubject, replyDropRate = dropRate) + val inboundContextA = new TestInboundContext(addressB, replySubject) + val outboundContextA = inboundContextA.association(addressB.address) val output = - send(N, 1.second, replyJunction, replyConnector) + send(N, 1.second, outboundContextA) .throttle(200, 1.second, 10, ThrottleMode.shaping) .via(randomDrop(dropRate)) - .via(inbound) + .via(inbound(inboundContextB)) .map(_.message.asInstanceOf[String]) .runWith(Sink.seq) 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..d54cff909e --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -0,0 +1,79 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.remote.UniqueAddress +import akka.actor.Address +import scala.concurrent.Future +import akka.remote.artery.ReplyJunction.ReplySubject +import akka.remote.RemoteActorRef +import scala.concurrent.Promise +import akka.Done +import akka.remote.artery.ReplyJunction.ReplyObserver +import java.util.concurrent.CopyOnWriteArrayList +import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.ThreadLocalRandom + +private[akka] class TestInboundContext( + override val localAddress: UniqueAddress, + val replySubject: TestReplySubject = new TestReplySubject, + replyDropRate: Double = 0.0) extends InboundContext { + + private val associations = new ConcurrentHashMap[Address, OutboundContext] + + def sendReply(to: Address, message: ControlMessage) = { + if (ThreadLocalRandom.current().nextDouble() >= replyDropRate) + replySubject.sendReply(InboundEnvelope(null, to, message, None)) + } + + def association(remoteAddress: Address): OutboundContext = + associations.get(remoteAddress) match { + case null ⇒ + val a = new TestOutboundContext(localAddress, remoteAddress, replySubject) + associations.putIfAbsent(remoteAddress, a) match { + case null ⇒ a + case existing ⇒ existing + } + case existing ⇒ existing + } + + protected def createAssociation(remoteAddress: Address): OutboundContext = + new TestOutboundContext(localAddress, remoteAddress, replySubject) +} + +private[akka] class TestOutboundContext( + override val localAddress: UniqueAddress, + override val remoteAddress: Address, + override val replySubject: TestReplySubject) extends OutboundContext { + + private val _uniqueRemoteAddress = Promise[UniqueAddress]() + def uniqueRemoteAddress: Future[UniqueAddress] = _uniqueRemoteAddress.future + def completeRemoteAddress(a: UniqueAddress): Unit = _uniqueRemoteAddress.trySuccess(a) + + // FIXME we should be able to Send without a recipient ActorRef + def dummyRecipient: RemoteActorRef = null + +} + +private[akka] class TestReplySubject extends ReplySubject { + + private var replyObservers = new CopyOnWriteArrayList[ReplyObserver] + + override def attach(observer: ReplyObserver): Future[Done] = { + replyObservers.add(observer) + Future.successful(Done) + } + + override def detach(observer: ReplyObserver): Unit = { + replyObservers.remove(observer) + } + + override def stopped: Future[Done] = Promise[Done]().future + + def sendReply(env: InboundEnvelope): Unit = { + val iter = replyObservers.iterator() + while (iter.hasNext()) + iter.next().reply(env) + } +} From 87386e18cfa3e380478290b2bcf84ea46c28bb5d Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 11 May 2016 15:55:06 +0200 Subject: [PATCH 019/186] inject outgoing control messages at lower level * instead of sending the the control messages (e.g. handshake reply, sys msg ack) via the normal message send ingress point they are now injected via side channel and will therefore not go through higher level stages such as handshake and system message delivery --- .../akka/remote/artery/ArteryTransport.scala | 20 +-- .../akka/remote/artery/Association.scala | 25 +++- .../scala/akka/remote/artery/Handshake.scala | 2 +- .../main/scala/akka/remote/artery/Reply.scala | 127 +++++++++++++++--- .../remote/artery/SystemMessageDelivery.scala | 11 +- .../akka/remote/artery/TestContext.scala | 4 +- 6 files changed, 144 insertions(+), 45 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 6b822f9521..cdff0acd03 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -23,7 +23,7 @@ import akka.remote.RemoteActorRef import akka.remote.RemoteActorRefProvider import akka.remote.RemoteTransport import akka.remote.UniqueAddress -import akka.remote.artery.ReplyJunction.ReplySubject +import akka.remote.artery.InboundReplyJunction.ReplySubject import akka.remote.transport.AkkaPduCodec import akka.remote.transport.AkkaPduProtobufCodec import akka.serialization.Serialization @@ -47,6 +47,7 @@ import io.aeron.exceptions.ConductorServiceTimeoutException import org.agrona.ErrorHandler import org.agrona.IoUtil import java.io.File +import akka.remote.artery.OutboundReplyJunction.OutboundReplyIngress /** * INTERNAL API @@ -72,7 +73,7 @@ private[akka] trait InboundContext { * An inbound stage can send reply message to the origin * address with this method. */ - def sendReply(to: Address, message: ControlMessage): Unit + def sendReply(to: Address, message: ControlMessage): Unit // FIXME rename to sendControl /** * Lookup the outbound association for a given address. @@ -112,7 +113,7 @@ private[akka] trait OutboundContext { * An outbound stage can listen to reply messages * via this observer subject. */ - def replySubject: ReplySubject + def replySubject: ReplySubject // FIXME rename to controlSubject // FIXME we should be able to Send without a recipient ActorRef def dummyRecipient: RemoteActorRef @@ -146,7 +147,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // TODO support port 0 private def inboundChannel = s"aeron:udp?endpoint=${localAddress.address.host.get}:${localAddress.address.port.get}" private def outboundChannel(a: Address) = s"aeron:udp?endpoint=${a.host.get}:${a.port.get}" - private val systemMessageStreamId = 1 + private val systemMessageStreamId = 1 // FIXME rename to controlStreamId private val ordinaryStreamId = 3 private val taskRunner = new TaskRunner(system) @@ -241,9 +242,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } // InboundContext - override def sendReply(to: Address, message: ControlMessage) = { - send(message, None, association(to).dummyRecipient) - } + override def sendReply(to: Address, message: ControlMessage) = + association(to).outboundReplyIngress.sendControlMessage(message) override def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = { val cached = recipient.cachedAssociation @@ -282,10 +282,11 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .to(new AeronSink(outboundChannel(outboundContext.remoteAddress), ordinaryStreamId, aeron, taskRunner)) } - def outboundSystemMessage(outboundContext: OutboundContext): Sink[Send, Any] = { + def outboundSystemMessage(outboundContext: OutboundContext): Sink[Send, OutboundReplyIngress] = { Flow.fromGraph(killSwitch.flow[Send]) .via(new OutboundHandshake(outboundContext)) .via(new SystemMessageDelivery(outboundContext, systemMessageResendInterval)) + .viaMat(new OutboundReplyJunction(outboundContext))(Keep.right) .via(encoder) .map(_.toArray) // TODO we should use ByteString all the way .to(new AeronSink(outboundChannel(outboundContext.remoteAddress), systemMessageStreamId, aeron, taskRunner)) @@ -338,13 +339,14 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R Source.maybe[ByteString].via(killSwitch.flow)) } + // FIXME rename to controlFlow val inboundSystemMessageFlow: Flow[ByteString, ByteString, ReplySubject] = { Flow.fromSinkAndSourceMat( decoder .via(deserializer) .via(new InboundHandshake(this)) .via(new SystemMessageAcker(this)) - .viaMat(new ReplyJunction)(Keep.right) + .viaMat(new InboundReplyJunction)(Keep.right) .to(messageDispatcherSink), Source.maybe[ByteString].via(killSwitch.flow))((a, b) ⇒ a) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 1e6b8df019..1254fd2139 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -5,20 +5,20 @@ package akka.remote.artery import scala.concurrent.Future import scala.concurrent.Promise - import akka.actor.ActorRef import akka.actor.Address - import akka.actor.RootActorPath import akka.dispatch.sysmsg.SystemMessage import akka.remote.EndpointManager.Send import akka.remote.RemoteActorRef import akka.remote.UniqueAddress -import akka.remote.artery.ReplyJunction.ReplySubject +import akka.remote.artery.InboundReplyJunction.ReplySubject import akka.stream.Materializer import akka.stream.OverflowStrategy import akka.stream.scaladsl.Source import akka.stream.scaladsl.SourceQueueWithComplete +import akka.remote.artery.OutboundReplyJunction.OutboundReplyIngress +import akka.stream.scaladsl.Keep /** * INTERNAL API @@ -34,6 +34,13 @@ private[akka] class Association( @volatile private[this] var queue: SourceQueueWithComplete[Send] = _ @volatile private[this] var systemMessageQueue: SourceQueueWithComplete[Send] = _ + @volatile private[this] var _outboundReplyIngress: OutboundReplyIngress = _ + + def outboundReplyIngress: OutboundReplyIngress = { + if (_outboundReplyIngress eq null) + throw new IllegalStateException("outboundReplyIngress not initialized yet") + _outboundReplyIngress + } override def localAddress: UniqueAddress = transport.localAddress @@ -49,7 +56,7 @@ private[akka] class Association( // TODO: lookup subchannel // FIXME: Use a different envelope than the old Send, but make sure the new is handled by deadLetters properly message match { - case _: SystemMessage | _: Reply ⇒ + case _: SystemMessage ⇒ implicit val ec = materializer.executionContext systemMessageQueue.offer(Send(message, senderOption, recipient, None)).onFailure { case e ⇒ @@ -73,8 +80,12 @@ private[akka] class Association( if (queue eq null) queue = Source.queue(256, OverflowStrategy.dropBuffer) .to(transport.outbound(this)).run()(materializer) - if (systemMessageQueue eq null) - systemMessageQueue = Source.queue(256, OverflowStrategy.dropBuffer) - .to(transport.outboundSystemMessage(this)).run()(materializer) + if (systemMessageQueue eq null) { + val (q, control) = Source.queue(256, OverflowStrategy.dropBuffer) + .toMat(transport.outboundSystemMessage(this))(Keep.both) + .run()(materializer) + systemMessageQueue = q + _outboundReplyIngress = control + } } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala index 6590f82967..5a6cdeee65 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -8,7 +8,7 @@ import scala.concurrent.duration._ import akka.Done import akka.remote.EndpointManager.Send import akka.remote.UniqueAddress -import akka.remote.artery.ReplyJunction.ReplyObserver +import akka.remote.artery.InboundReplyJunction.ReplyObserver import akka.stream.Attributes import akka.stream.FlowShape import akka.stream.Inlet diff --git a/akka-remote/src/main/scala/akka/remote/artery/Reply.scala b/akka-remote/src/main/scala/akka/remote/artery/Reply.scala index 676425894e..baddb3e59a 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Reply.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Reply.scala @@ -14,6 +14,9 @@ import akka.stream.stage.GraphStageLogic import akka.stream.stage.GraphStageWithMaterializedValue import akka.stream.stage.InHandler import akka.stream.stage.OutHandler +import akka.remote.EndpointManager.Send +import java.util.ArrayDeque +import akka.stream.stage.CallbackWrapper /** * Marker trait for reply messages @@ -29,7 +32,9 @@ trait ControlMessage /** * INTERNAL API */ -private[akka] object ReplyJunction { +private[akka] object InboundReplyJunction { + + // FIXME rename all Reply stuff to Control or ControlMessage private[akka] trait ReplySubject { def attach(observer: ReplyObserver): Future[Done] @@ -40,24 +45,42 @@ private[akka] object ReplyJunction { private[akka] trait ReplyObserver { def reply(inboundEnvelope: InboundEnvelope): Unit } + + private[InboundReplyJunction] sealed trait CallbackMessage + private[InboundReplyJunction] final case class Attach(observer: ReplyObserver, done: Promise[Done]) + extends CallbackMessage + private[InboundReplyJunction] final case class Dettach(observer: ReplyObserver) extends CallbackMessage } /** * INTERNAL API */ -private[akka] class ReplyJunction - extends GraphStageWithMaterializedValue[FlowShape[InboundEnvelope, InboundEnvelope], ReplyJunction.ReplySubject] { - import ReplyJunction._ +private[akka] class InboundReplyJunction + extends GraphStageWithMaterializedValue[FlowShape[InboundEnvelope, InboundEnvelope], InboundReplyJunction.ReplySubject] { + import InboundReplyJunction._ - val in: Inlet[InboundEnvelope] = Inlet("ReplyJunction.in") - val out: Outlet[InboundEnvelope] = Outlet("ReplyJunction.out") + val in: Inlet[InboundEnvelope] = Inlet("InboundReplyJunction.in") + val out: Outlet[InboundEnvelope] = Outlet("InboundReplyJunction.out") override val shape: FlowShape[InboundEnvelope, InboundEnvelope] = FlowShape(in, out) override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = { - val logic = new GraphStageLogic(shape) with InHandler with OutHandler with ReplySubject { + 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 replyObservers: Vector[ReplyObserver] = Vector.empty - private val stoppedPromise = Promise[Done]() + + private val callback = getAsyncCallback[CallbackMessage] { + case Attach(observer, done) ⇒ + replyObservers :+= observer + done.success(Done) + case Dettach(observer) ⇒ + replyObservers = replyObservers.filterNot(_ == observer) + } + + override def preStart(): Unit = { + initCallback(callback.invoke) + } override def postStop(): Unit = stoppedPromise.success(Done) @@ -75,23 +98,95 @@ private[akka] class ReplyJunction // OutHandler override def onPull(): Unit = pull(in) + setHandlers(in, out, this) + } + + // materialized value + val replySubject: ReplySubject = new ReplySubject { override def attach(observer: ReplyObserver): Future[Done] = { val p = Promise[Done]() - getAsyncCallback[Unit](_ ⇒ { - replyObservers :+= observer - p.success(Done) - }).invoke(()) + logic.invoke(Attach(observer, p)) p.future } - override def detach(observer: ReplyObserver): Unit = { - replyObservers = replyObservers.filterNot(_ == observer) + override def detach(observer: ReplyObserver): Unit = + logic.invoke(Dettach(observer)) + + override def stopped: Future[Done] = + stoppedPromise.future + } + + (logic, replySubject) + } +} + +/** + * INTERNAL API + */ +private[akka] object OutboundReplyJunction { + trait OutboundReplyIngress { + def sendControlMessage(message: ControlMessage): Unit + } +} + +/** + * INTERNAL API + */ +private[akka] class OutboundReplyJunction(outboundContext: OutboundContext) + extends GraphStageWithMaterializedValue[FlowShape[Send, Send], OutboundReplyJunction.OutboundReplyIngress] { + import OutboundReplyJunction._ + val in: Inlet[Send] = Inlet("OutboundReplyJunction.in") + val out: Outlet[Send] = Outlet("OutboundReplyJunction.out") + override val shape: FlowShape[Send, Send] = 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 { + import OutboundReplyJunction._ + + private val sendControlMessageCallback = getAsyncCallback[ControlMessage](internalSendControlMessage) + private val buffer = new ArrayDeque[Send] + + override def preStart(): Unit = { + initCallback(sendControlMessageCallback.invoke) } - override def stopped: Future[Done] = stoppedPromise.future + // 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 + buffer.offer(wrap(message)) + } + + private def wrap(message: ControlMessage): Send = + Send(message, None, outboundContext.dummyRecipient, None) setHandlers(in, out, this) } - (logic, logic) + + // materialized value + val outboundReplyIngress = new OutboundReplyIngress { + override def sendControlMessage(message: ControlMessage): Unit = + logic.invoke(message) + } + + (logic, outboundReplyIngress) } + } diff --git a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala index 271f4a629a..f66c6ef2c3 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -14,7 +14,7 @@ import scala.util.Try import akka.Done import akka.remote.EndpointManager.Send import akka.remote.UniqueAddress -import akka.remote.artery.ReplyJunction.ReplyObserver +import akka.remote.artery.InboundReplyJunction.ReplyObserver import akka.stream.Attributes import akka.stream.FlowShape import akka.stream.Inlet @@ -151,15 +151,6 @@ private[akka] class SystemMessageDelivery( // InHandler override def onPush(): Unit = { grab(in) match { - case s @ Send(reply: ControlMessage, _, _, _) ⇒ - // pass through - if (isAvailable(out)) - push(out, s) - else { - // it's ok to drop the replies, but we can try - resending.offer(s) - } - case s @ Send(msg: AnyRef, _, _, _) ⇒ seqNo += 1 val sendMsg = s.copy(message = SystemMessageEnvelope(msg, seqNo, localAddress)) diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala index d54cff909e..a5694a276d 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -6,11 +6,11 @@ package akka.remote.artery import akka.remote.UniqueAddress import akka.actor.Address import scala.concurrent.Future -import akka.remote.artery.ReplyJunction.ReplySubject +import akka.remote.artery.InboundReplyJunction.ReplySubject import akka.remote.RemoteActorRef import scala.concurrent.Promise import akka.Done -import akka.remote.artery.ReplyJunction.ReplyObserver +import akka.remote.artery.InboundReplyJunction.ReplyObserver import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ThreadLocalRandom From 8a04b6d05a0ae71866e6ee259815e881b2a70bbb Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 12 May 2016 08:56:28 +0200 Subject: [PATCH 020/186] rename reply to control --- .../akka/remote/artery/ArteryTransport.scala | 37 +++++---- .../akka/remote/artery/Association.scala | 20 ++--- .../artery/{Reply.scala => Control.scala} | 80 +++++++++++-------- .../scala/akka/remote/artery/Handshake.scala | 24 +++--- .../remote/artery/SystemMessageDelivery.scala | 20 ++--- .../artery/SystemMessageDeliverySpec.scala | 36 ++++----- .../akka/remote/artery/TestContext.scala | 34 ++++---- 7 files changed, 130 insertions(+), 121 deletions(-) rename akka-remote/src/main/scala/akka/remote/artery/{Reply.scala => Control.scala} (63%) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index cdff0acd03..511ef38cc3 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -23,7 +23,7 @@ import akka.remote.RemoteActorRef import akka.remote.RemoteActorRefProvider import akka.remote.RemoteTransport import akka.remote.UniqueAddress -import akka.remote.artery.InboundReplyJunction.ReplySubject +import akka.remote.artery.InboundControlJunction.ControlMessageSubject import akka.remote.transport.AkkaPduCodec import akka.remote.transport.AkkaPduProtobufCodec import akka.serialization.Serialization @@ -47,7 +47,7 @@ import io.aeron.exceptions.ConductorServiceTimeoutException import org.agrona.ErrorHandler import org.agrona.IoUtil import java.io.File -import akka.remote.artery.OutboundReplyJunction.OutboundReplyIngress +import akka.remote.artery.OutboundControlJunction.OutboundControlIngress /** * INTERNAL API @@ -70,10 +70,10 @@ private[akka] trait InboundContext { def localAddress: UniqueAddress /** - * An inbound stage can send reply message to the origin + * An inbound stage can send control message, e.g. a reply, to the origin * address with this method. */ - def sendReply(to: Address, message: ControlMessage): Unit // FIXME rename to sendControl + def sendControl(to: Address, message: ControlMessage): Unit /** * Lookup the outbound association for a given address. @@ -110,10 +110,10 @@ private[akka] trait OutboundContext { def completeRemoteAddress(a: UniqueAddress): Unit /** - * An outbound stage can listen to reply messages + * An outbound stage can listen to control messages * via this observer subject. */ - def replySubject: ReplySubject // FIXME rename to controlSubject + def controlSubject: ControlMessageSubject // FIXME we should be able to Send without a recipient ActorRef def dummyRecipient: RemoteActorRef @@ -130,7 +130,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R @volatile private[this] var _localAddress: UniqueAddress = _ override def localAddress: UniqueAddress = _localAddress @volatile private[this] var materializer: Materializer = _ - @volatile private[this] var replySubject: ReplySubject = _ + @volatile private[this] var controlSubject: ControlMessageSubject = _ @volatile private[this] var messageDispatcher: MessageDispatcher = _ @volatile private[this] var driver: MediaDriver = _ @volatile private[this] var aeron: Aeron = _ @@ -147,7 +147,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // TODO support port 0 private def inboundChannel = s"aeron:udp?endpoint=${localAddress.address.host.get}:${localAddress.address.port.get}" private def outboundChannel(a: Address) = s"aeron:udp?endpoint=${a.host.get}:${a.port.get}" - private val systemMessageStreamId = 1 // FIXME rename to controlStreamId + private val controlStreamId = 1 private val ordinaryStreamId = 3 private val taskRunner = new TaskRunner(system) @@ -215,10 +215,10 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundFlows(): Unit = { - replySubject = Source.fromGraph(new AeronSource(inboundChannel, systemMessageStreamId, aeron, taskRunner)) + controlSubject = Source.fromGraph(new AeronSource(inboundChannel, controlStreamId, aeron, taskRunner)) .async // FIXME measure .map(ByteString.apply) // TODO we should use ByteString all the way - .viaMat(inboundSystemMessageFlow)(Keep.right) + .viaMat(inboundControlFlow)(Keep.right) .to(Sink.ignore) .run()(materializer) @@ -242,8 +242,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } // InboundContext - override def sendReply(to: Address, message: ControlMessage) = - association(to).outboundReplyIngress.sendControlMessage(message) + override def sendControl(to: Address, message: ControlMessage) = + association(to).outboundControlIngress.sendControlMessage(message) override def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = { val cached = recipient.cachedAssociation @@ -262,7 +262,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R else { associations.computeIfAbsent(remoteAddress, new JFunction[Address, Association] { override def apply(remoteAddress: Address): Association = { - val newAssociation = new Association(ArteryTransport.this, materializer, remoteAddress, replySubject) + val newAssociation = new Association(ArteryTransport.this, materializer, remoteAddress, controlSubject) newAssociation.associate() // This is a bit costly for this blocking method :( newAssociation } @@ -282,14 +282,14 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .to(new AeronSink(outboundChannel(outboundContext.remoteAddress), ordinaryStreamId, aeron, taskRunner)) } - def outboundSystemMessage(outboundContext: OutboundContext): Sink[Send, OutboundReplyIngress] = { + def outboundControl(outboundContext: OutboundContext): Sink[Send, OutboundControlIngress] = { Flow.fromGraph(killSwitch.flow[Send]) .via(new OutboundHandshake(outboundContext)) .via(new SystemMessageDelivery(outboundContext, systemMessageResendInterval)) - .viaMat(new OutboundReplyJunction(outboundContext))(Keep.right) + .viaMat(new OutboundControlJunction(outboundContext))(Keep.right) .via(encoder) .map(_.toArray) // TODO we should use ByteString all the way - .to(new AeronSink(outboundChannel(outboundContext.remoteAddress), systemMessageStreamId, aeron, taskRunner)) + .to(new AeronSink(outboundChannel(outboundContext.remoteAddress), controlStreamId, aeron, taskRunner)) } // TODO: Try out parallelized serialization (mapAsync) for performance @@ -339,14 +339,13 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R Source.maybe[ByteString].via(killSwitch.flow)) } - // FIXME rename to controlFlow - val inboundSystemMessageFlow: Flow[ByteString, ByteString, ReplySubject] = { + val inboundControlFlow: Flow[ByteString, ByteString, ControlMessageSubject] = { Flow.fromSinkAndSourceMat( decoder .via(deserializer) .via(new InboundHandshake(this)) .via(new SystemMessageAcker(this)) - .viaMat(new InboundReplyJunction)(Keep.right) + .viaMat(new InboundControlJunction)(Keep.right) .to(messageDispatcherSink), Source.maybe[ByteString].via(killSwitch.flow))((a, b) ⇒ a) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 1254fd2139..323aac7266 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -12,12 +12,12 @@ import akka.dispatch.sysmsg.SystemMessage import akka.remote.EndpointManager.Send import akka.remote.RemoteActorRef import akka.remote.UniqueAddress -import akka.remote.artery.InboundReplyJunction.ReplySubject +import akka.remote.artery.InboundControlJunction.ControlMessageSubject import akka.stream.Materializer import akka.stream.OverflowStrategy import akka.stream.scaladsl.Source import akka.stream.scaladsl.SourceQueueWithComplete -import akka.remote.artery.OutboundReplyJunction.OutboundReplyIngress +import akka.remote.artery.OutboundControlJunction.OutboundControlIngress import akka.stream.scaladsl.Keep /** @@ -30,16 +30,16 @@ private[akka] class Association( val transport: ArteryTransport, val materializer: Materializer, override val remoteAddress: Address, - override val replySubject: ReplySubject) extends OutboundContext { + override val controlSubject: ControlMessageSubject) extends OutboundContext { @volatile private[this] var queue: SourceQueueWithComplete[Send] = _ @volatile private[this] var systemMessageQueue: SourceQueueWithComplete[Send] = _ - @volatile private[this] var _outboundReplyIngress: OutboundReplyIngress = _ + @volatile private[this] var _outboundControlIngress: OutboundControlIngress = _ - def outboundReplyIngress: OutboundReplyIngress = { - if (_outboundReplyIngress eq null) - throw new IllegalStateException("outboundReplyIngress not initialized yet") - _outboundReplyIngress + def outboundControlIngress: OutboundControlIngress = { + if (_outboundControlIngress eq null) + throw new IllegalStateException("outboundControlIngress not initialized yet") + _outboundControlIngress } override def localAddress: UniqueAddress = transport.localAddress @@ -82,10 +82,10 @@ private[akka] class Association( .to(transport.outbound(this)).run()(materializer) if (systemMessageQueue eq null) { val (q, control) = Source.queue(256, OverflowStrategy.dropBuffer) - .toMat(transport.outboundSystemMessage(this))(Keep.both) + .toMat(transport.outboundControl(this))(Keep.both) .run()(materializer) systemMessageQueue = q - _outboundReplyIngress = control + _outboundControlIngress = control } } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Reply.scala b/akka-remote/src/main/scala/akka/remote/artery/Control.scala similarity index 63% rename from akka-remote/src/main/scala/akka/remote/artery/Reply.scala rename to akka-remote/src/main/scala/akka/remote/artery/Control.scala index baddb3e59a..56127d956c 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Reply.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Control.scala @@ -32,35 +32,45 @@ trait ControlMessage /** * INTERNAL API */ -private[akka] object InboundReplyJunction { +private[akka] object InboundControlJunction { - // FIXME rename all Reply stuff to Control or ControlMessage - - private[akka] trait ReplySubject { - def attach(observer: ReplyObserver): Future[Done] - def detach(observer: ReplyObserver): Unit + /** + * Observer subject for inbound control messages. + * Interested observers can attach themselves to the + * subject to get notification of incoming control + * messages. + */ + private[akka] trait ControlMessageSubject { + def attach(observer: ControlMessageObserver): Future[Done] + def detach(observer: ControlMessageObserver): Unit def stopped: Future[Done] } - private[akka] trait ReplyObserver { - def reply(inboundEnvelope: InboundEnvelope): Unit + private[akka] trait ControlMessageObserver { + + /** + * Notification of incoming control message. The message + * of the envelope is always a `ControlMessage`. + */ + def notify(inboundEnvelope: InboundEnvelope): Unit } - private[InboundReplyJunction] sealed trait CallbackMessage - private[InboundReplyJunction] final case class Attach(observer: ReplyObserver, done: Promise[Done]) + // messages for the CallbackWrapper + private[InboundControlJunction] sealed trait CallbackMessage + private[InboundControlJunction] final case class Attach(observer: ControlMessageObserver, done: Promise[Done]) extends CallbackMessage - private[InboundReplyJunction] final case class Dettach(observer: ReplyObserver) extends CallbackMessage + private[InboundControlJunction] final case class Dettach(observer: ControlMessageObserver) extends CallbackMessage } /** * INTERNAL API */ -private[akka] class InboundReplyJunction - extends GraphStageWithMaterializedValue[FlowShape[InboundEnvelope, InboundEnvelope], InboundReplyJunction.ReplySubject] { - import InboundReplyJunction._ +private[akka] class InboundControlJunction + extends GraphStageWithMaterializedValue[FlowShape[InboundEnvelope, InboundEnvelope], InboundControlJunction.ControlMessageSubject] { + import InboundControlJunction._ - val in: Inlet[InboundEnvelope] = Inlet("InboundReplyJunction.in") - val out: Outlet[InboundEnvelope] = Outlet("InboundReplyJunction.out") + 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) = { @@ -68,14 +78,14 @@ private[akka] class InboundReplyJunction // 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 replyObservers: Vector[ReplyObserver] = Vector.empty + private var observers: Vector[ControlMessageObserver] = Vector.empty private val callback = getAsyncCallback[CallbackMessage] { case Attach(observer, done) ⇒ - replyObservers :+= observer + observers :+= observer done.success(Done) case Dettach(observer) ⇒ - replyObservers = replyObservers.filterNot(_ == observer) + observers = observers.filterNot(_ == observer) } override def preStart(): Unit = { @@ -87,8 +97,8 @@ private[akka] class InboundReplyJunction // InHandler override def onPush(): Unit = { grab(in) match { - case env @ InboundEnvelope(_, _, reply: Reply, _) ⇒ - replyObservers.foreach(_.reply(env)) + case env @ InboundEnvelope(_, _, _: ControlMessage, _) ⇒ + observers.foreach(_.notify(env)) pull(in) case env ⇒ push(out, env) @@ -102,29 +112,29 @@ private[akka] class InboundReplyJunction } // materialized value - val replySubject: ReplySubject = new ReplySubject { - override def attach(observer: ReplyObserver): Future[Done] = { + 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: ReplyObserver): Unit = + override def detach(observer: ControlMessageObserver): Unit = logic.invoke(Dettach(observer)) override def stopped: Future[Done] = stoppedPromise.future } - (logic, replySubject) + (logic, controlSubject) } } /** * INTERNAL API */ -private[akka] object OutboundReplyJunction { - trait OutboundReplyIngress { +private[akka] object OutboundControlJunction { + trait OutboundControlIngress { def sendControlMessage(message: ControlMessage): Unit } } @@ -132,17 +142,17 @@ private[akka] object OutboundReplyJunction { /** * INTERNAL API */ -private[akka] class OutboundReplyJunction(outboundContext: OutboundContext) - extends GraphStageWithMaterializedValue[FlowShape[Send, Send], OutboundReplyJunction.OutboundReplyIngress] { - import OutboundReplyJunction._ - val in: Inlet[Send] = Inlet("OutboundReplyJunction.in") - val out: Outlet[Send] = Outlet("OutboundReplyJunction.out") +private[akka] class OutboundControlJunction(outboundContext: OutboundContext) + extends GraphStageWithMaterializedValue[FlowShape[Send, Send], OutboundControlJunction.OutboundControlIngress] { + import OutboundControlJunction._ + val in: Inlet[Send] = Inlet("OutboundControlJunction.in") + val out: Outlet[Send] = Outlet("OutboundControlJunction.out") override val shape: FlowShape[Send, Send] = 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 { - import OutboundReplyJunction._ + import OutboundControlJunction._ private val sendControlMessageCallback = getAsyncCallback[ControlMessage](internalSendControlMessage) private val buffer = new ArrayDeque[Send] @@ -181,12 +191,12 @@ private[akka] class OutboundReplyJunction(outboundContext: OutboundContext) } // materialized value - val outboundReplyIngress = new OutboundReplyIngress { + val outboundControlIngress = new OutboundControlIngress { override def sendControlMessage(message: ControlMessage): Unit = logic.invoke(message) } - (logic, outboundReplyIngress) + (logic, outboundControlIngress) } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala index 5a6cdeee65..5357b0ee45 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -8,7 +8,7 @@ import scala.concurrent.duration._ import akka.Done import akka.remote.EndpointManager.Send import akka.remote.UniqueAddress -import akka.remote.artery.InboundReplyJunction.ReplyObserver +import akka.remote.artery.InboundControlJunction.ControlMessageObserver import akka.stream.Attributes import akka.stream.FlowShape import akka.stream.Inlet @@ -29,7 +29,7 @@ private[akka] object OutboundHandshake { private sealed trait HandshakeState private case object Start extends HandshakeState - private case object ReplyObserverAttached extends HandshakeState + private case object ControlMessageObserverAttached extends HandshakeState private case object ReqInProgress extends HandshakeState private case object Completed extends HandshakeState @@ -46,7 +46,7 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext) extends override val shape: FlowShape[Send, Send] = FlowShape(in, out) override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = - new TimerGraphStageLogic(shape) with InHandler with OutHandler with ReplyObserver { + new TimerGraphStageLogic(shape) with InHandler with OutHandler with ControlMessageObserver { import OutboundHandshake._ private val timeout: FiniteDuration = 10.seconds // FIXME config @@ -59,13 +59,13 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext) extends handshakeState = Completed } else { implicit val ec = materializer.executionContext - outboundContext.replySubject.attach(this).foreach { + outboundContext.controlSubject.attach(this).foreach { getAsyncCallback[Done] { _ ⇒ if (handshakeState != Completed) { if (isAvailable(out)) pushHandshakeReq() else - handshakeState = ReplyObserverAttached + handshakeState = ControlMessageObserverAttached } }.invoke } @@ -85,7 +85,7 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext) extends } override def postStop(): Unit = { - outboundContext.replySubject.detach(this) + outboundContext.controlSubject.detach(this) } // InHandler @@ -99,9 +99,9 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext) extends override def onPull(): Unit = { handshakeState match { case Completed ⇒ pull(in) - case ReplyObserverAttached ⇒ + case ControlMessageObserverAttached ⇒ pushHandshakeReq() - case Start ⇒ // will push HandshakeReq when ReplyObserver is attached + case Start ⇒ // will push HandshakeReq when ControlMessageObserver is attached case ReqInProgress ⇒ // will pull when handshake reply is received } } @@ -115,7 +115,7 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext) extends private def handshakeCompleted(): Unit = { handshakeState = Completed cancelTimer(HandshakeTimeout) - outboundContext.replySubject.detach(this) + outboundContext.controlSubject.detach(this) } override protected def onTimer(timerKey: Any): Unit = @@ -125,8 +125,8 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext) extends s"Handshake with [$remoteAddress] did not complete within ${timeout.toMillis} ms")) } - // ReplyObserver, external call - override def reply(inboundEnvelope: InboundEnvelope): Unit = { + // ControlMessageObserver, external call + override def notify(inboundEnvelope: InboundEnvelope): Unit = { inboundEnvelope.message match { case rsp: HandshakeRsp ⇒ if (rsp.from.address == remoteAddress) { @@ -165,7 +165,7 @@ private[akka] class InboundHandshake(inboundContext: InboundContext) extends Gra grab(in) match { case InboundEnvelope(_, _, HandshakeReq(from), _) ⇒ inboundContext.association(from.address).completeRemoteAddress(from) - inboundContext.sendReply(from.address, HandshakeRsp(inboundContext.localAddress)) + inboundContext.sendControl(from.address, HandshakeRsp(inboundContext.localAddress)) pull(in) case other ⇒ push(out, other) diff --git a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala index f66c6ef2c3..21e62ca917 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -14,7 +14,7 @@ import scala.util.Try import akka.Done import akka.remote.EndpointManager.Send import akka.remote.UniqueAddress -import akka.remote.artery.InboundReplyJunction.ReplyObserver +import akka.remote.artery.InboundControlJunction.ControlMessageObserver import akka.stream.Attributes import akka.stream.FlowShape import akka.stream.Inlet @@ -53,7 +53,7 @@ private[akka] class SystemMessageDelivery( override val shape: FlowShape[Send, Send] = FlowShape(in, out) override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = - new TimerGraphStageLogic(shape) with InHandler with OutHandler with ReplyObserver { + 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 @@ -69,7 +69,7 @@ private[akka] class SystemMessageDelivery( this.schedulePeriodically(ResendTick, resendInterval) implicit val ec = materializer.executionContext - outboundContext.replySubject.attach(this).foreach { + outboundContext.controlSubject.attach(this).foreach { getAsyncCallback[Done] { _ ⇒ replyObserverAttached = true if (isAvailable(out)) @@ -77,7 +77,7 @@ private[akka] class SystemMessageDelivery( }.invoke } - outboundContext.replySubject.stopped.onComplete { + outboundContext.controlSubject.stopped.onComplete { getAsyncCallback[Try[Done]] { // FIXME quarantine case Success(_) ⇒ completeStage() @@ -87,7 +87,7 @@ private[akka] class SystemMessageDelivery( } override def postStop(): Unit = { - outboundContext.replySubject.detach(this) + outboundContext.controlSubject.detach(this) } override def onUpstreamFinish(): Unit = { @@ -106,8 +106,8 @@ private[akka] class SystemMessageDelivery( } } - // ReplyObserver, external call - override def reply(inboundEnvelope: InboundEnvelope): Unit = { + // 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) @@ -201,15 +201,15 @@ private[akka] class SystemMessageAcker(inboundContext: InboundContext) extends G grab(in) match { case env @ InboundEnvelope(_, _, sysEnv @ SystemMessageEnvelope(_, n, ackReplyTo), _) ⇒ if (n == seqNo) { - inboundContext.sendReply(ackReplyTo.address, Ack(n, localAddress)) + inboundContext.sendControl(ackReplyTo.address, Ack(n, localAddress)) seqNo += 1 val unwrapped = env.copy(message = sysEnv.message) push(out, unwrapped) } else if (n < seqNo) { - inboundContext.sendReply(ackReplyTo.address, Ack(n, localAddress)) + inboundContext.sendControl(ackReplyTo.address, Ack(n, localAddress)) pull(in) } else { - inboundContext.sendReply(ackReplyTo.address, Nack(seqNo - 1, localAddress)) + inboundContext.sendControl(ackReplyTo.address, Nack(seqNo - 1, localAddress)) pull(in) } case env ⇒ diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index 8614aaf2b8..6675ff3e9b 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -58,17 +58,17 @@ object SystemMessageDeliverySpec { class ManualReplyInboundContext( replyProbe: ActorRef, localAddress: UniqueAddress, - replySubject: TestReplySubject) extends TestInboundContext(localAddress, replySubject) { + controlSubject: TestControlMessageSubject) extends TestInboundContext(localAddress, controlSubject) { private var lastReply: Option[(Address, ControlMessage)] = None - override def sendReply(to: Address, message: ControlMessage) = { + override def sendControl(to: Address, message: ControlMessage) = { lastReply = Some((to, message)) replyProbe ! message } def deliverLastReply(): Unit = { - lastReply.foreach { case (to, message) ⇒ super.sendReply(to, message) } + lastReply.foreach { case (to, message) ⇒ super.sendControl(to, message) } lastReply = None } } @@ -147,9 +147,9 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commo "be resent when some in the middle are lost" in { val replyProbe = TestProbe() - val replySubject = new TestReplySubject - val inboundContextB = new ManualReplyInboundContext(replyProbe.ref, addressB, replySubject) - val inboundContextA = new TestInboundContext(addressB, replySubject) + 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) @@ -181,9 +181,9 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commo "be resent when first is lost" in { val replyProbe = TestProbe() - val replySubject = new TestReplySubject - val inboundContextB = new ManualReplyInboundContext(replyProbe.ref, addressB, replySubject) - val inboundContextA = new TestInboundContext(addressB, replySubject) + 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) @@ -210,9 +210,9 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commo "be resent when last is lost" in { val replyProbe = TestProbe() - val replySubject = new TestReplySubject - val inboundContextB = new ManualReplyInboundContext(replyProbe.ref, addressB, replySubject) - val inboundContextA = new TestInboundContext(addressB, replySubject) + 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 = 1.seconds, outboundContextA) @@ -239,9 +239,9 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commo "deliver all during stress and random dropping" in { val N = 10000 val dropRate = 0.1 - val replySubject = new TestReplySubject - val inboundContextB = new TestInboundContext(addressB, replySubject, replyDropRate = dropRate) - val inboundContextA = new TestInboundContext(addressB, replySubject) + 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 = @@ -257,9 +257,9 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commo "deliver all during throttling and random dropping" in { val N = 500 val dropRate = 0.1 - val replySubject = new TestReplySubject - val inboundContextB = new TestInboundContext(addressB, replySubject, replyDropRate = dropRate) - val inboundContextA = new TestInboundContext(addressB, replySubject) + 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 = diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala index a5694a276d..a9a68e7247 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -6,31 +6,31 @@ package akka.remote.artery import akka.remote.UniqueAddress import akka.actor.Address import scala.concurrent.Future -import akka.remote.artery.InboundReplyJunction.ReplySubject +import akka.remote.artery.InboundControlJunction.ControlMessageSubject import akka.remote.RemoteActorRef import scala.concurrent.Promise import akka.Done -import akka.remote.artery.InboundReplyJunction.ReplyObserver +import akka.remote.artery.InboundControlJunction.ControlMessageObserver import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ThreadLocalRandom private[akka] class TestInboundContext( override val localAddress: UniqueAddress, - val replySubject: TestReplySubject = new TestReplySubject, + val controlSubject: TestControlMessageSubject = new TestControlMessageSubject, replyDropRate: Double = 0.0) extends InboundContext { private val associations = new ConcurrentHashMap[Address, OutboundContext] - def sendReply(to: Address, message: ControlMessage) = { + def sendControl(to: Address, message: ControlMessage) = { if (ThreadLocalRandom.current().nextDouble() >= replyDropRate) - replySubject.sendReply(InboundEnvelope(null, to, message, None)) + controlSubject.sendControl(InboundEnvelope(null, to, message, None)) } def association(remoteAddress: Address): OutboundContext = associations.get(remoteAddress) match { case null ⇒ - val a = new TestOutboundContext(localAddress, remoteAddress, replySubject) + val a = new TestOutboundContext(localAddress, remoteAddress, controlSubject) associations.putIfAbsent(remoteAddress, a) match { case null ⇒ a case existing ⇒ existing @@ -39,13 +39,13 @@ private[akka] class TestInboundContext( } protected def createAssociation(remoteAddress: Address): OutboundContext = - new TestOutboundContext(localAddress, remoteAddress, replySubject) + new TestOutboundContext(localAddress, remoteAddress, controlSubject) } private[akka] class TestOutboundContext( override val localAddress: UniqueAddress, override val remoteAddress: Address, - override val replySubject: TestReplySubject) extends OutboundContext { + override val controlSubject: TestControlMessageSubject) extends OutboundContext { private val _uniqueRemoteAddress = Promise[UniqueAddress]() def uniqueRemoteAddress: Future[UniqueAddress] = _uniqueRemoteAddress.future @@ -56,24 +56,24 @@ private[akka] class TestOutboundContext( } -private[akka] class TestReplySubject extends ReplySubject { +private[akka] class TestControlMessageSubject extends ControlMessageSubject { - private var replyObservers = new CopyOnWriteArrayList[ReplyObserver] + private var observers = new CopyOnWriteArrayList[ControlMessageObserver] - override def attach(observer: ReplyObserver): Future[Done] = { - replyObservers.add(observer) + override def attach(observer: ControlMessageObserver): Future[Done] = { + observers.add(observer) Future.successful(Done) } - override def detach(observer: ReplyObserver): Unit = { - replyObservers.remove(observer) + override def detach(observer: ControlMessageObserver): Unit = { + observers.remove(observer) } override def stopped: Future[Done] = Promise[Done]().future - def sendReply(env: InboundEnvelope): Unit = { - val iter = replyObservers.iterator() + def sendControl(env: InboundEnvelope): Unit = { + val iter = observers.iterator() while (iter.hasNext()) - iter.next().reply(env) + iter.next().notify(env) } } From 64b10071146b3699f43d7cc3fb3cbb14bf24a7bc Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 12 May 2016 11:42:09 +0200 Subject: [PATCH 021/186] add unit tests for the handshake and control stages, #20313 --- .../akka/remote/artery/ArteryTransport.scala | 9 +- .../scala/akka/remote/artery/Handshake.scala | 3 +- .../artery/InboundControlJunctionSpec.scala | 73 +++++++++++++ .../remote/artery/InboundHandshakeSpec.scala | 67 ++++++++++++ .../artery/OutboundControlJunctionSpec.scala | 70 ++++++++++++ .../remote/artery/OutboundHandshakeSpec.scala | 103 ++++++++++++++++++ .../artery/SystemMessageDeliverySpec.scala | 18 --- .../akka/remote/artery/TestContext.scala | 20 ++++ 8 files changed, 340 insertions(+), 23 deletions(-) create mode 100644 akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 511ef38cc3..fe3776a3a3 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -142,7 +142,10 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val codec: AkkaPduCodec = AkkaPduProtobufCodec private val killSwitch: SharedKillSwitch = KillSwitches.shared("transportKillSwitch") - private val systemMessageResendInterval: FiniteDuration = 1.second // FIXME config + + // FIXME config + private val systemMessageResendInterval: FiniteDuration = 1.second + private val handshakeTimeout: FiniteDuration = 10.seconds // TODO support port 0 private def inboundChannel = s"aeron:udp?endpoint=${localAddress.address.host.get}:${localAddress.address.port.get}" @@ -276,7 +279,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R def outbound(outboundContext: OutboundContext): Sink[Send, Any] = { Flow.fromGraph(killSwitch.flow[Send]) - .via(new OutboundHandshake(outboundContext)) + .via(new OutboundHandshake(outboundContext, handshakeTimeout)) .via(encoder) .map(_.toArray) // TODO we should use ByteString all the way .to(new AeronSink(outboundChannel(outboundContext.remoteAddress), ordinaryStreamId, aeron, taskRunner)) @@ -284,7 +287,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R def outboundControl(outboundContext: OutboundContext): Sink[Send, OutboundControlIngress] = { Flow.fromGraph(killSwitch.flow[Send]) - .via(new OutboundHandshake(outboundContext)) + .via(new OutboundHandshake(outboundContext, handshakeTimeout)) .via(new SystemMessageDelivery(outboundContext, systemMessageResendInterval)) .viaMat(new OutboundControlJunction(outboundContext))(Keep.right) .via(encoder) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala index 5357b0ee45..92793ff6e6 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -40,7 +40,7 @@ private[akka] object OutboundHandshake { /** * INTERNAL API */ -private[akka] class OutboundHandshake(outboundContext: OutboundContext) extends GraphStage[FlowShape[Send, Send]] { +private[akka] class OutboundHandshake(outboundContext: OutboundContext, timeout: FiniteDuration) extends GraphStage[FlowShape[Send, Send]] { val in: Inlet[Send] = Inlet("OutboundHandshake.in") val out: Outlet[Send] = Outlet("OutboundHandshake.out") override val shape: FlowShape[Send, Send] = FlowShape(in, out) @@ -49,7 +49,6 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext) extends new TimerGraphStageLogic(shape) with InHandler with OutHandler with ControlMessageObserver { import OutboundHandshake._ - private val timeout: FiniteDuration = 10.seconds // FIXME config private var handshakeState: HandshakeState = Start private def remoteAddress = outboundContext.remoteAddress 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..4b67be4274 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala @@ -0,0 +1,73 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.duration._ + +import akka.actor.Address +import akka.actor.InternalActorRef +import akka.remote.UniqueAddress +import akka.remote.artery.InboundControlJunction.ControlMessageObserver +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.testkit.TestProbe + +object InboundControlJunctionSpec { + case object Control1 extends ControlMessage + case object Control2 extends ControlMessage + case object Control3 extends ControlMessage +} + +class InboundControlJunctionSpec extends AkkaSpec with ImplicitSender { + import InboundControlJunctionSpec._ + + val matSettings = ActorMaterializerSettings(system).withFuzzing(true) + implicit val mat = ActorMaterializer(matSettings)(system) + + val addressA = UniqueAddress(Address("akka.artery", "sysA", "hostA", 1001), 1) + val addressB = UniqueAddress(Address("akka.artery", "sysB", "hostB", 1002), 2) + + "Control messages" must { + + "be emitted via side channel" in { + val observerProbe = TestProbe() + val inboundContext = new TestInboundContext(localAddress = addressB) + val recipient = null.asInstanceOf[InternalActorRef] // not used + + val ((upstream, controlSubject), downstream) = TestSource.probe[AnyRef] + .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, None)) + .viaMat(new InboundControlJunction)(Keep.both) + .map { case InboundEnvelope(_, _, msg, _) ⇒ msg } + .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..f2b378efc5 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala @@ -0,0 +1,67 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.duration._ + +import akka.actor.Address +import akka.actor.InternalActorRef +import akka.remote.UniqueAddress +import akka.remote.artery.OutboundHandshake.HandshakeReq +import akka.remote.artery.OutboundHandshake.HandshakeRsp +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 + +object InboundHandshakeSpec { + case object Control1 extends ControlMessage + case object Control2 extends ControlMessage + case object Control3 extends ControlMessage +} + +class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { + import InboundHandshakeSpec._ + + val matSettings = ActorMaterializerSettings(system).withFuzzing(true) + implicit val mat = ActorMaterializer(matSettings)(system) + + val addressA = UniqueAddress(Address("akka.artery", "sysA", "hostA", 1001), 1) + val addressB = UniqueAddress(Address("akka.artery", "sysB", "hostB", 1002), 2) + + private def setupStream(inboundContext: InboundContext, timeout: FiniteDuration = 5.seconds): (TestPublisher.Probe[AnyRef], TestSubscriber.Probe[Any]) = { + val recipient = null.asInstanceOf[InternalActorRef] // not used + TestSource.probe[AnyRef] + .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, None)) + .via(new InboundHandshake(inboundContext)) + .map { case InboundEnvelope(_, _, msg, _) ⇒ msg } + .toMat(TestSink.probe[Any])(Keep.both) + .run() + } + + "InboundHandshake stage" must { + + "send HandshakeRsp as reply to HandshakeReq" in { + val replyProbe = TestProbe() + val inboundContext = new ManualReplyInboundContext(replyProbe.ref, addressB, new TestControlMessageSubject) + val (upstream, downstream) = setupStream(inboundContext) + + downstream.request(10) + upstream.sendNext(HandshakeReq(addressA)) + upstream.sendNext("msg1") + replyProbe.expectMsg(HandshakeRsp(addressB)) + downstream.expectNext("msg1") + downstream.cancel() + } + + } + +} 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..ab1eb63f0a --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala @@ -0,0 +1,70 @@ +/** + * 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 + +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.artery", "sysA", "hostA", 1001), 1) + val addressB = UniqueAddress(Address("akka.artery", "sysB", "hostB", 1002), 2) + + "Control messages" must { + + "be injected via side channel" in { + val inboundContext = new TestInboundContext(localAddress = addressA) + val outboundContext = inboundContext.association(addressB.address) + val destination = null.asInstanceOf[RemoteActorRef] // not used + + val ((upstream, controlIngress), downstream) = TestSource.probe[String] + .map(msg ⇒ Send(msg, None, destination, None)) + .viaMat(new OutboundControlJunction(outboundContext))(Keep.both) + .map { case Send(msg, _, _, _) ⇒ msg } + .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..a75c8576bf --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala @@ -0,0 +1,103 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.concurrent.TimeoutException + +import scala.concurrent.duration._ + +import akka.actor.Address +import akka.actor.InternalActorRef +import akka.remote.EndpointManager.Send +import akka.remote.RemoteActorRef +import akka.remote.UniqueAddress +import akka.remote.artery.OutboundHandshake.HandshakeReq +import akka.remote.artery.OutboundHandshake.HandshakeRsp +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 + +class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { + + val matSettings = ActorMaterializerSettings(system).withFuzzing(true) + implicit val mat = ActorMaterializer(matSettings)(system) + + val addressA = UniqueAddress(Address("akka.artery", "sysA", "hostA", 1001), 1) + val addressB = UniqueAddress(Address("akka.artery", "sysB", "hostB", 1002), 2) + + private def setupStream(outboundContext: OutboundContext, timeout: FiniteDuration = 5.seconds): (TestPublisher.Probe[String], TestSubscriber.Probe[Any]) = { + val destination = null.asInstanceOf[RemoteActorRef] // not used + TestSource.probe[String] + .map(msg ⇒ Send(msg, None, destination, None)) + .via(new OutboundHandshake(outboundContext, timeout)) + .map { case Send(msg, _, _, _) ⇒ msg } + .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)) + downstream.cancel() + } + + "timeout if not receiving HandshakeRsp" 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)) + downstream.expectError().getClass should be(classOf[TimeoutException]) + } + + "not deliver messages from upstream until handshake completed" in { + val controlSubject = new TestControlMessageSubject + val inboundContext = new TestInboundContext(localAddress = addressA, controlSubject) + val outboundContext = inboundContext.association(addressB.address) + val recipient = null.asInstanceOf[InternalActorRef] // not used + val (upstream, downstream) = setupStream(outboundContext) + + downstream.request(10) + downstream.expectNext(HandshakeReq(addressA)) + upstream.sendNext("msg1") + downstream.expectNoMsg(200.millis) + controlSubject.sendControl(InboundEnvelope(recipient, addressA.address, HandshakeRsp(addressB), None)) + downstream.expectNext("msg1") + upstream.sendNext("msg2") + downstream.expectNext("msg2") + downstream.cancel() + } + + "complete handshake via another sub-channel" 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)) + upstream.sendNext("msg1") + // handshake completed first by another sub-channel + outboundContext.completeRemoteAddress(addressB) + downstream.expectNext("msg1") + upstream.sendNext("msg2") + downstream.expectNext("msg2") + 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 index 6675ff3e9b..6e24432483 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -54,24 +54,6 @@ object SystemMessageDeliverySpec { val configB = ConfigFactory.parseString(s"akka.remote.artery.port = $portB") .withFallback(commonConfig) - - 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) = { - lastReply = Some((to, message)) - replyProbe ! message - } - - def deliverLastReply(): Unit = { - lastReply.foreach { case (to, message) ⇒ super.sendControl(to, message) } - lastReply = None - } - } } class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commonConfig) with ImplicitSender { diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala index a9a68e7247..0db4672139 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -14,6 +14,7 @@ import akka.remote.artery.InboundControlJunction.ControlMessageObserver import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ThreadLocalRandom +import akka.actor.ActorRef private[akka] class TestInboundContext( override val localAddress: UniqueAddress, @@ -76,4 +77,23 @@ private[akka] class TestControlMessageSubject extends ControlMessageSubject { while (iter.hasNext()) iter.next().notify(env) } + +} + +private[akka] 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) = { + lastReply = Some((to, message)) + replyProbe ! message + } + + def deliverLastReply(): Unit = { + lastReply.foreach { case (to, message) ⇒ super.sendControl(to, message) } + lastReply = None + } } From 5e3eb4bd8c0a4d20ed2857e3af06a748086af999 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Tue, 17 May 2016 14:17:21 +0200 Subject: [PATCH 022/186] Auto port selection and SunnyWeatherSpec for Artery (#20512) * Automatic port selection when port 0 configured * Combine remoting and artery SunnyWeatherSpec * Default to port 0 for artery in MultiNodeSpec.nodeConfig --- .../scala/akka/cluster/SunnyWeatherSpec.scala | 57 +++++++++++++------ .../akka/remote/testkit/MultiNodeSpec.scala | 3 +- .../artery/AeronStreamConcistencySpec.scala | 8 +-- .../artery/AeronStreamLatencySpec.scala | 8 +-- .../artery/AeronStreamMaxThroughputSpec.scala | 8 +-- .../artery/HandshakeRestartReceiverSpec.scala | 18 ------ .../akka/remote/artery/LatencySpec.scala | 20 +------ .../remote/artery/MaxThroughputSpec.scala | 18 ------ .../akka/remote/artery/ArteryTransport.scala | 27 ++++++++- 9 files changed, 72 insertions(+), 95 deletions(-) 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..c9e0bb7104 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala @@ -14,35 +14,60 @@ import scala.collection.immutable.SortedSet import akka.actor.Props import akka.actor.Actor -object SunnyWeatherMultiJvmSpec extends MultiNodeConfig { +class SunnyWeatherMultiJvmSpec(artery: Boolean) extends MultiNodeConfig { val first = role("first") val second = role("second") val third = role("third") val fourth = role("fourth") val fifth = role("fifth") + val common = + """ + akka { + actor.provider = akka.cluster.ClusterActorRefProvider + loggers = ["akka.testkit.TestEventListener"] + loglevel = INFO + remote.log-remote-lifecycle-events = off + cluster.failure-detector.monitored-by-nr-of-members = 3 + } + """ + + val arteryConfig = + """ + akka.remote.artery { + enabled = on + } + """ + // 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( + if (artery) ConfigFactory.parseString(arteryConfig).withFallback(ConfigFactory.parseString(common)) + else ConfigFactory.parseString(common)) + } -class SunnyWeatherMultiJvmNode1 extends SunnyWeatherSpec -class SunnyWeatherMultiJvmNode2 extends SunnyWeatherSpec -class SunnyWeatherMultiJvmNode3 extends SunnyWeatherSpec -class SunnyWeatherMultiJvmNode4 extends SunnyWeatherSpec -class SunnyWeatherMultiJvmNode5 extends SunnyWeatherSpec +class SunnyWeatherRemotingMultiJvmNode1 extends SunnyWeatherRemotingSpec +class SunnyWeatherRemotingMultiJvmNode2 extends SunnyWeatherRemotingSpec +class SunnyWeatherRemotingMultiJvmNode3 extends SunnyWeatherRemotingSpec +class SunnyWeatherRemotingMultiJvmNode4 extends SunnyWeatherRemotingSpec +class SunnyWeatherRemotingMultiJvmNode5 extends SunnyWeatherRemotingSpec -abstract class SunnyWeatherSpec - extends MultiNodeSpec(SunnyWeatherMultiJvmSpec) +class SunnyWeatherArteryMultiJvmNode1 extends SunnyWeatherArterySpec +class SunnyWeatherArteryMultiJvmNode2 extends SunnyWeatherArterySpec +class SunnyWeatherArteryMultiJvmNode3 extends SunnyWeatherArterySpec +class SunnyWeatherArteryMultiJvmNode4 extends SunnyWeatherArterySpec +class SunnyWeatherArteryMultiJvmNode5 extends SunnyWeatherArterySpec + +abstract class SunnyWeatherRemotingSpec extends SunnyWeatherSpec(new SunnyWeatherMultiJvmSpec(artery = false)) +abstract class SunnyWeatherArterySpec extends SunnyWeatherSpec(new SunnyWeatherMultiJvmSpec(artery = true)) + +abstract class SunnyWeatherSpec(multiNodeConfig: SunnyWeatherMultiJvmSpec) + extends MultiNodeSpec(multiNodeConfig) with MultiNodeClusterSpec { - import SunnyWeatherMultiJvmSpec._ + import multiNodeConfig._ + import ClusterEvent._ "A normal cluster" must { 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 b8969bfc6a..5d9d3ae6ee 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 @@ -198,7 +198,8 @@ object MultiNodeSpec { "akka.actor.provider" -> "akka.remote.RemoteActorRefProvider", "akka.remote.artery.hostname" -> selfName, "akka.remote.netty.tcp.hostname" -> selfName, - "akka.remote.netty.tcp.port" -> selfPort)) + "akka.remote.netty.tcp.port" -> selfPort, + "akka.remote.artery.port" -> selfPort)) private[testkit] val baseConfig: Config = ConfigFactory.parseString(""" akka { 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 index 429a3569cd..c0edac7986 100644 --- 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 @@ -40,12 +40,6 @@ object AeronStreamConsistencySpec extends MultiNodeConfig { } """))) - def aeronPort(roleName: RoleName): Int = - roleName match { - case `first` ⇒ 20521 // TODO yeah, we should have support for dynamic port assignment - case `second` ⇒ 20522 - } - } class AeronStreamConsistencySpecMultiJvmNode1 extends AeronStreamConsistencySpec @@ -78,7 +72,7 @@ abstract class AeronStreamConsistencySpec def channel(roleName: RoleName) = { val a = node(roleName).address - s"aeron:udp?endpoint=${a.host.get}:${aeronPort(roleName)}" + s"aeron:udp?endpoint=${a.host.get}:${a.port.get}" } val streamId = 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 index d8cb7f34fb..d364a52cae 100644 --- 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 @@ -52,12 +52,6 @@ object AeronStreamLatencySpec extends MultiNodeConfig { } """))) - def aeronPort(roleName: RoleName): Int = - roleName match { - case `first` ⇒ 20521 // TODO yeah, we should have support for dynamic port assignment - case `second` ⇒ 20522 - } - final case class TestSettings( testName: String, messageRate: Int, // msg/s @@ -104,7 +98,7 @@ abstract class AeronStreamLatencySpec def channel(roleName: RoleName) = { val a = node(roleName).address - s"aeron:udp?endpoint=${a.host.get}:${aeronPort(roleName)}" + s"aeron:udp?endpoint=${a.host.get}:${a.port.get}" } val streamId = 1 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 index 042ed8fc90..e9356cad08 100644 --- 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 @@ -46,12 +46,6 @@ object AeronStreamMaxThroughputSpec extends MultiNodeConfig { } """))) - def aeronPort(roleName: RoleName): Int = - roleName match { - case `first` ⇒ 20511 // TODO yeah, we should have support for dynamic port assignment - case `second` ⇒ 20512 - } - final case class TestSettings( testName: String, totalMessages: Long, @@ -111,7 +105,7 @@ abstract class AeronStreamMaxThroughputSpec def channel(roleName: RoleName) = { val a = node(roleName).address - s"aeron:udp?endpoint=${a.host.get}:${aeronPort(roleName)}" + s"aeron:udp?endpoint=${a.host.get}:${a.port.get}" } val streamId = 1 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 index 0ea2f76dfb..7a377574b1 100644 --- 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 @@ -34,24 +34,6 @@ object HandshakeRestartReceiverSpec extends MultiNodeConfig { } """))) - def aeronPort(roleName: RoleName): Int = - roleName match { - case `first` ⇒ 20531 // TODO yeah, we should have support for dynamic port assignment - case `second` ⇒ 20532 - } - - nodeConfig(first) { - ConfigFactory.parseString(s""" - akka.remote.artery.port = ${aeronPort(first)} - """) - } - - nodeConfig(second) { - ConfigFactory.parseString(s""" - akka.remote.artery.port = ${aeronPort(second)} - """) - } - class Subject extends Actor { def receive = { case "shutdown" ⇒ context.system.terminate() 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 index 102ee4cb86..1e2a2bb39b 100644 --- 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 @@ -55,24 +55,6 @@ object LatencySpec extends MultiNodeConfig { } """))) - def aeronPort(roleName: RoleName): Int = - roleName match { - case `first` ⇒ 20501 // TODO yeah, we should have support for dynamic port assignment - case `second` ⇒ 20502 - } - - nodeConfig(first) { - ConfigFactory.parseString(s""" - akka.remote.artery.port = ${aeronPort(first)} - """) - } - - nodeConfig(second) { - ConfigFactory.parseString(s""" - akka.remote.artery.port = ${aeronPort(second)} - """) - } - final case object Reset def echoProps(): Props = @@ -175,7 +157,7 @@ abstract class LatencySpec def channel(roleName: RoleName) = { val a = node(roleName).address - s"aeron:udp?endpoint=${a.host.get}:${aeronPort(roleName)}" + s"aeron:udp?endpoint=${a.host.get}:${a.port.get}" } lazy val reporterExecutor = Executors.newFixedThreadPool(1) 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 index 2f4444adbb..45282e4d6f 100644 --- 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 @@ -40,24 +40,6 @@ object MaxThroughputSpec extends MultiNodeConfig { } """))) - def aeronPort(roleName: RoleName): Int = - roleName match { - case `first` ⇒ 20501 // TODO yeah, we should have support for dynamic port assignment - case `second` ⇒ 20502 - } - - nodeConfig(first) { - ConfigFactory.parseString(s""" - akka.remote.artery.port = ${aeronPort(first)} - """) - } - - nodeConfig(second) { - ConfigFactory.parseString(s""" - akka.remote.artery.port = ${aeronPort(second)} - """) - } - case object Run sealed trait Echo extends DeadLetterSuppression final case object Start extends Echo diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index fe3776a3a3..c2236542a1 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -6,6 +6,7 @@ package akka.remote.artery import java.nio.ByteOrder import java.util.concurrent.ConcurrentHashMap import java.util.function.{ Function ⇒ JFunction } + import scala.concurrent.Future import scala.concurrent.duration._ import akka.Done @@ -47,6 +48,9 @@ import io.aeron.exceptions.ConductorServiceTimeoutException import org.agrona.ErrorHandler import org.agrona.IoUtil import java.io.File +import java.net.InetSocketAddress +import java.nio.channels.DatagramChannel + import akka.remote.artery.OutboundControlJunction.OutboundControlIngress /** @@ -147,7 +151,6 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val systemMessageResendInterval: FiniteDuration = 1.second private val handshakeTimeout: FiniteDuration = 10.seconds - // TODO support port 0 private def inboundChannel = s"aeron:udp?endpoint=${localAddress.address.host.get}:${localAddress.address.port.get}" private def outboundChannel(a: Address) = s"aeron:udp?endpoint=${a.host.get}:${a.port.get}" private val controlStreamId = 1 @@ -162,10 +165,14 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R startAeron() taskRunner.start() + val port = + if (remoteSettings.ArteryPort == 0) ArteryTransport.autoSelectPort(remoteSettings.ArteryHostname) + else remoteSettings.ArteryPort + // TODO: Configure materializer properly // TODO: Have a supervisor actor _localAddress = UniqueAddress( - Address("akka.artery", system.name, remoteSettings.ArteryHostname, remoteSettings.ArteryPort), + Address("akka.artery", system.name, remoteSettings.ArteryHostname, port), AddressUidExtension(system).addressUid) materializer = ActorMaterializer()(system) @@ -355,3 +362,19 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } +object ArteryTransport { + + /** + * Internal API + * @return A port that is hopefully available + */ + private[remote] def autoSelectPort(hostname: String): Int = { + val socket = DatagramChannel.open().socket() + socket.bind(new InetSocketAddress(hostname, 0)) + val port = socket.getLocalPort + socket.close() + port + } + +} + From d15cab71bd7c27d001e5bfbe7ad7a5c4cea42cb3 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 17 May 2016 13:31:32 +0200 Subject: [PATCH 023/186] amend build for artery dev --- project/AkkaBuild.scala | 75 +++++++++++++++++++++-------------------- project/MiMa.scala | 4 +++ 2 files changed, 42 insertions(+), 37 deletions(-) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 8b4ea67bc8..af6cc00cb2 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -109,25 +109,25 @@ object AkkaBuild extends Build { lazy val actor = Project( id = "akka-actor", base = file("akka-actor") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val testkit = Project( id = "akka-testkit", base = file("akka-testkit"), dependencies = Seq(actor) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val typed = Project( id = "akka-typed-experimental", base = file("akka-typed"), dependencies = Seq(testkit % "compile;test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val actorTests = Project( id = "akka-actor-tests", base = file("akka-actor-tests"), dependencies = Seq(testkit % "compile;test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val benchJmh = Project( id = "akka-bench-jmh", @@ -135,7 +135,7 @@ object AkkaBuild extends Build { dependencies = Seq( actor, http, stream, streamTests, - persistence, distributedData, + remote, persistence, distributedData, testkit ).map(_ % "compile;compile->test;provided->provided") ).disablePlugins(ValidatePullRequest) @@ -143,43 +143,43 @@ object AkkaBuild extends Build { lazy val protobuf = Project( id = "akka-protobuf", base = file("akka-protobuf") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) 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( id = "akka-multi-node-testkit", base = file("akka-multi-node-testkit"), dependencies = Seq(remote, testkit) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) 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( id = "akka-cluster", base = file("akka-cluster"), dependencies = Seq(remote, remoteTests % "test->test" , testkit % "test->test") - ).configs(MultiJvm) + ).configs(MultiJvm).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val clusterMetrics = Project( id = "akka-cluster-metrics", base = file("akka-cluster-metrics"), dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm", slf4j % "test->compile") - ).configs(MultiJvm) + ).configs(MultiJvm).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val clusterTools = Project( id = "akka-cluster-tools", base = file("akka-cluster-tools"), dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm") - ).configs(MultiJvm) + ).configs(MultiJvm).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val clusterSharding = Project( id = "akka-cluster-sharding", @@ -190,31 +190,31 @@ object AkkaBuild extends Build { // provided. dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm", persistence % "compile;test->provided", distributedData % "provided;test", clusterTools) - ).configs(MultiJvm) + ).configs(MultiJvm).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val distributedData = Project( id = "akka-distributed-data-experimental", base = file("akka-distributed-data"), dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm") - ).configs(MultiJvm) + ).configs(MultiJvm).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val slf4j = Project( id = "akka-slf4j", base = file("akka-slf4j"), dependencies = Seq(actor, testkit % "test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val agent = Project( id = "akka-agent", base = file("akka-agent"), dependencies = Seq(actor, testkit % "test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val persistence = Project( id = "akka-persistence", base = file("akka-persistence"), dependencies = Seq(actor, testkit % "test->test", protobuf) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val persistenceQuery = Project( id = "akka-persistence-query-experimental", @@ -224,48 +224,48 @@ object AkkaBuild extends Build { persistence % "compile;provided->provided;test->test", testkit % "compile;test->test", streamTestkit % "compile;test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val persistenceTck = Project( id = "akka-persistence-tck", base = file("akka-persistence-tck"), dependencies = Seq(persistence % "compile;provided->provided;test->test", testkit % "compile;test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val persistenceShared = Project( id = "akka-persistence-shared", base = file("akka-persistence-shared"), dependencies = Seq(persistence % "test->test", testkit % "test->test", remote % "test", protobuf) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val httpCore = Project( id = "akka-http-core", base = file("akka-http-core"), dependencies = Seq(stream, parsing, streamTestkit % "test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val http = Project( id = "akka-http-experimental", base = file("akka-http"), dependencies = Seq(httpCore) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val httpTestkit = Project( id = "akka-http-testkit", base = file("akka-http-testkit"), dependencies = Seq(http, streamTestkit) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val httpTests = Project( id = "akka-http-tests", base = file("akka-http-tests"), dependencies = Seq(httpTestkit % "test", streamTestkit % "test->test", testkit % "test->test", httpSprayJson, httpXml, httpJackson) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val httpMarshallersScala = Project( id = "akka-http-marshallers-scala-experimental", base = file("akka-http-marshallers-scala") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) .settings(parentSettings: _*) .aggregate(httpSprayJson, httpXml) @@ -278,7 +278,7 @@ object AkkaBuild extends Build { lazy val httpMarshallersJava = Project( id = "akka-http-marshallers-java-experimental", base = file("akka-http-marshallers-java") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) .settings(parentSettings: _*) .aggregate(httpJackson) @@ -290,61 +290,61 @@ object AkkaBuild extends Build { id = s"akka-http-$name-experimental", base = file(s"akka-http-marshallers-scala/akka-http-$name"), dependencies = Seq(http) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) def httpMarshallersJavaSubproject(name: String) = Project( id = s"akka-http-$name-experimental", base = file(s"akka-http-marshallers-java/akka-http-$name"), dependencies = Seq(http) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val parsing = Project( id = "akka-parsing", base = file("akka-parsing") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val stream = Project( id = "akka-stream", base = file("akka-stream"), dependencies = Seq(actor) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val streamTestkit = Project( id = "akka-stream-testkit", base = file("akka-stream-testkit"), dependencies = Seq(stream, testkit % "compile;test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val streamTests = Project( id = "akka-stream-tests", base = file("akka-stream-tests"), dependencies = Seq(streamTestkit % "test->test", stream) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val streamTestsTck = Project( id = "akka-stream-tests-tck", base = file("akka-stream-tests-tck"), dependencies = Seq(streamTestkit % "test->test", stream) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val kernel = Project( id = "akka-kernel", base = file("akka-kernel"), dependencies = Seq(actor, testkit % "test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val camel = Project( id = "akka-camel", base = file("akka-camel"), dependencies = Seq(actor, slf4j, testkit % "test->test") - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val osgi = Project( id = "akka-osgi", base = file("akka-osgi"), dependencies = Seq(actor) - ) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val docs = Project( id = "akka-docs", @@ -365,7 +365,7 @@ object AkkaBuild extends Build { id = "akka-contrib", base = file("akka-contrib"), dependencies = Seq(remote, remoteTests % "test->test", cluster, clusterTools, persistence % "compile;test->provided") - ).configs(MultiJvm) + ).configs(MultiJvm).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val samplesSettings = parentSettings ++ ActivatorDist.settings @@ -429,6 +429,7 @@ object AkkaBuild extends Build { executeMvnCommands("Osgi sample Dining hakkers test failed", "clean", "install") }} ) + .disablePlugins(ValidatePullRequest, MimaPlugin) .settings(dontPublishSettings: _*) val dontPublishSettings = Seq( diff --git a/project/MiMa.scala b/project/MiMa.scala index b9c2b17b04..aeb1ea6955 100644 --- a/project/MiMa.scala +++ b/project/MiMa.scala @@ -747,6 +747,10 @@ object MiMa extends AutoPlugin { ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.http.scaladsl.DefaultSSLContextCreation.validateAndWarnAboutLooseSettings") ), "2.4.4" -> Seq( + // 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"), + // #20080, #20081 remove race condition on HTTP client ProblemFilters.exclude[DirectMissingMethodProblem]("akka.http.scaladsl.Http#HostConnectionPool.gatewayFuture"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.http.scaladsl.Http#HostConnectionPool.copy"), From 1296f9986f54a97838bb837852530458abdd4326 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 12 May 2016 13:19:57 +0200 Subject: [PATCH 024/186] make control message ingress buffer bounded --- .../src/main/scala/akka/remote/artery/Control.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Control.scala b/akka-remote/src/main/scala/akka/remote/artery/Control.scala index 56127d956c..a257060ce3 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Control.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Control.scala @@ -155,6 +155,7 @@ private[akka] class OutboundControlJunction(outboundContext: OutboundContext) import OutboundControlJunction._ private val sendControlMessageCallback = getAsyncCallback[ControlMessage](internalSendControlMessage) + private val maxControlMessageBufferSize: Int = 1024 // FIXME config private val buffer = new ArrayDeque[Send] override def preStart(): Unit = { @@ -180,8 +181,13 @@ private[akka] class OutboundControlJunction(outboundContext: OutboundContext) private def internalSendControlMessage(message: ControlMessage): Unit = { if (buffer.isEmpty && isAvailable(out)) push(out, wrap(message)) - else + else if (buffer.size < maxControlMessageBufferSize) buffer.offer(wrap(message)) + else { + // it's alright to drop control messages + // FIXME we need that stage logging support + println(s"dropping control message ${message.getClass.getName} due to full buffer") + } } private def wrap(message: ControlMessage): Send = From 67325da7229ee625ff7ac6aff5e91de2c32cb254 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 13 May 2016 08:06:13 +0200 Subject: [PATCH 025/186] init new handshake for unknown origin, receiver restarted, #20313 * handle UID incarnations, shared association state that can be swapped for new handshakes * detect that message comes from unknown origin and then initiate new handshake (handled by InboundHandshake stage) * simplify the OutboundHandshake stage * doesn't have to listen for HandshakeRsp replies, it can just listen to when the uniqueRemoteAddress future is completed, InboundHandshake stage completes the handshake when it receives HandshakeRsp * send the HandshakeReq via the control message ingress, instead of pushing it downstreams, than also means that HandshakeReq is only sent on the control stream, which is good * materialization race condition --- .../artery/HandshakeRestartReceiverSpec.scala | 18 +-- .../remote/artery/AbstractAssociation.java | 18 +++ .../akka/remote/artery/ArteryTransport.scala | 70 +++++++--- .../akka/remote/artery/Association.scala | 109 +++++++++++++--- .../scala/akka/remote/artery/Control.scala | 10 +- .../scala/akka/remote/artery/Handshake.scala | 122 +++++++++--------- .../remote/artery/MessageDispatcher.scala | 2 +- .../remote/artery/SystemMessageDelivery.scala | 2 +- .../artery/InboundControlJunctionSpec.scala | 4 +- .../remote/artery/InboundHandshakeSpec.scala | 34 ++++- .../remote/artery/OutboundHandshakeSpec.scala | 37 ++---- .../artery/SystemMessageDeliverySpec.scala | 4 +- .../akka/remote/artery/TestContext.scala | 62 ++++++--- 13 files changed, 317 insertions(+), 175 deletions(-) create mode 100644 akka-remote/src/main/java/akka/remote/artery/AbstractAssociation.java 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 index 7a377574b1..44d0278d39 100644 --- 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 @@ -3,21 +3,18 @@ */ package akka.remote.artery -import java.util.concurrent.Executors -import java.util.concurrent.TimeUnit.NANOSECONDS +import scala.concurrent.Await import scala.concurrent.duration._ + import akka.actor._ -import akka.remote.RemoteActorRefProvider +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 -import java.net.InetAddress -import scala.concurrent.Await -import akka.remote.RARP -import akka.remote.AddressUidExtension object HandshakeRestartReceiverSpec extends MultiNodeConfig { val first = role("first") @@ -77,7 +74,7 @@ abstract class HandshakeRestartReceiverSpec val secondAddress = node(second).address val secondAssociation = RARP(system).provider.transport.asInstanceOf[ArteryTransport].association(secondAddress) - val secondUniqueRemoteAddress = Await.result(secondAssociation.uniqueRemoteAddress, 3.seconds) + val secondUniqueRemoteAddress = Await.result(secondAssociation.associationState.uniqueRemoteAddress, 3.seconds) secondUniqueRemoteAddress.address should ===(secondAddress) secondUniqueRemoteAddress.uid should ===(secondUid) @@ -93,7 +90,10 @@ abstract class HandshakeRestartReceiverSpec } val (secondUid2, subject2) = identifyWithUid(secondRootPath, "subject2") secondUid2 should !==(secondUid) - // FIXME verify that UID in association was replaced (not implemented yet) + val secondUniqueRemoteAddress2 = Await.result(secondAssociation.associationState.uniqueRemoteAddress, 3.seconds) + secondUniqueRemoteAddress2.uid should ===(secondUid2) + secondUniqueRemoteAddress2.address should ===(secondAddress) + secondUniqueRemoteAddress2 should !==(secondUniqueRemoteAddress) subject2 ! "shutdown" } 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/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index c2236542a1..c8b694190a 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -3,18 +3,24 @@ */ package akka.remote.artery +import java.io.File import java.nio.ByteOrder import java.util.concurrent.ConcurrentHashMap import java.util.function.{ Function ⇒ JFunction } import scala.concurrent.Future +import scala.concurrent.Promise import scala.concurrent.duration._ +import scala.util.Failure +import scala.util.Success + import akka.Done import akka.NotUsed import akka.actor.ActorRef import akka.actor.Address import akka.actor.ExtendedActorSystem import akka.actor.InternalActorRef +import akka.actor.Props import akka.event.Logging import akka.event.LoggingAdapter import akka.remote.AddressUidExtension @@ -23,8 +29,10 @@ import akka.remote.MessageSerializer import akka.remote.RemoteActorRef import akka.remote.RemoteActorRefProvider import akka.remote.RemoteTransport +import akka.remote.SeqNo import akka.remote.UniqueAddress import akka.remote.artery.InboundControlJunction.ControlMessageSubject +import akka.remote.artery.OutboundControlJunction.OutboundControlIngress import akka.remote.transport.AkkaPduCodec import akka.remote.transport.AkkaPduProtobufCodec import akka.serialization.Serialization @@ -60,7 +68,8 @@ private[akka] final case class InboundEnvelope( recipient: InternalActorRef, recipientAddress: Address, message: AnyRef, - senderOption: Option[ActorRef]) + senderOption: Option[ActorRef], + originAddress: UniqueAddress) /** * INTERNAL API @@ -75,7 +84,7 @@ private[akka] trait InboundContext { /** * An inbound stage can send control message, e.g. a reply, to the origin - * address with this method. + * address with this method. It will be sent over the control sub-channel. */ def sendControl(to: Address, message: ControlMessage): Unit @@ -85,6 +94,26 @@ private[akka] trait InboundContext { def association(remoteAddress: Address): OutboundContext } +final class AssociationState( + val incarnation: Int, + val uniqueRemoteAddressPromise: Promise[UniqueAddress]) { + + /** + * Full outbound address with UID for this association. + * Completed when by the handshake. + */ + def uniqueRemoteAddress: Future[UniqueAddress] = uniqueRemoteAddressPromise.future + + 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. @@ -101,17 +130,15 @@ private[akka] trait OutboundContext { */ def remoteAddress: Address - /** - * Full outbound address with UID for this association. - * Completed when by the handshake. - */ - def uniqueRemoteAddress: Future[UniqueAddress] + def associationState: AssociationState + + def completeHandshake(peer: UniqueAddress): Unit /** - * Set the outbound address with UID when the - * handshake is completed. + * 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 completeRemoteAddress(a: UniqueAddress): Unit + def sendControl(message: ControlMessage): Unit /** * An outbound stage can listen to control messages @@ -139,7 +166,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R @volatile private[this] var driver: MediaDriver = _ @volatile private[this] var aeron: Aeron = _ - override val log: LoggingAdapter = Logging(system.eventStream, getClass.getName) + override val log: LoggingAdapter = Logging(system, getClass.getName) override def defaultAddress: Address = localAddress.address override def addresses: Set[Address] = Set(defaultAddress) override def localAddressForRemote(remote: Address): Address = defaultAddress @@ -280,9 +307,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } } - override def quarantine(remoteAddress: Address, uid: Option[Int]): Unit = { - ??? - } + override def quarantine(remoteAddress: Address, uid: Option[Int]): Unit = + association(remoteAddress).quarantine(uid) def outbound(outboundContext: OutboundContext): Sink[Send, Any] = { Flow.fromGraph(killSwitch.flow[Send]) @@ -302,6 +328,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .to(new AeronSink(outboundChannel(outboundContext.remoteAddress), controlStreamId, aeron, taskRunner)) } + // FIXME hack until real envelopes, encoding originAddress in sender :) + private val dummySender = system.systemActorOf(Props.empty, "dummy") + // TODO: Try out parallelized serialization (mapAsync) for performance val encoder: Flow[Send, ByteString, NotUsed] = Flow[Send].map { sendEnvelope ⇒ val pdu: ByteString = codec.constructMessage( @@ -310,8 +339,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R Serialization.currentTransportInformation.withValue(Serialization.Information(localAddress.address, system)) { MessageSerializer.serialize(system, sendEnvelope.message.asInstanceOf[AnyRef]) }, - sendEnvelope.senderOption, - seqOption = None, // FIXME: Acknowledgements will be handled differently I just reused the old codec + if (sendEnvelope.senderOption.isDefined) sendEnvelope.senderOption else Some(dummySender), // FIXME: hack until real envelopes + seqOption = Some(SeqNo(localAddress.uid)), // FIXME: hack until real envelopes ackOption = None) // TODO: Drop unserializable messages @@ -337,14 +366,15 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R m.recipient, m.recipientAddress, MessageSerializer.deserialize(system, m.serializedMessage), - m.senderOption) + if (m.senderOption.get.path.name == "dummy") None else m.senderOption, // FIXME hack until real envelopes + UniqueAddress(m.senderOption.get.path.address, m.seq.rawValue.toInt)) // FIXME hack until real envelopes } val inboundFlow: Flow[ByteString, ByteString, NotUsed] = { Flow.fromSinkAndSource( decoder .via(deserializer) - .via(new InboundHandshake(this)) + .via(new InboundHandshake(this, inControlStream = false)) .to(messageDispatcherSink), Source.maybe[ByteString].via(killSwitch.flow)) } @@ -353,9 +383,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R Flow.fromSinkAndSourceMat( decoder .via(deserializer) - .via(new InboundHandshake(this)) - .via(new SystemMessageAcker(this)) + .via(new InboundHandshake(this, inControlStream = true)) .viaMat(new InboundControlJunction)(Keep.right) + .via(new SystemMessageAcker(this)) .to(messageDispatcherSink), Source.maybe[ByteString].via(killSwitch.flow))((a, b) ⇒ a) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 323aac7266..55e9c3b004 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -3,22 +3,27 @@ */ package akka.remote.artery -import scala.concurrent.Future import scala.concurrent.Promise +import scala.util.Success import akka.actor.ActorRef import akka.actor.Address import akka.actor.RootActorPath import akka.dispatch.sysmsg.SystemMessage +import akka.event.Logging import akka.remote.EndpointManager.Send import akka.remote.RemoteActorRef import akka.remote.UniqueAddress import akka.remote.artery.InboundControlJunction.ControlMessageSubject +import akka.remote.artery.OutboundControlJunction.OutboundControlIngress import akka.stream.Materializer import akka.stream.OverflowStrategy +import akka.stream.scaladsl.Keep import akka.stream.scaladsl.Source import akka.stream.scaladsl.SourceQueueWithComplete -import akka.remote.artery.OutboundControlJunction.OutboundControlIngress -import akka.stream.scaladsl.Keep +import akka.util.Unsafe +import java.util.concurrent.locks.ReentrantLock +import java.util.concurrent.CountDownLatch +import java.util.concurrent.TimeUnit /** * INTERNAL API @@ -30,35 +35,87 @@ private[akka] class Association( val transport: ArteryTransport, val materializer: Materializer, override val remoteAddress: Address, - override val controlSubject: ControlMessageSubject) extends OutboundContext { + override val controlSubject: ControlMessageSubject) + extends AbstractAssociation with OutboundContext { + + private val log = Logging(transport.system, getClass.getName) @volatile private[this] var queue: SourceQueueWithComplete[Send] = _ - @volatile private[this] var systemMessageQueue: SourceQueueWithComplete[Send] = _ + @volatile private[this] var controlQueue: SourceQueueWithComplete[Send] = _ @volatile private[this] var _outboundControlIngress: OutboundControlIngress = _ + private val materializing = new CountDownLatch(1) def outboundControlIngress: OutboundControlIngress = { - if (_outboundControlIngress eq null) - throw new IllegalStateException("outboundControlIngress not initialized yet") - _outboundControlIngress + if (_outboundControlIngress ne null) + _outboundControlIngress + else { + // materialization not completed yet + materializing.await(10, TimeUnit.SECONDS) + if (_outboundControlIngress eq null) + throw new IllegalStateException("outboundControlIngress not initialized yet") + _outboundControlIngress + } } override def localAddress: UniqueAddress = transport.localAddress - // FIXME we also need to be able to switch to new uid - private val _uniqueRemoteAddress = Promise[UniqueAddress]() - override def uniqueRemoteAddress: Future[UniqueAddress] = _uniqueRemoteAddress.future - override def completeRemoteAddress(a: UniqueAddress): Unit = { - require(a.address == remoteAddress, s"Wrong UniqueAddress got [$a.address], expected [$remoteAddress]") - _uniqueRemoteAddress.trySuccess(a) + /** + * Holds reference to shared state of Association - *access only via helper methods* + */ + @volatile + private[this] var _sharedStateDoNotCallMeDirectly: AssociationState = + new AssociationState(incarnation = 1, uniqueRemoteAddressPromise = Promise()) + + /** + * 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] + + override def completeHandshake(peer: UniqueAddress): Unit = { + require(remoteAddress == peer.address, + s"wrong remote address in completeHandshake, got ${peer.address}, expected ${remoteAddress}") + val current = associationState + current.uniqueRemoteAddressPromise.trySuccess(peer) + current.uniqueRemoteAddress.value match { + case Some(Success(`peer`)) ⇒ // our value + case _ ⇒ + val newState = new AssociationState(incarnation = current.incarnation + 1, Promise.successful(peer)) + if (swapState(current, newState)) { + current.uniqueRemoteAddress.value match { + case Some(Success(old)) ⇒ + log.debug("Incarnation {} of association to [{}] with new UID [{}] (old UID [{}])", + newState.incarnation, peer.address, peer.uid, old.uid) + quarantine(Some(old.uid)) + case _ ⇒ // Failed, nothing to do + } + // if swap failed someone else completed before us, and that is fine + } + } } + // OutboundContext + override def sendControl(message: ControlMessage): Unit = + outboundControlIngress.sendControlMessage(message) + def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = { // TODO: lookup subchannel // FIXME: Use a different envelope than the old Send, but make sure the new is handled by deadLetters properly message match { case _: SystemMessage ⇒ implicit val ec = materializer.executionContext - systemMessageQueue.offer(Send(message, senderOption, recipient, None)).onFailure { + controlQueue.offer(Send(message, senderOption, recipient, None)).onFailure { case e ⇒ // FIXME proper error handling, and quarantining println(s"# System message dropped, due to $e") // FIXME @@ -72,20 +129,30 @@ private[akka] class Association( override val dummyRecipient: RemoteActorRef = transport.provider.resolveActorRef(RootActorPath(remoteAddress) / "system" / "dummy").asInstanceOf[RemoteActorRef] - def quarantine(uid: Option[Int]): Unit = () + def quarantine(uid: Option[Int]): Unit = { + // FIXME implement + log.error("Association to [{}] with UID [{}] is irrecoverably failed. Quarantining address.", + remoteAddress, uid.getOrElse("unknown")) + } // Idempotent def associate(): Unit = { // FIXME detect and handle stream failure, e.g. handshake timeout - if (queue eq null) - queue = Source.queue(256, OverflowStrategy.dropBuffer) - .to(transport.outbound(this)).run()(materializer) - if (systemMessageQueue eq null) { + + // it's important to materialize the outboundControl stream first, + // so that outboundControlIngress is ready when stages for all streams start + if (controlQueue eq null) { val (q, control) = Source.queue(256, OverflowStrategy.dropBuffer) .toMat(transport.outboundControl(this))(Keep.both) .run()(materializer) - systemMessageQueue = q + controlQueue = q _outboundControlIngress = control + // stage in the control stream may access the outboundControlIngress before returned here + // using CountDownLatch to make sure that materialization is completed before accessing outboundControlIngress + materializing.countDown() + + queue = Source.queue(256, OverflowStrategy.dropBuffer) + .to(transport.outbound(this)).run()(materializer) } } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Control.scala b/akka-remote/src/main/scala/akka/remote/artery/Control.scala index a257060ce3..ae3b51c3bd 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Control.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Control.scala @@ -3,20 +3,22 @@ */ package akka.remote.artery +import java.util.ArrayDeque + import scala.concurrent.Future import scala.concurrent.Promise + import akka.Done +import akka.remote.EndpointManager.Send 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.EndpointManager.Send -import java.util.ArrayDeque -import akka.stream.stage.CallbackWrapper /** * Marker trait for reply messages @@ -97,7 +99,7 @@ private[akka] class InboundControlJunction // InHandler override def onPush(): Unit = { grab(in) match { - case env @ InboundEnvelope(_, _, _: ControlMessage, _) ⇒ + case env @ InboundEnvelope(_, _, _: ControlMessage, _, _) ⇒ observers.foreach(_.notify(env)) pull(in) case env ⇒ diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala index 92793ff6e6..a838e133dd 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -4,11 +4,12 @@ package akka.remote.artery import java.util.concurrent.TimeoutException + import scala.concurrent.duration._ -import akka.Done +import scala.util.Success + import akka.remote.EndpointManager.Send import akka.remote.UniqueAddress -import akka.remote.artery.InboundControlJunction.ControlMessageObserver import akka.stream.Attributes import akka.stream.FlowShape import akka.stream.Inlet @@ -29,7 +30,6 @@ private[akka] object OutboundHandshake { private sealed trait HandshakeState private case object Start extends HandshakeState - private case object ControlMessageObserverAttached extends HandshakeState private case object ReqInProgress extends HandshakeState private case object Completed extends HandshakeState @@ -46,34 +46,24 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext, timeout: override val shape: FlowShape[Send, Send] = FlowShape(in, out) override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = - new TimerGraphStageLogic(shape) with InHandler with OutHandler with ControlMessageObserver { + new TimerGraphStageLogic(shape) with InHandler with OutHandler { import OutboundHandshake._ private var handshakeState: HandshakeState = Start - private def remoteAddress = outboundContext.remoteAddress - override def preStart(): Unit = { - if (outboundContext.uniqueRemoteAddress.isCompleted) { + val uniqueRemoteAddress = outboundContext.associationState.uniqueRemoteAddress + if (uniqueRemoteAddress.isCompleted) { handshakeState = Completed } else { + // The InboundHandshake stage will complete the uniqueRemoteAddress future + // when it receives the HandshakeRsp reply implicit val ec = materializer.executionContext - outboundContext.controlSubject.attach(this).foreach { - getAsyncCallback[Done] { _ ⇒ - if (handshakeState != Completed) { - if (isAvailable(out)) - pushHandshakeReq() - else - handshakeState = ControlMessageObserverAttached - } - }.invoke - } - - outboundContext.uniqueRemoteAddress.foreach { + uniqueRemoteAddress.foreach { getAsyncCallback[UniqueAddress] { a ⇒ if (handshakeState != Completed) { handshakeCompleted() - if (isAvailable(out) && !hasBeenPulled(in)) + if (isAvailable(out)) pull(in) } }.invoke @@ -83,10 +73,6 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext, timeout: } } - override def postStop(): Unit = { - outboundContext.controlSubject.detach(this) - } - // InHandler override def onPush(): Unit = { if (handshakeState != Completed) @@ -98,50 +84,27 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext, timeout: override def onPull(): Unit = { handshakeState match { case Completed ⇒ pull(in) - case ControlMessageObserverAttached ⇒ - pushHandshakeReq() - case Start ⇒ // will push HandshakeReq when ControlMessageObserver is attached + case Start ⇒ + // will pull when handshake reply is received (uniqueRemoteAddress completed) + handshakeState = ReqInProgress + outboundContext.sendControl(HandshakeReq(outboundContext.localAddress)) case ReqInProgress ⇒ // will pull when handshake reply is received } } - private def pushHandshakeReq(): Unit = { - handshakeState = ReqInProgress - // FIXME we should be able to Send without recipient ActorRef - push(out, Send(HandshakeReq(outboundContext.localAddress), None, outboundContext.dummyRecipient, None)) - } - private def handshakeCompleted(): Unit = { handshakeState = Completed cancelTimer(HandshakeTimeout) - outboundContext.controlSubject.detach(this) } override protected def onTimer(timerKey: Any): Unit = timerKey match { case HandshakeTimeout ⇒ + // FIXME would it make sense to retry a few times before failing? failStage(new TimeoutException( - s"Handshake with [$remoteAddress] did not complete within ${timeout.toMillis} ms")) + s"Handshake with [${outboundContext.remoteAddress}] did not complete within ${timeout.toMillis} ms")) } - // ControlMessageObserver, external call - override def notify(inboundEnvelope: InboundEnvelope): Unit = { - inboundEnvelope.message match { - case rsp: HandshakeRsp ⇒ - if (rsp.from.address == remoteAddress) { - getAsyncCallback[HandshakeRsp] { reply ⇒ - if (handshakeState != Completed) { - handshakeCompleted() - outboundContext.completeRemoteAddress(reply.from) - if (isAvailable(out) && !hasBeenPulled(in)) - pull(in) - } - }.invoke(rsp) - } - case _ ⇒ // not interested - } - } - setHandlers(in, out, this) } @@ -150,31 +113,62 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext, timeout: /** * INTERNAL API */ -private[akka] class InboundHandshake(inboundContext: InboundContext) extends GraphStage[FlowShape[InboundEnvelope, InboundEnvelope]] { +private[akka] 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 InHandler with OutHandler { + new TimerGraphStageLogic(shape) with OutHandler { import OutboundHandshake._ // InHandler - override def onPush(): Unit = { - grab(in) match { - case InboundEnvelope(_, _, HandshakeReq(from), _) ⇒ - inboundContext.association(from.address).completeRemoteAddress(from) - inboundContext.sendControl(from.address, HandshakeRsp(inboundContext.localAddress)) - pull(in) - case other ⇒ - push(out, other) + if (inControlStream) + setHandler(in, new InHandler { + override def onPush(): Unit = { + grab(in) match { + case InboundEnvelope(_, _, HandshakeReq(from), _, _) ⇒ + inboundContext.association(from.address).completeHandshake(from) + inboundContext.sendControl(from.address, HandshakeRsp(inboundContext.localAddress)) + pull(in) + case InboundEnvelope(_, _, HandshakeRsp(from), _, _) ⇒ + inboundContext.association(from.address).completeHandshake(from) + pull(in) + case other ⇒ onMessage(other) + } + } + }) + else + setHandler(in, new InHandler { + override def onPush(): Unit = onMessage(grab(in)) + }) + + private def onMessage(env: InboundEnvelope): Unit = { + if (isKnownOrigin(env.originAddress)) + push(out, env) + else { + inboundContext.sendControl(env.originAddress.address, HandshakeReq(inboundContext.localAddress)) + // FIXME Note that we have the originAddress that would be needed to complete the handshake + // but it is not done here because the handshake might exchange more information. + // Is that a valid thought? + // drop message from unknown, this system was probably restarted + pull(in) + } + } + + private def isKnownOrigin(originAddress: UniqueAddress): Boolean = { + // FIXME these association lookups are probably too costly for each message, need local cache or something + val associationState = inboundContext.association(originAddress.address).associationState + associationState.uniqueRemoteAddress.value match { + case Some(Success(a)) if a.uid == originAddress.uid ⇒ true + case x ⇒ false } } // OutHandler override def onPull(): Unit = pull(in) - setHandlers(in, out, this) + setHandler(out, this) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala b/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala index 3286729521..a76a4b478b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala @@ -25,7 +25,7 @@ private[akka] class MessageDispatcher( provider: RemoteActorRefProvider) { private val remoteDaemon = provider.remoteDaemon - private val log = Logging(system.eventStream, getClass.getName) + private val log = Logging(system, getClass.getName) def dispatch(recipient: InternalActorRef, recipientAddress: Address, diff --git a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala index 21e62ca917..4fa7a2d23f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -199,7 +199,7 @@ private[akka] class SystemMessageAcker(inboundContext: InboundContext) extends G // InHandler override def onPush(): Unit = { grab(in) match { - case env @ InboundEnvelope(_, _, sysEnv @ SystemMessageEnvelope(_, n, ackReplyTo), _) ⇒ + case env @ InboundEnvelope(_, _, sysEnv @ SystemMessageEnvelope(_, n, ackReplyTo), _, _) ⇒ if (n == seqNo) { inboundContext.sendControl(ackReplyTo.address, Ack(n, localAddress)) seqNo += 1 diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala index 4b67be4274..1ead987ae9 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala @@ -42,9 +42,9 @@ class InboundControlJunctionSpec extends AkkaSpec with ImplicitSender { val recipient = null.asInstanceOf[InternalActorRef] // not used val ((upstream, controlSubject), downstream) = TestSource.probe[AnyRef] - .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, None)) + .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, None, addressA)) .viaMat(new InboundControlJunction)(Keep.both) - .map { case InboundEnvelope(_, _, msg, _) ⇒ msg } + .map { case InboundEnvelope(_, _, msg, _, _) ⇒ msg } .toMat(TestSink.probe[Any])(Keep.both) .run() diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala index f2b378efc5..ea1b460711 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala @@ -3,6 +3,7 @@ */ package akka.remote.artery +import scala.concurrent.Await import scala.concurrent.duration._ import akka.actor.Address @@ -40,9 +41,9 @@ class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { private def setupStream(inboundContext: InboundContext, timeout: FiniteDuration = 5.seconds): (TestPublisher.Probe[AnyRef], TestSubscriber.Probe[Any]) = { val recipient = null.asInstanceOf[InternalActorRef] // not used TestSource.probe[AnyRef] - .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, None)) - .via(new InboundHandshake(inboundContext)) - .map { case InboundEnvelope(_, _, msg, _) ⇒ msg } + .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, None, addressA)) + .via(new InboundHandshake(inboundContext, inControlStream = true)) + .map { case InboundEnvelope(_, _, msg, _, _) ⇒ msg } .toMat(TestSink.probe[Any])(Keep.both) .run() } @@ -62,6 +63,33 @@ class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { 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)) + upstream.sendNext("msg1") + downstream.expectNext("msg1") + val uniqueRemoteAddress = Await.result( + inboundContext.association(addressA.address).associationState.uniqueRemoteAddress, remainingOrDefault) + uniqueRemoteAddress should ===(addressA) + downstream.cancel() + } + + "send HandshakeReq as when receiving message from unknown (receiving system restarted)" in { + val replyProbe = TestProbe() + val inboundContext = new ManualReplyInboundContext(replyProbe.ref, addressB, new TestControlMessageSubject) + val (upstream, downstream) = setupStream(inboundContext) + + downstream.request(10) + // no HandshakeReq + upstream.sendNext("msg17") + replyProbe.expectMsg(HandshakeReq(addressB)) + downstream.expectNoMsg(200.millis) // messages from unknown are dropped + 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 index a75c8576bf..3e9d73e0f5 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala @@ -8,12 +8,10 @@ import java.util.concurrent.TimeoutException import scala.concurrent.duration._ import akka.actor.Address -import akka.actor.InternalActorRef import akka.remote.EndpointManager.Send import akka.remote.RemoteActorRef import akka.remote.UniqueAddress import akka.remote.artery.OutboundHandshake.HandshakeReq -import akka.remote.artery.OutboundHandshake.HandshakeRsp import akka.remote.artery.SystemMessageDelivery._ import akka.stream.ActorMaterializer import akka.stream.ActorMaterializerSettings @@ -24,6 +22,7 @@ import akka.stream.testkit.scaladsl.TestSink import akka.stream.testkit.scaladsl.TestSource import akka.testkit.AkkaSpec import akka.testkit.ImplicitSender +import akka.testkit.TestProbe class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { @@ -45,53 +44,37 @@ class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { "OutboundHandshake stage" must { "send HandshakeReq when first pulled" in { - val inboundContext = new TestInboundContext(localAddress = addressA) + val controlProbe = TestProbe() + val inboundContext = new TestInboundContext(localAddress = addressA, controlProbe = Some(controlProbe.ref)) val outboundContext = inboundContext.association(addressB.address) val (upstream, downstream) = setupStream(outboundContext) downstream.request(10) - downstream.expectNext(HandshakeReq(addressA)) + controlProbe.expectMsg(HandshakeReq(addressA)) downstream.cancel() } - "timeout if not receiving HandshakeRsp" in { + "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)) downstream.expectError().getClass should be(classOf[TimeoutException]) } "not deliver messages from upstream until handshake completed" in { - val controlSubject = new TestControlMessageSubject - val inboundContext = new TestInboundContext(localAddress = addressA, controlSubject) + val controlProbe = TestProbe() + val inboundContext = new TestInboundContext(localAddress = addressA, controlProbe = Some(controlProbe.ref)) val outboundContext = inboundContext.association(addressB.address) - val recipient = null.asInstanceOf[InternalActorRef] // not used val (upstream, downstream) = setupStream(outboundContext) downstream.request(10) - downstream.expectNext(HandshakeReq(addressA)) + controlProbe.expectMsg(HandshakeReq(addressA)) upstream.sendNext("msg1") downstream.expectNoMsg(200.millis) - controlSubject.sendControl(InboundEnvelope(recipient, addressA.address, HandshakeRsp(addressB), None)) - downstream.expectNext("msg1") - upstream.sendNext("msg2") - downstream.expectNext("msg2") - downstream.cancel() - } - - "complete handshake via another sub-channel" 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)) - upstream.sendNext("msg1") - // handshake completed first by another sub-channel - outboundContext.completeRemoteAddress(addressB) + // InboundHandshake stage will complete the handshake when receiving HandshakeRsp + inboundContext.association(addressB.address).completeHandshake(addressB) downstream.expectNext("msg1") upstream.sendNext("msg2") downstream.expectNext("msg2") diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index 6e24432483..62f1ecd768 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -10,9 +10,7 @@ import scala.concurrent.duration._ import akka.NotUsed import akka.actor.ActorIdentity -import akka.actor.ActorRef import akka.actor.ActorSystem -import akka.actor.Address import akka.actor.ExtendedActorSystem import akka.actor.Identify import akka.actor.InternalActorRef @@ -84,7 +82,7 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commo Flow[Send] .map { case Send(sysEnv: SystemMessageEnvelope, _, _, _) ⇒ - InboundEnvelope(recipient, addressB.address, sysEnv, None) + InboundEnvelope(recipient, addressB.address, sysEnv, None, addressA) } .async .via(new SystemMessageAcker(inboundContext)) diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala index 0db4672139..d1af1446ea 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -3,35 +3,39 @@ */ package akka.remote.artery -import akka.remote.UniqueAddress -import akka.actor.Address -import scala.concurrent.Future -import akka.remote.artery.InboundControlJunction.ControlMessageSubject -import akka.remote.RemoteActorRef -import scala.concurrent.Promise -import akka.Done -import akka.remote.artery.InboundControlJunction.ControlMessageObserver -import java.util.concurrent.CopyOnWriteArrayList 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.RemoteActorRef +import akka.remote.UniqueAddress +import akka.remote.artery.InboundControlJunction.ControlMessageObserver +import akka.remote.artery.InboundControlJunction.ControlMessageSubject private[akka] class TestInboundContext( override val localAddress: UniqueAddress, val controlSubject: TestControlMessageSubject = new TestControlMessageSubject, - replyDropRate: Double = 0.0) extends InboundContext { + val controlProbe: Option[ActorRef] = None, + val replyDropRate: Double = 0.0) extends InboundContext { private val associations = new ConcurrentHashMap[Address, OutboundContext] - def sendControl(to: Address, message: ControlMessage) = { + override def sendControl(to: Address, message: ControlMessage) = { if (ThreadLocalRandom.current().nextDouble() >= replyDropRate) - controlSubject.sendControl(InboundEnvelope(null, to, message, None)) + association(to).sendControl(message) } - def association(remoteAddress: Address): OutboundContext = + override def association(remoteAddress: Address): OutboundContext = associations.get(remoteAddress) match { case null ⇒ - val a = new TestOutboundContext(localAddress, remoteAddress, controlSubject) + val a = createAssociation(remoteAddress) associations.putIfAbsent(remoteAddress, a) match { case null ⇒ a case existing ⇒ existing @@ -40,20 +44,38 @@ private[akka] class TestInboundContext( } protected def createAssociation(remoteAddress: Address): OutboundContext = - new TestOutboundContext(localAddress, remoteAddress, controlSubject) + new TestOutboundContext(localAddress, remoteAddress, controlSubject, controlProbe) } private[akka] class TestOutboundContext( override val localAddress: UniqueAddress, override val remoteAddress: Address, - override val controlSubject: TestControlMessageSubject) extends OutboundContext { + override val controlSubject: TestControlMessageSubject, + val controlProbe: Option[ActorRef] = None) extends OutboundContext { - private val _uniqueRemoteAddress = Promise[UniqueAddress]() - def uniqueRemoteAddress: Future[UniqueAddress] = _uniqueRemoteAddress.future - def completeRemoteAddress(a: UniqueAddress): Unit = _uniqueRemoteAddress.trySuccess(a) + // access to this is synchronized (it's a test utility) + private var _associationState = new AssociationState(1, Promise()) + + override def associationState: AssociationState = synchronized { + _associationState + } + + override def completeHandshake(peer: UniqueAddress): Unit = synchronized { + _associationState.uniqueRemoteAddressPromise.trySuccess(peer) + _associationState.uniqueRemoteAddress.value match { + case Some(Success(`peer`)) ⇒ // our value + case _ ⇒ + _associationState = new AssociationState(incarnation = _associationState.incarnation + 1, Promise.successful(peer)) + } + } + + override def sendControl(message: ControlMessage) = { + controlProbe.foreach(_ ! message) + controlSubject.sendControl(InboundEnvelope(null, remoteAddress, message, None, localAddress)) + } // FIXME we should be able to Send without a recipient ActorRef - def dummyRecipient: RemoteActorRef = null + override def dummyRecipient: RemoteActorRef = null } From d4c086c0ffc9d1de4c70d0750a1b82c930ca885d Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 13 May 2016 15:34:37 +0200 Subject: [PATCH 026/186] add quarantining state and inbound quarantine check, #20313 * InboundQuarantineCheck stage sends back Quarantined control message if sender system is quarantined * Ported old RemoteRestartedQuarantinedSpec * outbound quarantine check and piercing with ActorSelectionMessage * port of old RemoteQuarantinePiercingSpec and PiercingShouldKeepQuarantineSpec * quarantine if sys msg buffer overflow --- .../artery/HandshakeRestartReceiverSpec.scala | 15 +- .../PiercingShouldKeepQuarantineSpec.scala | 102 +++++++++++ .../artery/RemoteQuarantinePiercingSpec.scala | 137 ++++++++++++++ .../RemoteRestartedQuarantinedSpec.scala | 169 ++++++++++++++++++ .../akka/remote/artery/ArteryTransport.scala | 81 ++++++++- .../akka/remote/artery/Association.scala | 92 +++++++--- .../scala/akka/remote/artery/Control.scala | 15 +- .../scala/akka/remote/artery/Handshake.scala | 2 +- .../artery/InboundQuarantineCheck.scala | 44 +++++ .../remote/artery/SystemMessageDelivery.scala | 53 ++++-- .../remote/artery/InboundHandshakeSpec.scala | 6 +- .../artery/SystemMessageDeliverySpec.scala | 2 +- .../akka/remote/artery/TestContext.scala | 8 +- 13 files changed, 659 insertions(+), 67 deletions(-) create mode 100644 akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/PiercingShouldKeepQuarantineSpec.scala create mode 100644 akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteQuarantinePiercingSpec.scala create mode 100644 akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteRestartedQuarantinedSpec.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala 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 index 44d0278d39..5fea761616 100644 --- 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 @@ -55,9 +55,11 @@ abstract class HandshakeRestartReceiverSpec super.afterAll() } - def identifyWithUid(rootPath: ActorPath, actorName: String): (Int, ActorRef) = { - system.actorSelection(rootPath / "user" / actorName) ! "identify" - expectMsgType[(Int, ActorRef)] + def identifyWithUid(rootPath: ActorPath, actorName: String, timeout: FiniteDuration = remainingOrDefault): (Int, ActorRef) = { + within(timeout) { + system.actorSelection(rootPath / "user" / actorName) ! "identify" + expectMsgType[(Int, ActorRef)] + } } "Artery Handshake" must { @@ -70,7 +72,7 @@ abstract class HandshakeRestartReceiverSpec runOn(first) { val secondRootPath = node(second) - val (secondUid, _) = identifyWithUid(secondRootPath, "subject") + val (secondUid, _) = identifyWithUid(secondRootPath, "subject", 5.seconds) val secondAddress = node(second).address val secondAssociation = RARP(system).provider.transport.asInstanceOf[ArteryTransport].association(secondAddress) @@ -83,14 +85,13 @@ abstract class HandshakeRestartReceiverSpec within(30.seconds) { awaitAssert { - within(1.second) { - identifyWithUid(secondRootPath, "subject2") - } + identifyWithUid(secondRootPath, "subject2", 1.second) } } val (secondUid2, subject2) = identifyWithUid(secondRootPath, "subject2") secondUid2 should !==(secondUid) val secondUniqueRemoteAddress2 = Await.result(secondAssociation.associationState.uniqueRemoteAddress, 3.seconds) + println(s"# ${secondAssociation.associationState} secondUid $secondUid $secondUid2") // FIXME secondUniqueRemoteAddress2.uid should ===(secondUid2) secondUniqueRemoteAddress2.address should ===(secondAddress) secondUniqueRemoteAddress2 should !==(secondUniqueRemoteAddress) diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/PiercingShouldKeepQuarantineSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/PiercingShouldKeepQuarantineSpec.scala new file mode 100644 index 0000000000..fd292f172f --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/PiercingShouldKeepQuarantineSpec.scala @@ -0,0 +1,102 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.duration._ +import com.typesafe.config.ConfigFactory +import akka.actor._ +import akka.testkit._ +import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec } +import akka.remote.testconductor.RoleName +import akka.remote.AddressUidExtension +import akka.remote.RARP + +object PiercingShouldKeepQuarantineSpec 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 + akka.remote.retry-gate-closed-for = 0.5s + + akka.remote.artery.enabled = on + """))) + + def aeronPort(roleName: RoleName): Int = + roleName match { + case `first` ⇒ 20561 // TODO yeah, we should have support for dynamic port assignment + case `second` ⇒ 20562 + } + + nodeConfig(first) { + ConfigFactory.parseString(s""" + akka.remote.artery.port = ${aeronPort(first)} + """) + } + + nodeConfig(second) { + ConfigFactory.parseString(s""" + akka.remote.artery.port = ${aeronPort(second)} + """) + } + + class Subject extends Actor { + def receive = { + case "getuid" ⇒ sender() ! AddressUidExtension(context.system).addressUid + } + } + +} + +class PiercingShouldKeepQuarantineSpecMultiJvmNode1 extends PiercingShouldKeepQuarantineSpec +class PiercingShouldKeepQuarantineSpecMultiJvmNode2 extends PiercingShouldKeepQuarantineSpec + +abstract class PiercingShouldKeepQuarantineSpec extends MultiNodeSpec(PiercingShouldKeepQuarantineSpec) + with STMultiNodeSpec + with ImplicitSender { + + import PiercingShouldKeepQuarantineSpec._ + + override def initialParticipants = roles.size + + "While probing through the quarantine remoting" must { + + "not lose existing quarantine marker" taggedAs LongRunningTest in { + runOn(first) { + enterBarrier("actors-started") + + // Communicate with second system + system.actorSelection(node(second) / "user" / "subject") ! "getuid" + val uid = expectMsgType[Int](10.seconds) + enterBarrier("actor-identified") + + // Manually Quarantine the other system + RARP(system).provider.transport.quarantine(node(second).address, Some(uid)) + + // Quarantining is not immediate + Thread.sleep(1000) + + // Quarantine is up -- Should not be able to communicate with remote system any more + for (_ ← 1 to 4) { + system.actorSelection(node(second) / "user" / "subject") ! "getuid" + expectNoMsg(2.seconds) + } + + enterBarrier("quarantine-intact") + + } + + runOn(second) { + system.actorOf(Props[Subject], "subject") + enterBarrier("actors-started") + enterBarrier("actor-identified") + enterBarrier("quarantine-intact") + } + + } + + } +} diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteQuarantinePiercingSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteQuarantinePiercingSpec.scala new file mode 100644 index 0000000000..e546a293e6 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteQuarantinePiercingSpec.scala @@ -0,0 +1,137 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ +package akka.remote.artery + +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.{ ForceDisassociate, Direction } +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 +import akka.remote.RARP + +object RemoteQuarantinePiercingSpec 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 + akka.remote.artery.enabled = on + """))) + + def aeronPort(roleName: RoleName): Int = + roleName match { + case `first` ⇒ 20551 // TODO yeah, we should have support for dynamic port assignment + case `second` ⇒ 20552 + } + + nodeConfig(first) { + ConfigFactory.parseString(s""" + akka.remote.artery.port = ${aeronPort(first)} + """) + } + + nodeConfig(second) { + ConfigFactory.parseString(s""" + akka.remote.artery.port = ${aeronPort(second)} + """) + } + + class Subject extends Actor { + def receive = { + case "shutdown" ⇒ context.system.terminate() + case "identify" ⇒ sender() ! (AddressUidExtension(context.system).addressUid -> self) + } + } + +} + +class RemoteQuarantinePiercingSpecMultiJvmNode1 extends RemoteQuarantinePiercingSpec +class RemoteQuarantinePiercingSpecMultiJvmNode2 extends RemoteQuarantinePiercingSpec + +abstract class RemoteQuarantinePiercingSpec extends MultiNodeSpec(RemoteQuarantinePiercingSpec) + with STMultiNodeSpec + with ImplicitSender { + + import RemoteQuarantinePiercingSpec._ + + override def initialParticipants = roles.size + + def identifyWithUid(role: RoleName, actorName: String, timeout: FiniteDuration = remainingOrDefault): (Int, ActorRef) = { + within(timeout) { + system.actorSelection(node(role) / "user" / actorName) ! "identify" + expectMsgType[(Int, ActorRef)] + } + } + + "RemoteNodeShutdownAndComesBack" must { + + "allow piercing through the quarantine when remote UID is new" taggedAs LongRunningTest in { + runOn(first) { + val secondAddress = node(second).address + enterBarrier("actors-started") + + // Acquire ActorRef from first system + 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)) + + // Quarantine is up -- Cannot communicate with remote system any more + system.actorSelection(RootActorPath(secondAddress) / "user" / "subject") ! "identify" + expectNoMsg(2.seconds) + + // Shut down the other system -- which results in restart (see runOn(second)) + Await.result(testConductor.shutdown(second), 30.seconds) + + // Now wait until second system becomes alive again + within(30.seconds) { + // 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) + uidSecond should not be (uidFirst) + subjectSecond should not be (subjectFirst) + } + } + + // If we got here the Quarantine was successfully pierced since it is configured to last 1 day + + system.actorSelection(RootActorPath(secondAddress) / "user" / "subject") ! "shutdown" + + } + + runOn(second) { + val addr = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + system.actorOf(Props[Subject], "subject") + enterBarrier("actors-started") + + enterBarrier("actor-identified") + + Await.ready(system.whenTerminated, 30.seconds) + + val freshSystem = ActorSystem(system.name, ConfigFactory.parseString(s""" + akka.remote.artery.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/artery/RemoteRestartedQuarantinedSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteRestartedQuarantinedSpec.scala new file mode 100644 index 0000000000..24402358bd --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteRestartedQuarantinedSpec.scala @@ -0,0 +1,169 @@ +/** + * 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.{ ForceDisassociateExplicitly, ForceDisassociate, Direction } +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 +import akka.remote.RARP +import akka.remote.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 + + # Keep it long, we don't want reconnects + akka.remote.retry-gate-closed-for = 1 s + + # Important, otherwise it is very racy to get a non-writing endpoint: the only way to do it if the two nodes + # associate to each other at the same time. Setting this will ensure that the right scenario happens. + akka.remote.use-passive-connections = off + + # TODO should not be needed, but see TODO at the end of the test + akka.remote.transport-failure-detector.heartbeat-interval = 1 s + akka.remote.transport-failure-detector.acceptable-heartbeat-pause = 10 s + + akka.remote.artery.enabled = on + """))) + + def aeronPort(roleName: RoleName): Int = + roleName match { + case `first` ⇒ 20541 // TODO yeah, we should have support for dynamic port assignment + case `second` ⇒ 20542 + } + + nodeConfig(first) { + ConfigFactory.parseString(s""" + akka.remote.artery.port = ${aeronPort(first)} + """) + } + + nodeConfig(second) { + ConfigFactory.parseString(s""" + akka.remote.artery.port = ${aeronPort(second)} + """) + } + + class Subject extends Actor { + def receive = { + case "shutdown" ⇒ context.system.terminate() + case "identify" ⇒ sender() ! (AddressUidExtension(context.system).addressUid -> self) + } + } + +} + +class RemoteRestartedQuarantinedSpecMultiJvmNode1 extends RemoteRestartedQuarantinedSpec +class RemoteRestartedQuarantinedSpecMultiJvmNode2 extends RemoteRestartedQuarantinedSpec + +abstract class RemoteRestartedQuarantinedSpec + extends MultiNodeSpec(RemoteRestartedQuarantinedSpec) + with STMultiNodeSpec with ImplicitSender { + + import RemoteRestartedQuarantinedSpec._ + + override def initialParticipants = 2 + + def identifyWithUid(role: RoleName, actorName: String, timeout: FiniteDuration = remainingOrDefault): (Int, ActorRef) = { + within(timeout) { + system.actorSelection(node(role) / "user" / actorName) ! "identify" + expectMsgType[(Int, 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)) + + 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.port = ${addr.port.get} + """).withFallback(system.settings.config)) + + val probe = TestProbe()(freshSystem) + + freshSystem.actorSelection(RootActorPath(firstAddress) / "user" / "subject").tell(Identify("subject"), probe.ref) + // TODO sometimes it takes long time until the new connection is established, + // It seems like there must first be a transport failure detector timeout, that triggers + // "No response from remote. Handshake timed out or transport failure detector triggered". + probe.expectMsgType[ActorIdentity](30.second).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/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index c8b694190a..9952a82c3a 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -13,7 +13,6 @@ import scala.concurrent.Promise import scala.concurrent.duration._ import scala.util.Failure import scala.util.Success - import akka.Done import akka.NotUsed import akka.actor.ActorRef @@ -25,12 +24,16 @@ import akka.event.Logging import akka.event.LoggingAdapter import akka.remote.AddressUidExtension import akka.remote.EndpointManager.Send +import akka.remote.EventPublisher import akka.remote.MessageSerializer import akka.remote.RemoteActorRef import akka.remote.RemoteActorRefProvider import akka.remote.RemoteTransport +import akka.remote.RemotingLifecycleEvent import akka.remote.SeqNo +import akka.remote.ThisActorSystemQuarantinedEvent import akka.remote.UniqueAddress +import akka.remote.artery.InboundControlJunction.ControlMessageObserver import akka.remote.artery.InboundControlJunction.ControlMessageSubject import akka.remote.artery.OutboundControlJunction.OutboundControlIngress import akka.remote.transport.AkkaPduCodec @@ -55,6 +58,7 @@ import io.aeron.driver.MediaDriver import io.aeron.exceptions.ConductorServiceTimeoutException import org.agrona.ErrorHandler import org.agrona.IoUtil +import scala.util.Try import java.io.File import java.net.InetSocketAddress import java.nio.channels.DatagramChannel @@ -92,11 +96,24 @@ private[akka] trait InboundContext { * Lookup the outbound association for a given address. */ def association(remoteAddress: Address): OutboundContext + } -final class AssociationState( +/** + * INTERNAL API + */ +private[akka] object AssociationState { + def apply(): AssociationState = + new AssociationState(incarnation = 1, uniqueRemoteAddressPromise = Promise(), quarantined = Set.empty) +} + +/** + * INTERNAL API + */ +private[akka] final class AssociationState( val incarnation: Int, - val uniqueRemoteAddressPromise: Promise[UniqueAddress]) { + val uniqueRemoteAddressPromise: Promise[UniqueAddress], + val quarantined: Set[Long]) { /** * Full outbound address with UID for this association. @@ -104,6 +121,33 @@ final class AssociationState( */ def uniqueRemoteAddress: Future[UniqueAddress] = uniqueRemoteAddressPromise.future + def uniqueRemoteAddressValue(): Option[Try[UniqueAddress]] = { + // FIXME we should cache access to uniqueRemoteAddress.value (avoid allocations), used in many places + uniqueRemoteAddress.value + } + + 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 + a.uid) + case _ ⇒ this + } + + def isQuarantined(): Boolean = { + uniqueRemoteAddressValue match { + case Some(Success(a)) ⇒ isQuarantined(a.uid) + case _ ⇒ false // handshake not completed yet + } + } + + def isQuarantined(uid: Long): Boolean = { + // FIXME does this mean boxing (allocation) because of Set[Long]? Use specialized Set. LongMap? + quarantined(uid) + } + override def toString(): String = { val a = uniqueRemoteAddressPromise.future.value match { case Some(Success(a)) ⇒ a @@ -134,6 +178,8 @@ private[akka] trait OutboundContext { def completeHandshake(peer: UniqueAddress): Unit + 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. @@ -166,10 +212,11 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R @volatile private[this] var driver: MediaDriver = _ @volatile private[this] var aeron: Aeron = _ - override val log: LoggingAdapter = Logging(system, getClass.getName) override def defaultAddress: Address = localAddress.address override def addresses: Set[Address] = Set(defaultAddress) override def localAddressForRemote(remote: Address): Address = defaultAddress + override val log: LoggingAdapter = Logging(system, getClass.getName) + private val eventPublisher = new EventPublisher(system, log, remoteSettings.RemoteLifecycleEventsLogLevel) private val codec: AkkaPduCodec = AkkaPduProtobufCodec private val killSwitch: SharedKillSwitch = KillSwitches.shared("transportKillSwitch") @@ -252,6 +299,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundFlows(): Unit = { + // control stream controlSubject = Source.fromGraph(new AeronSource(inboundChannel, controlStreamId, aeron, taskRunner)) .async // FIXME measure .map(ByteString.apply) // TODO we should use ByteString all the way @@ -259,6 +307,20 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .to(Sink.ignore) .run()(materializer) + controlSubject.attach(new ControlMessageObserver { + override def notify(inboundEnvelope: InboundEnvelope): Unit = { + inboundEnvelope.message match { + case Quarantined(from, to) if to == localAddress ⇒ + val lifecycleEvent = ThisActorSystemQuarantinedEvent(localAddress.address, from.address) + publishLifecycleEvent(lifecycleEvent) + // quarantine the other system from here + association(from.address).quarantine(lifecycleEvent.toString, Some(from.uid)) + case _ ⇒ // not interesting + } + } + }) + + // ordinary messages stream Source.fromGraph(new AeronSource(inboundChannel, ordinaryStreamId, aeron, taskRunner)) .async // FIXME measure .map(ByteString.apply) // TODO we should use ByteString all the way @@ -307,8 +369,11 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } } + private def publishLifecycleEvent(event: RemotingLifecycleEvent): Unit = + eventPublisher.notifyListeners(event) + override def quarantine(remoteAddress: Address, uid: Option[Int]): Unit = - association(remoteAddress).quarantine(uid) + association(remoteAddress).quarantine(reason = "", uid) // FIXME change the method signature (old remoting) to include reason? def outbound(outboundContext: OutboundContext): Sink[Send, Any] = { Flow.fromGraph(killSwitch.flow[Send]) @@ -321,11 +386,13 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R def outboundControl(outboundContext: OutboundContext): Sink[Send, OutboundControlIngress] = { Flow.fromGraph(killSwitch.flow[Send]) .via(new OutboundHandshake(outboundContext, handshakeTimeout)) - .via(new SystemMessageDelivery(outboundContext, systemMessageResendInterval)) + .via(new SystemMessageDelivery(outboundContext, systemMessageResendInterval, remoteSettings.SysMsgBufferSize)) .viaMat(new OutboundControlJunction(outboundContext))(Keep.right) .via(encoder) .map(_.toArray) // TODO we should use ByteString all the way .to(new AeronSink(outboundChannel(outboundContext.remoteAddress), controlStreamId, aeron, taskRunner)) + + // FIXME we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages } // FIXME hack until real envelopes, encoding originAddress in sender :) @@ -375,6 +442,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R decoder .via(deserializer) .via(new InboundHandshake(this, inControlStream = false)) + .via(new InboundQuarantineCheck(this)) .to(messageDispatcherSink), Source.maybe[ByteString].via(killSwitch.flow)) } @@ -384,6 +452,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R decoder .via(deserializer) .via(new InboundHandshake(this, inControlStream = true)) + .via(new InboundQuarantineCheck(this)) .viaMat(new InboundControlJunction)(Keep.right) .via(new SystemMessageAcker(this)) .to(messageDispatcherSink), diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 55e9c3b004..4c0b427441 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -3,6 +3,7 @@ */ package akka.remote.artery +import scala.annotation.tailrec import scala.concurrent.Promise import scala.util.Success import akka.actor.ActorRef @@ -24,6 +25,8 @@ import akka.util.Unsafe import java.util.concurrent.locks.ReentrantLock import java.util.concurrent.CountDownLatch import java.util.concurrent.TimeUnit +import akka.actor.ActorSelectionMessage +import akka.remote.artery.SystemMessageDelivery.ClearSystemMessageDelivery /** * INTERNAL API @@ -39,6 +42,7 @@ private[akka] class Association( extends AbstractAssociation with OutboundContext { private val log = Logging(transport.system, getClass.getName) + private val controlQueueSize = transport.provider.remoteSettings.SysMsgBufferSize @volatile private[this] var queue: SourceQueueWithComplete[Send] = _ @volatile private[this] var controlQueue: SourceQueueWithComplete[Send] = _ @@ -63,8 +67,7 @@ private[akka] class Association( * Holds reference to shared state of Association - *access only via helper methods* */ @volatile - private[this] var _sharedStateDoNotCallMeDirectly: AssociationState = - new AssociationState(incarnation = 1, uniqueRemoteAddressPromise = Promise()) + private[this] var _sharedStateDoNotCallMeDirectly: AssociationState = AssociationState() /** * Helper method for access to underlying state via Unsafe @@ -88,17 +91,17 @@ private[akka] class Association( s"wrong remote address in completeHandshake, got ${peer.address}, expected ${remoteAddress}") val current = associationState current.uniqueRemoteAddressPromise.trySuccess(peer) - current.uniqueRemoteAddress.value match { + current.uniqueRemoteAddressValue() match { case Some(Success(`peer`)) ⇒ // our value case _ ⇒ - val newState = new AssociationState(incarnation = current.incarnation + 1, Promise.successful(peer)) + val newState = current.newIncarnation(Promise.successful(peer)) if (swapState(current, newState)) { - current.uniqueRemoteAddress.value match { + current.uniqueRemoteAddressValue() match { case Some(Success(old)) ⇒ log.debug("Incarnation {} of association to [{}] with new UID [{}] (old UID [{}])", newState.incarnation, peer.address, peer.uid, old.uid) - quarantine(Some(old.uid)) - case _ ⇒ // Failed, nothing to do + case _ ⇒ + // Failed, nothing to do } // if swap failed someone else completed before us, and that is fine } @@ -110,29 +113,68 @@ private[akka] class Association( outboundControlIngress.sendControlMessage(message) def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = { - // TODO: lookup subchannel - // FIXME: Use a different envelope than the old Send, but make sure the new is handled by deadLetters properly - message match { - case _: SystemMessage ⇒ - implicit val ec = materializer.executionContext - controlQueue.offer(Send(message, senderOption, recipient, None)).onFailure { - case e ⇒ - // FIXME proper error handling, and quarantining - println(s"# System message dropped, due to $e") // FIXME - } - case _ ⇒ - queue.offer(Send(message, senderOption, recipient, None)) - } + // allow ActorSelectionMessage to pass through quarantine, to be able to establish interaction with new system + // FIXME where is that ActorSelectionMessage check in old remoting? + if (message.isInstanceOf[ActorSelectionMessage] || !associationState.isQuarantined() || message.isInstanceOf[ClearSystemMessageDelivery.type]) { + // FIXME: Use a different envelope than the old Send, but make sure the new is handled by deadLetters properly + message match { + case _: SystemMessage | ClearSystemMessageDelivery ⇒ + implicit val ec = materializer.executionContext + controlQueue.offer(Send(message, senderOption, recipient, None)).onFailure { + case e ⇒ + quarantine(reason = s"Due to overflow of control queue, size [$controlQueueSize]") + } + case _ ⇒ + queue.offer(Send(message, senderOption, recipient, None)) + } + } else if (log.isDebugEnabled) + log.debug("Dropping message to quarantined system {}", remoteAddress) } // FIXME we should be able to Send without a recipient ActorRef override val dummyRecipient: RemoteActorRef = transport.provider.resolveActorRef(RootActorPath(remoteAddress) / "system" / "dummy").asInstanceOf[RemoteActorRef] - def quarantine(uid: Option[Int]): Unit = { - // FIXME implement - log.error("Association to [{}] with UID [{}] is irrecoverably failed. Quarantining address.", - remoteAddress, uid.getOrElse("unknown")) + // OutboundContext + override def quarantine(reason: String): Unit = { + val uid = associationState.uniqueRemoteAddressValue() match { + case Some(Success(a)) ⇒ Some(a.uid) + case _ ⇒ None + } + quarantine(reason, uid) + } + + @tailrec final def quarantine(reason: String, uid: Option[Int]): Unit = { + uid match { + case Some(u) ⇒ + val current = associationState + current.uniqueRemoteAddressValue() match { + case Some(Success(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. Quarantining address. {}", + remoteAddress, u, reason) + // end delivery of system messages to that incarnation after this point + send(ClearSystemMessageDelivery, None, dummyRecipient) + // try to tell the other system that we have quarantined it + sendControl(Quarantined(localAddress, peer)) + } else + quarantine(reason, uid) // recursive + } + case Some(Success(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 ⇒ + // FIXME should we do something more, old impl used gating? + log.warning("Quarantine of [{}] ignored because unknown UID", remoteAddress) + } + } // Idempotent @@ -142,7 +184,7 @@ private[akka] class Association( // it's important to materialize the outboundControl stream first, // so that outboundControlIngress is ready when stages for all streams start if (controlQueue eq null) { - val (q, control) = Source.queue(256, OverflowStrategy.dropBuffer) + val (q, control) = Source.queue(controlQueueSize, OverflowStrategy.backpressure) .toMat(transport.outboundControl(this))(Keep.both) .run()(materializer) controlQueue = q diff --git a/akka-remote/src/main/scala/akka/remote/artery/Control.scala b/akka-remote/src/main/scala/akka/remote/artery/Control.scala index ae3b51c3bd..b371ae2650 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Control.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Control.scala @@ -4,10 +4,8 @@ package akka.remote.artery import java.util.ArrayDeque - import scala.concurrent.Future import scala.concurrent.Promise - import akka.Done import akka.remote.EndpointManager.Send import akka.stream.Attributes @@ -19,17 +17,24 @@ import akka.stream.stage.GraphStageLogic import akka.stream.stage.GraphStageWithMaterializedValue import akka.stream.stage.InHandler import akka.stream.stage.OutHandler +import akka.remote.UniqueAddress /** - * Marker trait for reply messages + * INTERNAL API: Marker trait for reply messages */ -trait Reply extends ControlMessage +private[akka] 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`. */ -trait ControlMessage +private[akka] trait ControlMessage + +/** + * INTERNAL API + */ +private[akka] final case class Quarantined(from: UniqueAddress, to: UniqueAddress) extends ControlMessage // FIXME serialization /** * INTERNAL API diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala index a838e133dd..a889acac38 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -159,7 +159,7 @@ private[akka] class InboundHandshake(inboundContext: InboundContext, inControlSt private def isKnownOrigin(originAddress: UniqueAddress): Boolean = { // FIXME these association lookups are probably too costly for each message, need local cache or something val associationState = inboundContext.association(originAddress.address).associationState - associationState.uniqueRemoteAddress.value match { + associationState.uniqueRemoteAddressValue() match { case Some(Success(a)) if a.uid == originAddress.uid ⇒ true case x ⇒ false } 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..cf0a916ee9 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala @@ -0,0 +1,44 @@ +/** + * 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 + +/** + * INTERNAL API + */ +private[akka] 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 { + + // InHandler + override def onPush(): Unit = { + val env = grab(in) + val association = inboundContext.association(env.originAddress.address) + if (association.associationState.isQuarantined(env.originAddress.uid)) { + inboundContext.sendControl(env.originAddress.address, + Quarantined(inboundContext.localAddress, env.originAddress)) + pull(in) + } else + 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/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala index 4fa7a2d23f..32a31ee111 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -35,6 +35,8 @@ private[akka] object SystemMessageDelivery { final case class Ack(seqNo: Long, from: UniqueAddress) extends Reply final case class Nack(seqNo: Long, from: UniqueAddress) extends Reply + final case object ClearSystemMessageDelivery + private case object ResendTick } @@ -43,7 +45,8 @@ private[akka] object SystemMessageDelivery { */ private[akka] class SystemMessageDelivery( outboundContext: OutboundContext, - resendInterval: FiniteDuration) + resendInterval: FiniteDuration, + maxBufferSize: Int) extends GraphStage[FlowShape[Send, Send]] { import SystemMessageDelivery._ @@ -120,16 +123,17 @@ private[akka] class SystemMessageDelivery( } private val nackCallback = getAsyncCallback[Nack] { reply ⇒ - ack(reply.seqNo) - if (reply.seqNo > resendingFromSeqNo) - resending = unacknowledged.clone() - tryResend() + if (reply.seqNo <= seqNo) { + ack(reply.seqNo) + if (reply.seqNo > resendingFromSeqNo) + resending = unacknowledged.clone() + tryResend() + } } private def ack(n: Long): Unit = { - if (n > seqNo) - throw new IllegalArgumentException(s"Unexpected ack $n, when highest sent seqNo is $seqNo") - clearUnacknowledged(n) + if (n <= seqNo) + clearUnacknowledged(n) } @tailrec private def clearUnacknowledged(ackedSeqNo: Long): Unit = { @@ -151,20 +155,35 @@ private[akka] class SystemMessageDelivery( // InHandler override def onPush(): Unit = { grab(in) match { + case s @ Send(ClearSystemMessageDelivery, _, _, _) ⇒ + clear() + pull(in) case s @ Send(msg: AnyRef, _, _, _) ⇒ - seqNo += 1 - val sendMsg = s.copy(message = SystemMessageEnvelope(msg, seqNo, localAddress)) - // FIXME quarantine if unacknowledged is full - unacknowledged.offer(sendMsg) - if (resending.isEmpty && isAvailable(out)) - push(out, sendMsg) - else { - resending.offer(sendMsg) - tryResend() + if (unacknowledged.size < maxBufferSize) { + seqNo += 1 + val sendMsg = s.copy(message = SystemMessageEnvelope(msg, seqNo, localAddress)) + unacknowledged.offer(sendMsg) + if (resending.isEmpty && isAvailable(out)) + push(out, sendMsg) + else { + resending.offer(sendMsg) + tryResend() + } + } else { + // buffer overflow + outboundContext.quarantine(reason = s"System message delivery buffer overflow, size [$maxBufferSize]") + pull(in) } } } + private def clear(): Unit = { + seqNo = 0L // sequence number for the first message will be 1 + unacknowledged.clear() + resending.clear() + resendingFromSeqNo = -1L + } + // OutHandler override def onPull(): Unit = { if (replyObserverAttached) { // otherwise it will be pulled after attached diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala index ea1b460711..bf7cfcf8bc 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala @@ -52,7 +52,7 @@ class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { "send HandshakeRsp as reply to HandshakeReq" in { val replyProbe = TestProbe() - val inboundContext = new ManualReplyInboundContext(replyProbe.ref, addressB, new TestControlMessageSubject) + val inboundContext = new TestInboundContext(addressB, controlProbe = Some(replyProbe.ref)) val (upstream, downstream) = setupStream(inboundContext) downstream.request(10) @@ -77,9 +77,9 @@ class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { downstream.cancel() } - "send HandshakeReq as when receiving message from unknown (receiving system restarted)" in { + "send HandshakeReq when receiving message from unknown (receiving system restarted)" in { val replyProbe = TestProbe() - val inboundContext = new ManualReplyInboundContext(replyProbe.ref, addressB, new TestControlMessageSubject) + val inboundContext = new TestInboundContext(addressB, controlProbe = Some(replyProbe.ref)) val (upstream, downstream) = setupStream(inboundContext) downstream.request(10) diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index 62f1ecd768..940f2bd62f 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -74,7 +74,7 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commo val remoteRef = null.asInstanceOf[RemoteActorRef] // not used Source(1 to sendCount) .map(n ⇒ Send("msg-" + n, None, remoteRef, None)) - .via(new SystemMessageDelivery(outboundContext, resendInterval)) + .via(new SystemMessageDelivery(outboundContext, resendInterval, maxBufferSize = 1000)) } private def inbound(inboundContext: InboundContext): Flow[Send, InboundEnvelope, NotUsed] = { diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala index d1af1446ea..30ed97698f 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -54,7 +54,7 @@ private[akka] class TestOutboundContext( val controlProbe: Option[ActorRef] = None) extends OutboundContext { // access to this is synchronized (it's a test utility) - private var _associationState = new AssociationState(1, Promise()) + private var _associationState = AssociationState() override def associationState: AssociationState = synchronized { _associationState @@ -65,10 +65,14 @@ private[akka] class TestOutboundContext( _associationState.uniqueRemoteAddress.value match { case Some(Success(`peer`)) ⇒ // our value case _ ⇒ - _associationState = new AssociationState(incarnation = _associationState.incarnation + 1, Promise.successful(peer)) + _associationState = _associationState.newIncarnation(Promise.successful(peer)) } } + override def quarantine(reason: String): Unit = synchronized { + _associationState = _associationState.newQuarantined() + } + override def sendControl(message: ControlMessage) = { controlProbe.foreach(_ ! message) controlSubject.sendControl(InboundEnvelope(null, remoteAddress, message, None, localAddress)) From 8e316d06e9360d7a0306915eccac5053e52e79ef Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 17 May 2016 14:05:50 +0200 Subject: [PATCH 027/186] harden resend sys msg test * the periodic resend kicked in too early, resending 2 before the Ack(2) was delivered * also improved SystemMessageDelivery stage to not tick periodically when there are no messages to deliver (buffer empty) --- .../scala/akka/remote/artery/SystemMessageDelivery.scala | 9 +++++++-- .../akka/remote/artery/SystemMessageDeliverySpec.scala | 7 +++++-- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala index 32a31ee111..9fa61c25dd 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -69,8 +69,6 @@ private[akka] class SystemMessageDelivery( private def remoteAddress = outboundContext.remoteAddress override def preStart(): Unit = { - this.schedulePeriodically(ResendTick, resendInterval) - implicit val ec = materializer.executionContext outboundContext.controlSubject.attach(this).foreach { getAsyncCallback[Done] { _ ⇒ @@ -107,6 +105,8 @@ private[akka] class SystemMessageDelivery( resending = unacknowledged.clone() tryResend() } + if (!unacknowledged.isEmpty) + scheduleOnce(ResendTick, resendInterval) } // ControlMessageObserver, external call @@ -140,6 +140,9 @@ private[akka] class SystemMessageDelivery( if (!unacknowledged.isEmpty && unacknowledged.peek().message.asInstanceOf[SystemMessageEnvelope].seqNo <= ackedSeqNo) { unacknowledged.removeFirst() + if (unacknowledged.isEmpty) + cancelTimer(resendInterval) + if (stopping && unacknowledged.isEmpty) completeStage() else @@ -163,6 +166,7 @@ private[akka] class SystemMessageDelivery( seqNo += 1 val sendMsg = s.copy(message = SystemMessageEnvelope(msg, seqNo, localAddress)) unacknowledged.offer(sendMsg) + scheduleOnce(ResendTick, resendInterval) if (resending.isEmpty && isAvailable(out)) push(out, sendMsg) else { @@ -182,6 +186,7 @@ private[akka] class SystemMessageDelivery( unacknowledged.clear() resending.clear() resendingFromSeqNo = -1L + cancelTimer(resendInterval) } // OutHandler diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index 940f2bd62f..280f04455b 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -195,7 +195,7 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commo val inboundContextA = new TestInboundContext(addressB, controlSubject) val outboundContextA = inboundContextA.association(addressB.address) - val sink = send(sendCount = 3, resendInterval = 1.seconds, outboundContextA) + val sink = send(sendCount = 3, resendInterval = 2.seconds, outboundContextA) .via(drop(dropSeqNumbers = Vector(3L))) .via(inbound(inboundContextB)) .map(_.message.asInstanceOf[String]) @@ -211,8 +211,11 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commo sink.expectNoMsg(200.millis) // 3 was dropped // resending 3 due to timeout sink.expectNext("msg-3") - replyProbe.expectMsg(Ack(3L, addressB)) + replyProbe.expectMsg(4.seconds, Ack(3L, addressB)) + // continue resending + replyProbe.expectMsg(4.seconds, Ack(3L, addressB)) inboundContextB.deliverLastReply() + replyProbe.expectNoMsg(2200.millis) sink.expectComplete() } From 96b697d92f9eedc133111af345b031cf15021cdd Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 17 May 2016 17:34:57 +0200 Subject: [PATCH 028/186] handle stream failures by restarting, #20317 * allow X restarts within Y seconds * and handle handshake timeout --- .../scala/akka/remote/artery/AeronSink.scala | 29 +++++-- .../akka/remote/artery/ArteryTransport.scala | 86 ++++++++++++++++--- .../akka/remote/artery/Association.scala | 86 ++++++++++++++----- .../scala/akka/remote/artery/Handshake.scala | 29 +++++-- .../akka/remote/artery/RestartCounter.scala | 52 +++++++++++ .../remote/artery/HandshakeFailureSpec.scala | 68 +++++++++++++++ .../remote/artery/HandshakeRetrySpec.scala | 62 +++++++++++++ .../remote/artery/OutboundHandshakeSpec.scala | 23 +++-- .../remote/artery/RestartCounterSpec.scala | 42 +++++++++ 9 files changed, 425 insertions(+), 52 deletions(-) create mode 100644 akka-remote/src/main/scala/akka/remote/artery/RestartCounter.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/RestartCounterSpec.scala diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala index 590035ea01..9807591bc1 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala @@ -4,18 +4,23 @@ package akka.remote.artery import java.nio.ByteBuffer -import java.util.concurrent.TimeUnit 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 akka.Done import akka.stream.Attributes import akka.stream.Inlet import akka.stream.SinkShape import akka.stream.stage.AsyncCallback -import akka.stream.stage.GraphStage import akka.stream.stage.GraphStageLogic +import akka.stream.stage.GraphStageWithMaterializedValue import akka.stream.stage.InHandler import io.aeron.Aeron import io.aeron.Publication @@ -51,19 +56,23 @@ object AeronSink { /** * @param channel eg. "aeron:udp?endpoint=localhost:40123" */ -class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRunner) extends GraphStage[SinkShape[AeronSink.Bytes]] { +class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRunner) + extends GraphStageWithMaterializedValue[SinkShape[AeronSink.Bytes], Future[Done]] { import AeronSink._ import TaskRunner._ val in: Inlet[Bytes] = Inlet("AeronSink") override val shape: SinkShape[Bytes] = SinkShape(in) - override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = - new GraphStageLogic(shape) with InHandler { + override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, Future[Done]) = { + val completed = Promise[Done]() + val logic = new GraphStageLogic(shape) with InHandler { private val buffer = new UnsafeBuffer(ByteBuffer.allocateDirect(128 * 1024)) private val pub = aeron.addPublication(channel, streamId) + private var completedValue: Try[Done] = Success(Done) + private val spinning = 1000 private var backoffCount = spinning private var lastMsgSize = 0 @@ -80,6 +89,7 @@ class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRu override def postStop(): Unit = { taskRunner.command(Remove(addOfferTask.task)) pub.close() + completed.complete(completedValue) } // InHandler @@ -124,6 +134,15 @@ class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRu 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/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 9952a82c3a..a9473a9d9f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -13,6 +13,7 @@ import scala.concurrent.Promise import scala.concurrent.duration._ import scala.util.Failure import scala.util.Success +import scala.util.Try import akka.Done import akka.NotUsed import akka.actor.ActorRef @@ -39,6 +40,7 @@ import akka.remote.artery.OutboundControlJunction.OutboundControlIngress import akka.remote.transport.AkkaPduCodec import akka.remote.transport.AkkaPduProtobufCodec import akka.serialization.Serialization +import akka.stream.AbruptTerminationException import akka.stream.ActorMaterializer import akka.stream.KillSwitches import akka.stream.Materializer @@ -50,6 +52,8 @@ import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Source import akka.util.ByteString import akka.util.ByteStringBuilder +import akka.util.Helpers.ConfigOps +import akka.util.Helpers.Requiring import io.aeron.Aeron import io.aeron.AvailableImageHandler import io.aeron.Image @@ -58,7 +62,6 @@ import io.aeron.driver.MediaDriver import io.aeron.exceptions.ConductorServiceTimeoutException import org.agrona.ErrorHandler import org.agrona.IoUtil -import scala.util.Try import java.io.File import java.net.InetSocketAddress import java.nio.channels.DatagramChannel @@ -156,6 +159,7 @@ private[akka] final class AssociationState( } s"AssociationState($incarnation, $a)" } + } /** @@ -220,10 +224,14 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val codec: AkkaPduCodec = AkkaPduProtobufCodec private val killSwitch: SharedKillSwitch = KillSwitches.shared("transportKillSwitch") + @volatile private[this] var _shutdown = false // FIXME config private val systemMessageResendInterval: FiniteDuration = 1.second - private val handshakeTimeout: FiniteDuration = 10.seconds + private val handshakeRetryInterval: FiniteDuration = 1.second + private val handshakeTimeout: FiniteDuration = + system.settings.config.getMillisDuration("akka.remote.handshake-timeout").requiring(_ > Duration.Zero, + "handshake-timeout must be > 0") private def inboundChannel = s"aeron:udp?endpoint=${localAddress.address.host.get}:${localAddress.address.port.get}" private def outboundChannel(a: Address) = s"aeron:udp?endpoint=${a.host.get}:${a.port.get}" @@ -234,6 +242,10 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // FIXME: This does locking on putIfAbsent, we need something smarter private[this] val associations = new ConcurrentHashMap[Address, Association]() + private val restartTimeout: FiniteDuration = 5.seconds // FIXME config + private val maxRestarts = 5 // FIXME config + private val restartCounter = new RestartCounter(maxRestarts, restartTimeout) + override def start(): Unit = { startMediaDriver() startAeron() @@ -252,7 +264,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R messageDispatcher = new MessageDispatcher(system, provider) - runInboundFlows() + runInboundStreams() } private def startMediaDriver(): Unit = { @@ -298,14 +310,19 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R aeron = Aeron.connect(ctx) } - private def runInboundFlows(): Unit = { - // control stream - controlSubject = Source.fromGraph(new AeronSource(inboundChannel, controlStreamId, aeron, taskRunner)) + private def runInboundStreams(): Unit = { + runInboundControlStream() + runInboundOrdinaryMessagesStream() + } + + private def runInboundControlStream(): Unit = { + val (c, completed) = Source.fromGraph(new AeronSource(inboundChannel, controlStreamId, aeron, taskRunner)) .async // FIXME measure .map(ByteString.apply) // TODO we should use ByteString all the way .viaMat(inboundControlFlow)(Keep.right) - .to(Sink.ignore) + .toMat(Sink.ignore)(Keep.both) .run()(materializer) + controlSubject = c controlSubject.attach(new ControlMessageObserver { override def notify(inboundEnvelope: InboundEnvelope): Unit = { @@ -321,14 +338,51 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R }) // ordinary messages stream - Source.fromGraph(new AeronSource(inboundChannel, ordinaryStreamId, aeron, taskRunner)) + controlSubject.attach(new ControlMessageObserver { + override def notify(inboundEnvelope: InboundEnvelope): Unit = { + inboundEnvelope.message match { + case Quarantined(from, to) if to == localAddress ⇒ + val lifecycleEvent = ThisActorSystemQuarantinedEvent(localAddress.address, from.address) + publishLifecycleEvent(lifecycleEvent) + // quarantine the other system from here + association(from.address).quarantine(lifecycleEvent.toString, Some(from.uid)) + case _ ⇒ // not interesting + } + } + }) + + attachStreamRestart("Inbound control stream", completed, () ⇒ runInboundControlStream()) + } + + private def runInboundOrdinaryMessagesStream(): Unit = { + val completed = Source.fromGraph(new AeronSource(inboundChannel, ordinaryStreamId, aeron, taskRunner)) .async // FIXME measure .map(ByteString.apply) // TODO we should use ByteString all the way .via(inboundFlow) .runWith(Sink.ignore)(materializer) + + attachStreamRestart("Inbound message stream", completed, () ⇒ runInboundOrdinaryMessagesStream()) + } + + private def attachStreamRestart(streamName: String, streamCompleted: Future[Done], restart: () ⇒ Unit): Unit = { + implicit val ec = materializer.executionContext + streamCompleted.onFailure { + case _: AbruptTerminationException ⇒ // ActorSystem shutdown + case cause ⇒ + if (!isShutdown) + if (restartCounter.restart()) { + log.error(cause, "{} failed. Restarting it.", streamName) + restart() + } else { + log.error(cause, "{} failed and restarted {} times within {} seconds. Terminating system.", + streamName, maxRestarts, restartTimeout.toSeconds) + system.terminate() + } + } } override def shutdown(): Future[Done] = { + _shutdown = true killSwitch.shutdown() if (taskRunner != null) taskRunner.stop() if (aeron != null) aeron.close() @@ -340,6 +394,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R Future.successful(Done) } + private[remote] def isShutdown(): Boolean = _shutdown + // InboundContext override def sendControl(to: Address, message: ControlMessage) = association(to).outboundControlIngress.sendControlMessage(message) @@ -375,22 +431,24 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R override def quarantine(remoteAddress: Address, uid: Option[Int]): Unit = association(remoteAddress).quarantine(reason = "", uid) // FIXME change the method signature (old remoting) to include reason? - def outbound(outboundContext: OutboundContext): Sink[Send, Any] = { + def outbound(outboundContext: OutboundContext): Sink[Send, Future[Done]] = { Flow.fromGraph(killSwitch.flow[Send]) - .via(new OutboundHandshake(outboundContext, handshakeTimeout)) + .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval)) .via(encoder) .map(_.toArray) // TODO we should use ByteString all the way - .to(new AeronSink(outboundChannel(outboundContext.remoteAddress), ordinaryStreamId, aeron, taskRunner)) + .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), ordinaryStreamId, aeron, taskRunner))(Keep.right) } - def outboundControl(outboundContext: OutboundContext): Sink[Send, OutboundControlIngress] = { + def outboundControl(outboundContext: OutboundContext): Sink[Send, (OutboundControlIngress, Future[Done])] = { Flow.fromGraph(killSwitch.flow[Send]) - .via(new OutboundHandshake(outboundContext, handshakeTimeout)) + .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval)) .via(new SystemMessageDelivery(outboundContext, systemMessageResendInterval, remoteSettings.SysMsgBufferSize)) .viaMat(new OutboundControlJunction(outboundContext))(Keep.right) .via(encoder) .map(_.toArray) // TODO we should use ByteString all the way - .to(new AeronSink(outboundChannel(outboundContext.remoteAddress), controlStreamId, aeron, taskRunner)) + .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), controlStreamId, aeron, taskRunner))(Keep.both) + + // FIXME we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages // FIXME we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 4c0b427441..219dc15906 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -3,10 +3,19 @@ */ package akka.remote.artery +import java.util.concurrent.CountDownLatch +import java.util.concurrent.TimeUnit + import scala.annotation.tailrec +import scala.concurrent.Future import scala.concurrent.Promise +import scala.concurrent.duration._ +import scala.concurrent.duration.FiniteDuration import scala.util.Success + +import akka.Done import akka.actor.ActorRef +import akka.actor.ActorSelectionMessage import akka.actor.Address import akka.actor.RootActorPath import akka.dispatch.sysmsg.SystemMessage @@ -16,17 +25,15 @@ import akka.remote.RemoteActorRef import akka.remote.UniqueAddress 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.stream.AbruptTerminationException import akka.stream.Materializer import akka.stream.OverflowStrategy import akka.stream.scaladsl.Keep import akka.stream.scaladsl.Source import akka.stream.scaladsl.SourceQueueWithComplete import akka.util.Unsafe -import java.util.concurrent.locks.ReentrantLock -import java.util.concurrent.CountDownLatch -import java.util.concurrent.TimeUnit -import akka.actor.ActorSelectionMessage -import akka.remote.artery.SystemMessageDelivery.ClearSystemMessageDelivery /** * INTERNAL API @@ -44,10 +51,14 @@ private[akka] class Association( private val log = Logging(transport.system, getClass.getName) private val controlQueueSize = transport.provider.remoteSettings.SysMsgBufferSize + private val restartTimeout: FiniteDuration = 5.seconds // FIXME config + private val maxRestarts = 5 // FIXME config + private val restartCounter = new RestartCounter(maxRestarts, restartTimeout) + @volatile private[this] var queue: SourceQueueWithComplete[Send] = _ @volatile private[this] var controlQueue: SourceQueueWithComplete[Send] = _ @volatile private[this] var _outboundControlIngress: OutboundControlIngress = _ - private val materializing = new CountDownLatch(1) + @volatile private[this] var materializing = new CountDownLatch(1) def outboundControlIngress: OutboundControlIngress = { if (_outboundControlIngress ne null) @@ -115,7 +126,7 @@ private[akka] class Association( def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = { // allow ActorSelectionMessage to pass through quarantine, to be able to establish interaction with new system // FIXME where is that ActorSelectionMessage check in old remoting? - if (message.isInstanceOf[ActorSelectionMessage] || !associationState.isQuarantined() || message.isInstanceOf[ClearSystemMessageDelivery.type]) { + if (message.isInstanceOf[ActorSelectionMessage] || !associationState.isQuarantined() || message == ClearSystemMessageDelivery) { // FIXME: Use a different envelope than the old Send, but make sure the new is handled by deadLetters properly message match { case _: SystemMessage | ClearSystemMessageDelivery ⇒ @@ -179,22 +190,55 @@ private[akka] class Association( // Idempotent def associate(): Unit = { - // FIXME detect and handle stream failure, e.g. handshake timeout - - // it's important to materialize the outboundControl stream first, - // so that outboundControlIngress is ready when stages for all streams start if (controlQueue eq null) { - val (q, control) = Source.queue(controlQueueSize, OverflowStrategy.backpressure) - .toMat(transport.outboundControl(this))(Keep.both) - .run()(materializer) - controlQueue = q - _outboundControlIngress = control - // stage in the control stream may access the outboundControlIngress before returned here - // using CountDownLatch to make sure that materialization is completed before accessing outboundControlIngress - materializing.countDown() + // it's important to materialize the outboundControl stream first, + // so that outboundControlIngress is ready when stages for all streams start + runOutboundControlStream() + runOutboundOrdinaryMessagesStream() + } + } - queue = Source.queue(256, OverflowStrategy.dropBuffer) - .to(transport.outbound(this)).run()(materializer) + private def runOutboundControlStream(): Unit = { + // stage in the control stream may access the outboundControlIngress before returned here + // using CountDownLatch to make sure that materialization is completed before accessing outboundControlIngress + materializing = new CountDownLatch(1) + val (q, (control, completed)) = Source.queue(controlQueueSize, OverflowStrategy.backpressure) + .toMat(transport.outboundControl(this))(Keep.both) + .run()(materializer) + controlQueue = q + _outboundControlIngress = control + materializing.countDown() + attachStreamRestart("Outbound control stream", completed, cause ⇒ { + runOutboundControlStream() + cause match { + case _: HandshakeTimeoutException ⇒ // ok, quarantine not possible without UID + case _ ⇒ quarantine("Outbound control stream restarted") + } + }) + } + + private def runOutboundOrdinaryMessagesStream(): Unit = { + val (q, completed) = Source.queue(256, OverflowStrategy.dropBuffer) + .toMat(transport.outbound(this))(Keep.both) + .run()(materializer) + queue = q + attachStreamRestart("Outbound message stream", completed, _ ⇒ runOutboundOrdinaryMessagesStream()) + } + + private def attachStreamRestart(streamName: String, streamCompleted: Future[Done], restart: Throwable ⇒ Unit): Unit = { + implicit val ec = materializer.executionContext + streamCompleted.onFailure { + case _: AbruptTerminationException ⇒ // ActorSystem shutdown + case cause ⇒ + if (!transport.isShutdown) + if (restartCounter.restart()) { + log.error(cause, "{} failed. Restarting it.", streamName) + restart(cause) + } else { + log.error(cause, "{} failed and restarted {} times within {} seconds. Terminating system.", + streamName, maxRestarts, restartTimeout.toSeconds) + transport.system.terminate() + } } } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala index a889acac38..cb2fe64f24 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -3,10 +3,9 @@ */ package akka.remote.artery -import java.util.concurrent.TimeoutException - import scala.concurrent.duration._ import scala.util.Success +import scala.util.control.NoStackTrace import akka.remote.EndpointManager.Send import akka.remote.UniqueAddress @@ -24,6 +23,13 @@ import akka.stream.stage.TimerGraphStageLogic * INTERNAL API */ private[akka] 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 + // FIXME serialization for these messages final case class HandshakeReq(from: UniqueAddress) extends ControlMessage final case class HandshakeRsp(from: UniqueAddress) extends Reply @@ -34,13 +40,16 @@ private[akka] object OutboundHandshake { private case object Completed extends HandshakeState private case object HandshakeTimeout + private case object HandshakeRetryTick } /** * INTERNAL API */ -private[akka] class OutboundHandshake(outboundContext: OutboundContext, timeout: FiniteDuration) extends GraphStage[FlowShape[Send, Send]] { +private[akka] class OutboundHandshake(outboundContext: OutboundContext, timeout: FiniteDuration, retryInterval: FiniteDuration) + extends GraphStage[FlowShape[Send, Send]] { + val in: Inlet[Send] = Inlet("OutboundHandshake.in") val out: Outlet[Send] = Outlet("OutboundHandshake.out") override val shape: FlowShape[Send, Send] = FlowShape(in, out) @@ -68,8 +77,6 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext, timeout: } }.invoke } - - scheduleOnce(HandshakeTimeout, timeout) } } @@ -87,21 +94,29 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext, timeout: case Start ⇒ // will pull when handshake reply is received (uniqueRemoteAddress completed) handshakeState = ReqInProgress - outboundContext.sendControl(HandshakeReq(outboundContext.localAddress)) + scheduleOnce(HandshakeTimeout, timeout) + schedulePeriodically(HandshakeRetryTick, retryInterval) + sendHandshakeReq() case ReqInProgress ⇒ // will pull when handshake reply is received } } + private def sendHandshakeReq(): Unit = + outboundContext.sendControl(HandshakeReq(outboundContext.localAddress)) + private def handshakeCompleted(): Unit = { handshakeState = Completed + cancelTimer(HandshakeRetryTick) cancelTimer(HandshakeTimeout) } override protected def onTimer(timerKey: Any): Unit = timerKey match { + case HandshakeRetryTick ⇒ + sendHandshakeReq() case HandshakeTimeout ⇒ // FIXME would it make sense to retry a few times before failing? - failStage(new TimeoutException( + failStage(new HandshakeTimeoutException( s"Handshake with [${outboundContext.remoteAddress}] did not complete within ${timeout.toMillis} ms")) } 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..6f23b239d7 --- /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[akka] object RestartCounter { + final case class State(count: Int, deadline: Deadline) +} + +/** + * INTERNAL API: Thread safe "restarts with duration" counter + */ +private[akka] 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/test/scala/akka/remote/artery/HandshakeFailureSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala new file mode 100644 index 0000000000..08475939fd --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala @@ -0,0 +1,68 @@ +/** + * 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 + +object HandshakeFailureSpec { + + val Seq(portA, portB) = SocketUtil.temporaryServerAddresses(2, "localhost", udp = true).map(_.getPort) + + val commonConfig = ConfigFactory.parseString(s""" + akka { + actor.provider = "akka.remote.RemoteActorRefProvider" + remote.artery.enabled = on + remote.artery.hostname = localhost + remote.artery.port = $portA + remote.handshake-timeout = 2s + } + """) + + val configB = ConfigFactory.parseString(s"akka.remote.artery.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.artery://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 { + println(s"# identify $sel") // FIXME + sel ! "hello2" + 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..df22b2cb84 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala @@ -0,0 +1,62 @@ +/** + * 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 + +object HandshakeRetrySpec { + + val Seq(portA, portB) = SocketUtil.temporaryServerAddresses(2, "localhost", udp = true).map(_.getPort) + + val commonConfig = ConfigFactory.parseString(s""" + akka { + actor.provider = "akka.remote.RemoteActorRefProvider" + remote.artery.enabled = on + remote.artery.hostname = localhost + remote.artery.port = $portA + remote.handshake-timeout = 10s + } + """) + + val configB = ConfigFactory.parseString(s"akka.remote.artery.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.artery://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/OutboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala index 3e9d73e0f5..2580a18dec 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala @@ -3,8 +3,6 @@ */ package akka.remote.artery -import java.util.concurrent.TimeoutException - import scala.concurrent.duration._ import akka.actor.Address @@ -12,6 +10,7 @@ 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 @@ -32,11 +31,12 @@ class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { val addressA = UniqueAddress(Address("akka.artery", "sysA", "hostA", 1001), 1) val addressB = UniqueAddress(Address("akka.artery", "sysB", "hostB", 1002), 2) - private def setupStream(outboundContext: OutboundContext, timeout: FiniteDuration = 5.seconds): (TestPublisher.Probe[String], TestSubscriber.Probe[Any]) = { + private def setupStream(outboundContext: OutboundContext, timeout: FiniteDuration = 5.seconds, + retryInterval: FiniteDuration = 10.seconds): (TestPublisher.Probe[String], TestSubscriber.Probe[Any]) = { val destination = null.asInstanceOf[RemoteActorRef] // not used TestSource.probe[String] .map(msg ⇒ Send(msg, None, destination, None)) - .via(new OutboundHandshake(outboundContext, timeout)) + .via(new OutboundHandshake(outboundContext, timeout, retryInterval)) .map { case Send(msg, _, _, _) ⇒ msg } .toMat(TestSink.probe[Any])(Keep.both) .run() @@ -60,7 +60,20 @@ class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { val (upstream, downstream) = setupStream(outboundContext, timeout = 200.millis) downstream.request(1) - downstream.expectError().getClass should be(classOf[TimeoutException]) + downstream.expectError().getClass should be(classOf[HandshakeTimeoutException]) + } + + "retry HandshakeReq" in { + val controlProbe = TestProbe() + val inboundContext = new TestInboundContext(localAddress = addressA, controlProbe = Some(controlProbe.ref)) + val outboundContext = inboundContext.association(addressB.address) + val (upstream, downstream) = setupStream(outboundContext, retryInterval = 100.millis) + + downstream.request(10) + controlProbe.expectMsg(HandshakeReq(addressA)) + controlProbe.expectMsg(HandshakeReq(addressA)) + controlProbe.expectMsg(HandshakeReq(addressA)) + downstream.cancel() } "not deliver messages from upstream until handshake completed" in { 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) + } + } +} From 935ab0d82ee6d2643723f705014ce7d2a2ce651c Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 18 May 2016 09:22:22 +0200 Subject: [PATCH 029/186] make use of auto port selection in tests --- .../PiercingShouldKeepQuarantineSpec.scala | 18 ------------------ .../artery/RemoteQuarantinePiercingSpec.scala | 18 ------------------ .../RemoteRestartedQuarantinedSpec.scala | 18 ------------------ .../akka/remote/artery/ArteryTransport.scala | 2 ++ .../remote/artery/HandshakeFailureSpec.scala | 6 +++--- .../remote/artery/HandshakeRetrySpec.scala | 5 +++-- .../artery/RemoteSendConsistencySpec.scala | 15 ++++----------- .../artery/SystemMessageDeliverySpec.scala | 13 ++++--------- 8 files changed, 16 insertions(+), 79 deletions(-) diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/PiercingShouldKeepQuarantineSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/PiercingShouldKeepQuarantineSpec.scala index fd292f172f..cb17da59d2 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/PiercingShouldKeepQuarantineSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/PiercingShouldKeepQuarantineSpec.scala @@ -25,24 +25,6 @@ object PiercingShouldKeepQuarantineSpec extends MultiNodeConfig { akka.remote.artery.enabled = on """))) - def aeronPort(roleName: RoleName): Int = - roleName match { - case `first` ⇒ 20561 // TODO yeah, we should have support for dynamic port assignment - case `second` ⇒ 20562 - } - - nodeConfig(first) { - ConfigFactory.parseString(s""" - akka.remote.artery.port = ${aeronPort(first)} - """) - } - - nodeConfig(second) { - ConfigFactory.parseString(s""" - akka.remote.artery.port = ${aeronPort(second)} - """) - } - class Subject extends Actor { def receive = { case "getuid" ⇒ sender() ! AddressUidExtension(context.system).addressUid diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteQuarantinePiercingSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteQuarantinePiercingSpec.scala index e546a293e6..299857cfc5 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteQuarantinePiercingSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteQuarantinePiercingSpec.scala @@ -31,24 +31,6 @@ object RemoteQuarantinePiercingSpec extends MultiNodeConfig { akka.remote.artery.enabled = on """))) - def aeronPort(roleName: RoleName): Int = - roleName match { - case `first` ⇒ 20551 // TODO yeah, we should have support for dynamic port assignment - case `second` ⇒ 20552 - } - - nodeConfig(first) { - ConfigFactory.parseString(s""" - akka.remote.artery.port = ${aeronPort(first)} - """) - } - - nodeConfig(second) { - ConfigFactory.parseString(s""" - akka.remote.artery.port = ${aeronPort(second)} - """) - } - class Subject extends Actor { def receive = { case "shutdown" ⇒ context.system.terminate() 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 index 24402358bd..b22d35c0fb 100644 --- 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 @@ -45,24 +45,6 @@ object RemoteRestartedQuarantinedSpec extends MultiNodeConfig { akka.remote.artery.enabled = on """))) - def aeronPort(roleName: RoleName): Int = - roleName match { - case `first` ⇒ 20541 // TODO yeah, we should have support for dynamic port assignment - case `second` ⇒ 20542 - } - - nodeConfig(first) { - ConfigFactory.parseString(s""" - akka.remote.artery.port = ${aeronPort(first)} - """) - } - - nodeConfig(second) { - ConfigFactory.parseString(s""" - akka.remote.artery.port = ${aeronPort(second)} - """) - } - class Subject extends Actor { def receive = { case "shutdown" ⇒ context.system.terminate() diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index a9473a9d9f..0ef73d7d47 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -265,6 +265,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R messageDispatcher = new MessageDispatcher(system, provider) runInboundStreams() + + log.info("Remoting started; listening on address: {}", defaultAddress) } private def startMediaDriver(): Unit = { diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala index 08475939fd..d350151041 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala @@ -13,14 +13,15 @@ import com.typesafe.config.ConfigFactory object HandshakeFailureSpec { - val Seq(portA, portB) = SocketUtil.temporaryServerAddresses(2, "localhost", udp = true).map(_.getPort) + // need the port before systemB is started + val portB = SocketUtil.temporaryServerAddress("localhost", udp = true).getPort val commonConfig = ConfigFactory.parseString(s""" akka { actor.provider = "akka.remote.RemoteActorRefProvider" remote.artery.enabled = on remote.artery.hostname = localhost - remote.artery.port = $portA + remote.artery.port = 0 remote.handshake-timeout = 2s } """) @@ -47,7 +48,6 @@ class HandshakeFailureSpec extends AkkaSpec(HandshakeFailureSpec.commonConfig) w within(10.seconds) { awaitAssert { - println(s"# identify $sel") // FIXME sel ! "hello2" expectMsg(1.second, "hello2") } diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala index df22b2cb84..a334efa642 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala @@ -13,14 +13,15 @@ import com.typesafe.config.ConfigFactory object HandshakeRetrySpec { - val Seq(portA, portB) = SocketUtil.temporaryServerAddresses(2, "localhost", udp = true).map(_.getPort) + // need the port before systemB is started + val portB = SocketUtil.temporaryServerAddress("localhost", udp = true).getPort val commonConfig = ConfigFactory.parseString(s""" akka { actor.provider = "akka.remote.RemoteActorRefProvider" remote.artery.enabled = on remote.artery.hostname = localhost - remote.artery.port = $portA + remote.artery.port = 0 remote.handshake-timeout = 10s } """) diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala index c94e21ad64..a7256705a8 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala @@ -7,31 +7,24 @@ 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 RemoteSendConsistencySpec._ import akka.actor.Actor.Receive -import akka.testkit.SocketUtil object RemoteSendConsistencySpec { - val Seq(portA, portB) = SocketUtil.temporaryServerAddresses(2, "localhost", udp = true).map(_.getPort) - - val commonConfig = ConfigFactory.parseString(s""" + val config = ConfigFactory.parseString(s""" akka { actor.provider = "akka.remote.RemoteActorRefProvider" remote.artery.enabled = on remote.artery.hostname = localhost - remote.artery.port = $portA + remote.artery.port = 0 } """) - val configB = ConfigFactory.parseString(s"akka.remote.artery.port = $portB") - .withFallback(commonConfig) - } -class RemoteSendConsistencySpec extends AkkaSpec(commonConfig) with ImplicitSender { +class RemoteSendConsistencySpec extends AkkaSpec(RemoteSendConsistencySpec.config) with ImplicitSender { - val systemB = ActorSystem("systemB", RemoteSendConsistencySpec.configB) + val systemB = ActorSystem("systemB", system.settings.config) val addressB = systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress println(addressB) val rootB = RootActorPath(addressB) diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index 280f04455b..55fe0b941f 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -30,37 +30,32 @@ import akka.stream.scaladsl.Source import akka.stream.testkit.scaladsl.TestSink 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 SystemMessageDeliverySpec { - val Seq(portA, portB) = SocketUtil.temporaryServerAddresses(2, "localhost", udp = true).map(_.getPort) - - val commonConfig = ConfigFactory.parseString(s""" + val config = ConfigFactory.parseString(s""" akka { actor.provider = "akka.remote.RemoteActorRefProvider" remote.artery.enabled = on remote.artery.hostname = localhost - remote.artery.port = $portA + remote.artery.port = 0 } akka.actor.serialize-creators = off akka.actor.serialize-messages = off """) - val configB = ConfigFactory.parseString(s"akka.remote.artery.port = $portB") - .withFallback(commonConfig) } -class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.commonConfig) with ImplicitSender { +class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.config) with ImplicitSender { import SystemMessageDeliverySpec._ val addressA = UniqueAddress( system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress, AddressUidExtension(system).addressUid) - val systemB = ActorSystem("systemB", configB) + val systemB = ActorSystem("systemB", system.settings.config) val addressB = UniqueAddress( systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress, AddressUidExtension(systemB).addressUid) From 76b29a35e0e4ae5c44344c05c6c07e2d84732685 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 18 May 2016 13:34:51 +0200 Subject: [PATCH 030/186] log Aeron errors, #20317 --- .../akka/remote/artery/AeronErrorLog.java | 76 +++++++++++++++++++ .../akka/remote/artery/ArteryTransport.scala | 18 +++++ 2 files changed, 94 insertions(+) create mode 100644 akka-remote/src/main/java/akka/remote/artery/AeronErrorLog.java 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..edee08f987 --- /dev/null +++ b/akka-remote/src/main/java/akka/remote/artery/AeronErrorLog.java @@ -0,0 +1,76 @@ +/* + * 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; + + public AeronErrorLog(File cncFile) + { + this.cncFile = cncFile; + } + + public long logErrors(LoggingAdapter log, long sinceTimestamp) + { + final MappedByteBuffer cncByteBuffer = IoUtil.mapExistingFile(cncFile, "cnc"); + final DirectBuffer cncMetaDataBuffer = CncFileDescriptor.createMetaDataBuffer(cncByteBuffer); + final int cncVersion = cncMetaDataBuffer.getInt(CncFileDescriptor.cncVersionOffset(0)); + + if (CncFileDescriptor.CNC_VERSION != cncVersion) + { + throw new IllegalStateException("CNC version not supported: file version=" + cncVersion); + } + + final AtomicBuffer buffer = CncFileDescriptor.createErrorLogBuffer(cncByteBuffer, cncMetaDataBuffer); + final SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSSZ"); + + // 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(); + } +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 0ef73d7d47..f732f3d4d7 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -67,6 +67,9 @@ import java.net.InetSocketAddress import java.nio.channels.DatagramChannel import akka.remote.artery.OutboundControlJunction.OutboundControlIngress +import io.aeron.CncFileDescriptor +import java.util.concurrent.atomic.AtomicLong +import akka.actor.Cancellable /** * INTERNAL API @@ -215,6 +218,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R @volatile private[this] var messageDispatcher: MessageDispatcher = _ @volatile private[this] var driver: MediaDriver = _ @volatile private[this] var aeron: Aeron = _ + @volatile private[this] var aeronErrorLogTask: Cancellable = _ override def defaultAddress: Address = localAddress.address override def addresses: Set[Address] = Set(defaultAddress) @@ -249,6 +253,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R override def start(): Unit = { startMediaDriver() startAeron() + startAeronErrorLog() taskRunner.start() val port = @@ -312,6 +317,18 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R aeron = Aeron.connect(ctx) } + private def startAeronErrorLog(): Unit = { + val errorLog = new AeronErrorLog(new File(driver.aeronDirectoryName, CncFileDescriptor.CNC_FILE)) + val lastTimestamp = new AtomicLong(0L) + import system.dispatcher // FIXME perhaps use another dispatcher for this + aeronErrorLogTask = system.scheduler.schedule(3.seconds, 5.seconds) { + if (!isShutdown) { + val newLastTimestamp = errorLog.logErrors(log, lastTimestamp.get) + lastTimestamp.set(newLastTimestamp + 1) + } + } + } + private def runInboundStreams(): Unit = { runInboundControlStream() runInboundOrdinaryMessagesStream() @@ -387,6 +404,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R _shutdown = true killSwitch.shutdown() if (taskRunner != null) taskRunner.stop() + if (aeronErrorLogTask != null) aeronErrorLogTask.cancel() if (aeron != null) aeron.close() if (driver != null) { driver.close() From 1828a8def3e7fb29edf9f6b19847a1b83ae77c88 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Endre=20S=C3=A1ndor=20Varga?= Date: Thu, 5 May 2016 14:38:48 +0200 Subject: [PATCH 031/186] Buffer and header parsing support --- .../artery/AeronStreamConcistencySpec.scala | 35 +- .../artery/AeronStreamLatencySpec.scala | 32 +- .../artery/AeronStreamMaxThroughputSpec.scala | 20 +- .../scala/akka/remote/MessageSerializer.scala | 33 +- .../scala/akka/remote/artery/AeronSink.scala | 66 ++-- .../akka/remote/artery/AeronSource.scala | 31 +- .../akka/remote/artery/ArteryTransport.scala | 71 ++-- .../scala/akka/remote/artery/BufferPool.scala | 316 ++++++++++++++++++ .../scala/akka/remote/artery/Codecs.scala | 103 ++++++ .../akka/remote/artery/Compression.scala | 23 ++ .../akka/remote/artery/AeronStreamsApp.scala | 72 ++-- .../remote/artery/EnvelopeBufferSpec.scala | 169 ++++++++++ 12 files changed, 828 insertions(+), 143 deletions(-) create mode 100644 akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/Codecs.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/Compression.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala 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 index c0edac7986..526f321bd0 100644 --- 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 @@ -4,6 +4,7 @@ package akka.remote.artery import java.util.concurrent.atomic.AtomicInteger + import scala.concurrent.Await import scala.concurrent.duration._ import akka.Done @@ -23,6 +24,8 @@ 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") @@ -65,6 +68,8 @@ abstract class AeronStreamConsistencySpec r } + val pool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) + lazy implicit val mat = ActorMaterializer()(system) import system.dispatcher @@ -90,8 +95,8 @@ abstract class AeronStreamConsistencySpec "start echo" in { runOn(second) { // just echo back - Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner)) - .runWith(new AeronSink(channel(first), streamId, aeron, taskRunner)) + Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner, pool)) + .runWith(new AeronSink(channel(first), streamId, aeron, taskRunner, pool)) } enterBarrier("echo-started") } @@ -104,35 +109,47 @@ abstract class AeronStreamConsistencySpec val killSwitch = KillSwitches.shared("test") val started = TestProbe() val startMsg = "0".getBytes("utf-8") - Source.fromGraph(new AeronSource(channel(first), streamId, aeron, taskRunner)) + Source.fromGraph(new AeronSource(channel(first), streamId, aeron, taskRunner, pool)) .via(killSwitch.flow) - .runForeach { bytes ⇒ + .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, "utf-8").toInt + 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(_ ⇒ startMsg) + 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)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool)) started.expectMsg(Done) } Source(1 to totalMessages) .throttle(10000, 1.second, 1000, ThrottleMode.Shaping) - .map { n ⇒ n.toString.getBytes("utf-8") } - .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner)) + .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)) Await.ready(done, 20.seconds) killSwitch.shutdown() 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 index d364a52cae..6ce14f0b3b 100644 --- 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 @@ -7,6 +7,7 @@ import java.util.concurrent.CyclicBarrier import java.util.concurrent.Executors import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicLongArray + import scala.concurrent.duration._ import akka.actor._ import akka.remote.testconductor.RoleName @@ -22,11 +23,14 @@ import io.aeron.Aeron import io.aeron.driver.MediaDriver import org.HdrHistogram.Histogram import java.util.concurrent.atomic.AtomicBoolean + import akka.stream.KillSwitches import akka.Done import org.agrona.IoUtil import java.io.File import java.io.File + +import akka.util.ByteString import io.aeron.CncFileDescriptor object AeronStreamLatencySpec extends MultiNodeConfig { @@ -76,6 +80,8 @@ abstract class AeronStreamLatencySpec val driver = MediaDriver.launchEmbedded() + val pool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) + val stats = new AeronStat(AeronStat.mapCounters(new File(driver.aeronDirectoryName, CncFileDescriptor.CNC_FILE))) @@ -193,9 +199,10 @@ abstract class AeronStreamLatencySpec val killSwitch = KillSwitches.shared(testName) val started = TestProbe() val startMsg = "0".getBytes("utf-8") - Source.fromGraph(new AeronSource(channel(first), streamId, aeron, taskRunner)) + Source.fromGraph(new AeronSource(channel(first), streamId, aeron, taskRunner, pool)) .via(killSwitch.flow) - .runForeach { bytes ⇒ + .runForeach { envelope ⇒ + val bytes = ByteString.fromByteBuffer(envelope.byteBuffer) if (bytes.length == 1 && bytes(0) == startMsg(0)) started.ref ! Done else { @@ -209,12 +216,18 @@ abstract class AeronStreamLatencySpec barrier.await() // this is always the last party } } + pool.release(envelope) } within(10.seconds) { - Source(1 to 50).map(_ ⇒ startMsg) + 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)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool)) started.expectMsg(Done) } @@ -226,10 +239,13 @@ abstract class AeronStreamLatencySpec Source(1 to totalMessages) .throttle(messageRate, 1.second, math.max(messageRate / 10, 1), ThrottleMode.Shaping) .map { n ⇒ + val envelope = pool.acquire() + envelope.byteBuffer.put(payload) + envelope.byteBuffer.flip() sendTimes.set(n - 1, System.nanoTime()) - payload + envelope } - .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool)) barrier.await((totalMessages / messageRate) + 10, SECONDS) } @@ -247,8 +263,8 @@ abstract class AeronStreamLatencySpec "start echo" in { runOn(second) { // just echo back - Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner)) - .runWith(new AeronSink(channel(first), streamId, aeron, taskRunner)) + Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner, pool)) + .runWith(new AeronSink(channel(first), streamId, aeron, taskRunner, pool)) } enterBarrier("echo-started") } 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 index e9356cad08..e4db9dca62 100644 --- 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 @@ -5,6 +5,7 @@ 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._ @@ -21,6 +22,8 @@ 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 @@ -81,6 +84,8 @@ abstract class AeronStreamMaxThroughputSpec val driver = MediaDriver.launchEmbedded() + val pool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) + val stats = new AeronStat(AeronStat.mapCounters(new File(driver.aeronDirectoryName, CncFileDescriptor.CNC_FILE))) @@ -168,9 +173,10 @@ abstract class AeronStreamMaxThroughputSpec var count = 0L val done = TestLatch(1) val killSwitch = KillSwitches.shared(testName) - Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner)) + Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner, pool)) .via(killSwitch.flow) - .runForeach { bytes ⇒ + .runForeach { envelope ⇒ + val bytes = ByteString.fromByteBuffer(envelope.byteBuffer) rep.onMessage(1, bytes.length) count += 1 if (count == 1) { @@ -180,6 +186,7 @@ abstract class AeronStreamMaxThroughputSpec done.countDown() killSwitch.shutdown() } + pool.release(envelope) }.onFailure { case e ⇒ e.printStackTrace @@ -198,8 +205,13 @@ abstract class AeronStreamMaxThroughputSpec val payload = ("0" * payloadSize).getBytes("utf-8") val t0 = System.nanoTime() Source.fromIterator(() ⇒ iterate(1, totalMessages)) - .map { n ⇒ payload } - .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner)) + .map { n ⇒ + val envelope = pool.acquire() + envelope.byteBuffer.put(payload) + envelope.byteBuffer.flip() + envelope + } + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool)) printStats("sender") enterBarrier(testName + "-done") diff --git a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala index e222adacd5..e11627a078 100644 --- a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala @@ -7,8 +7,8 @@ package akka.remote import akka.remote.WireFormats._ import akka.protobuf.ByteString import akka.actor.ExtendedActorSystem -import akka.serialization.SerializationExtension -import akka.serialization.SerializerWithStringManifest +import akka.remote.artery.{ EnvelopeBuffer, HeaderBuilder } +import akka.serialization.{ Serialization, SerializationExtension, SerializerWithStringManifest } /** * INTERNAL API @@ -47,4 +47,33 @@ private[akka] object MessageSerializer { } builder.build } + + def serializeForArtery(serialization: Serialization, message: AnyRef, headerBuilder: HeaderBuilder, envelope: EnvelopeBuffer): Unit = { + val serializer = serialization.findSerializerFor(message) + + // FIXME: This should be a FQCN instead + headerBuilder.serializer = serializer.identifier.toString + serializer match { + case ser2: SerializerWithStringManifest ⇒ + val manifest = ser2.manifest(message) + headerBuilder.classManifest = manifest + case _ ⇒ + headerBuilder.classManifest = message.getClass.getName + } + + envelope.writeHeader(headerBuilder) + // FIXME: This should directly write to the buffer instead + envelope.byteBuffer.put(serializer.toBinary(message)) + } + + def deserializeForArtery(system: ExtendedActorSystem, headerBuilder: HeaderBuilder, envelope: EnvelopeBuffer): AnyRef = { + // FIXME: Use the buffer directly + val size = envelope.byteBuffer.limit - envelope.byteBuffer.position + val bytes = Array.ofDim[Byte](size) + envelope.byteBuffer.get(bytes) + SerializationExtension(system).deserialize( + bytes, + Integer.parseInt(headerBuilder.serializer), // FIXME: Use FQCN + headerBuilder.classManifest).get + } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala index 9807591bc1..611809a449 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala @@ -27,48 +27,49 @@ import io.aeron.Publication import org.agrona.concurrent.UnsafeBuffer object AeronSink { - type Bytes = Array[Byte] - private def offerTask(pub: Publication, buffer: UnsafeBuffer, msgSize: AtomicInteger, onOfferSuccess: AsyncCallback[Unit]): () ⇒ Boolean = { + class OfferTask(pub: Publication, var buffer: UnsafeBuffer, msgSize: AtomicInteger, onOfferSuccess: AsyncCallback[Unit]) + extends (() ⇒ Boolean) { + var n = 0L var localMsgSize = -1 - () ⇒ - { - n += 1 - if (localMsgSize == -1) - localMsgSize = msgSize.get - val result = pub.offer(buffer, 0, localMsgSize) - if (result >= 0) { - n = 0 - localMsgSize = -1 - onOfferSuccess.invoke(()) - true - } else { - // FIXME drop after too many attempts? - if (n > 1000000 && n % 100000 == 0) - println(s"# offer not accepted after $n") // FIXME - false - } + + override def apply(): Boolean = { + n += 1 + if (localMsgSize == -1) + localMsgSize = msgSize.get + val result = pub.offer(buffer, 0, localMsgSize) + if (result >= 0) { + n = 0 + localMsgSize = -1 + onOfferSuccess.invoke(()) + true + } else { + // FIXME drop after too many attempts? + if (n > 1000000 && n % 100000 == 0) + println(s"# offer not accepted after $n") // FIXME + false } + } } } /** * @param channel eg. "aeron:udp?endpoint=localhost:40123" */ -class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRunner) - extends GraphStageWithMaterializedValue[SinkShape[AeronSink.Bytes], Future[Done]] { +class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRunner, pool: EnvelopeBufferPool) + extends GraphStageWithMaterializedValue[SinkShape[EnvelopeBuffer], Future[Done]] { import AeronSink._ import TaskRunner._ - val in: Inlet[Bytes] = Inlet("AeronSink") - override val shape: SinkShape[Bytes] = SinkShape(in) + 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 val buffer = new UnsafeBuffer(ByteBuffer.allocateDirect(128 * 1024)) + private var envelopeInFlight: EnvelopeBuffer = null private val pub = aeron.addPublication(channel, streamId) private var completedValue: Try[Done] = Success(Done) @@ -76,8 +77,9 @@ class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRu private val spinning = 1000 private var backoffCount = spinning private var lastMsgSize = 0 - private var lastMsgSizeRef = new AtomicInteger // used in the external backoff task - private val addOfferTask: Add = Add(offerTask(pub, buffer, lastMsgSizeRef, getAsyncCallback(_ ⇒ onOfferSuccess()))) + private val lastMsgSizeRef = new AtomicInteger // used in the external backoff task + private val offerTask = new OfferTask(pub, null, lastMsgSizeRef, getAsyncCallback(_ ⇒ onOfferSuccess())) + private val addOfferTask: Add = Add(offerTask) private var offerTaskInProgress = false @@ -94,15 +96,14 @@ class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRu // InHandler override def onPush(): Unit = { - val msg = grab(in) - buffer.putBytes(0, msg); + envelopeInFlight = grab(in) backoffCount = spinning - lastMsgSize = msg.length + lastMsgSize = envelopeInFlight.byteBuffer.limit publish() } @tailrec private def publish(): Unit = { - val result = pub.offer(buffer, 0, lastMsgSize) + val result = pub.offer(envelopeInFlight.aeronBuffer, 0, lastMsgSize) // FIXME handle Publication.CLOSED // TODO the backoff strategy should be measured and tuned if (result < 0) { @@ -113,6 +114,7 @@ class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRu // delegate backoff to shared TaskRunner lastMsgSizeRef.set(lastMsgSize) offerTaskInProgress = true + offerTask.buffer = envelopeInFlight.aeronBuffer taskRunner.command(addOfferTask) } } else { @@ -122,6 +124,10 @@ class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRu private def onOfferSuccess(): Unit = { offerTaskInProgress = false + pool.release(envelopeInFlight) + offerTask.buffer = null + envelopeInFlight = null + if (isClosed(in)) completeStage() else diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala index c03aa69dea..48af59f14d 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala @@ -24,9 +24,8 @@ import org.agrona.DirectBuffer import org.agrona.concurrent.BackoffIdleStrategy object AeronSource { - type Bytes = Array[Byte] - private def pollTask(sub: Subscription, handler: MessageHandler, onMessage: AsyncCallback[Bytes]): () ⇒ Boolean = { + private def pollTask(sub: Subscription, handler: MessageHandler, onMessage: AsyncCallback[EnvelopeBuffer]): () ⇒ Boolean = { () ⇒ { handler.reset @@ -41,19 +40,20 @@ object AeronSource { } } - class MessageHandler { + class MessageHandler(pool: EnvelopeBufferPool) { def reset(): Unit = messageReceived = null - var messageReceived: Bytes = null + var messageReceived: EnvelopeBuffer = null - val fragmentsHandler = new Fragments(data ⇒ messageReceived = data) + val fragmentsHandler = new Fragments(data ⇒ messageReceived = data, pool) } - class Fragments(onMessage: Bytes ⇒ Unit) extends FragmentAssembler(new FragmentHandler { - override def onFragment(buffer: DirectBuffer, offset: Int, length: Int, header: Header): Unit = { - val data = Array.ofDim[Byte](length) - buffer.getBytes(offset, data) - onMessage(data) + 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) } }) } @@ -61,12 +61,13 @@ object AeronSource { /** * @param channel eg. "aeron:udp?endpoint=localhost:40123" */ -class AeronSource(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRunner) extends GraphStage[SourceShape[AeronSource.Bytes]] { +class AeronSource(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRunner, pool: EnvelopeBufferPool) + extends GraphStage[SourceShape[EnvelopeBuffer]] { import AeronSource._ import TaskRunner._ - val out: Outlet[Bytes] = Outlet("AeronSource") - override val shape: SourceShape[Bytes] = SourceShape(out) + val out: Outlet[EnvelopeBuffer] = Outlet("AeronSource") + override val shape: SourceShape[EnvelopeBuffer] = SourceShape(out) override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) with OutHandler { @@ -81,7 +82,7 @@ class AeronSource(channel: String, streamId: Int, aeron: Aeron, taskRunner: Task private var backoffCount = idleStrategyRetries // the fragmentHandler is called from `poll` in same thread, i.e. no async callback is needed - private val messageHandler = new MessageHandler + private val messageHandler = new MessageHandler(pool) private val addPollTask: Add = Add(pollTask(sub, messageHandler, getAsyncCallback(onMessage))) override def postStop(): Unit = { @@ -119,7 +120,7 @@ class AeronSource(channel: String, streamId: Int, aeron: Aeron, taskRunner: Task } } - private def onMessage(data: Bytes): Unit = { + private def onMessage(data: EnvelopeBuffer): Unit = { push(out, data) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 0ef73d7d47..05aa1aa075 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -246,6 +246,11 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val maxRestarts = 5 // FIXME config private val restartCounter = new RestartCounter(maxRestarts, restartTimeout) + val envelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) + // FIXME: Compression table must be owned by each channel instead + // of having a global one + val compression = new Compression(system) + override def start(): Unit = { startMediaDriver() startAeron() @@ -318,9 +323,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundControlStream(): Unit = { - val (c, completed) = Source.fromGraph(new AeronSource(inboundChannel, controlStreamId, aeron, taskRunner)) + val (c, completed) = Source.fromGraph(new AeronSource(inboundChannel, controlStreamId, aeron, taskRunner, envelopePool)) .async // FIXME measure - .map(ByteString.apply) // TODO we should use ByteString all the way .viaMat(inboundControlFlow)(Keep.right) .toMat(Sink.ignore)(Keep.both) .run()(materializer) @@ -357,9 +361,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundOrdinaryMessagesStream(): Unit = { - val completed = Source.fromGraph(new AeronSource(inboundChannel, ordinaryStreamId, aeron, taskRunner)) + val completed = Source.fromGraph(new AeronSource(inboundChannel, ordinaryStreamId, aeron, taskRunner, envelopePool)) .async // FIXME measure - .map(ByteString.apply) // TODO we should use ByteString all the way .via(inboundFlow) .runWith(Sink.ignore)(materializer) @@ -437,8 +440,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R Flow.fromGraph(killSwitch.flow[Send]) .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval)) .via(encoder) - .map(_.toArray) // TODO we should use ByteString all the way - .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), ordinaryStreamId, aeron, taskRunner))(Keep.right) + .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), ordinaryStreamId, aeron, taskRunner, envelopePool))(Keep.right) } def outboundControl(outboundContext: OutboundContext): Sink[Send, (OutboundControlIngress, Future[Done])] = { @@ -447,10 +449,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .via(new SystemMessageDelivery(outboundContext, systemMessageResendInterval, remoteSettings.SysMsgBufferSize)) .viaMat(new OutboundControlJunction(outboundContext))(Keep.right) .via(encoder) - .map(_.toArray) // TODO we should use ByteString all the way - .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), controlStreamId, aeron, taskRunner))(Keep.both) - - // FIXME we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages + .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), controlStreamId, aeron, taskRunner, envelopePool))(Keep.both) // FIXME we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages } @@ -458,59 +457,27 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // FIXME hack until real envelopes, encoding originAddress in sender :) private val dummySender = system.systemActorOf(Props.empty, "dummy") - // TODO: Try out parallelized serialization (mapAsync) for performance - val encoder: Flow[Send, ByteString, NotUsed] = Flow[Send].map { sendEnvelope ⇒ - val pdu: ByteString = codec.constructMessage( - sendEnvelope.recipient.localAddressToUse, - sendEnvelope.recipient, - Serialization.currentTransportInformation.withValue(Serialization.Information(localAddress.address, system)) { - MessageSerializer.serialize(system, sendEnvelope.message.asInstanceOf[AnyRef]) - }, - if (sendEnvelope.senderOption.isDefined) sendEnvelope.senderOption else Some(dummySender), // FIXME: hack until real envelopes - seqOption = Some(SeqNo(localAddress.uid)), // FIXME: hack until real envelopes - ackOption = None) - - // TODO: Drop unserializable messages - // TODO: Drop oversized messages - (new ByteStringBuilder).putInt(pdu.size)(ByteOrder.LITTLE_ENDIAN).result() ++ pdu - } - - val decoder: Flow[ByteString, AkkaPduCodec.Message, NotUsed] = - Framing.lengthField(4, maximumFrameLength = 256000) - .map { frame ⇒ - // TODO: Drop unserializable messages - val pdu = codec.decodeMessage(frame.drop(4), provider, localAddress.address)._2.get - pdu - } + val encoder: Flow[Send, EnvelopeBuffer, NotUsed] = + Flow.fromGraph(new Encoder(this, compression)) val messageDispatcherSink: Sink[InboundEnvelope, Future[Done]] = Sink.foreach[InboundEnvelope] { m ⇒ messageDispatcher.dispatch(m.recipient, m.recipientAddress, m.message, m.senderOption) } - val deserializer: Flow[AkkaPduCodec.Message, InboundEnvelope, NotUsed] = - Flow[AkkaPduCodec.Message].map { m ⇒ - InboundEnvelope( - m.recipient, - m.recipientAddress, - MessageSerializer.deserialize(system, m.serializedMessage), - if (m.senderOption.get.path.name == "dummy") None else m.senderOption, // FIXME hack until real envelopes - UniqueAddress(m.senderOption.get.path.address, m.seq.rawValue.toInt)) // FIXME hack until real envelopes - } + val decoder = Flow.fromGraph(new Decoder(this, compression)) - val inboundFlow: Flow[ByteString, ByteString, NotUsed] = { + val inboundFlow: Flow[EnvelopeBuffer, ByteString, NotUsed] = { Flow.fromSinkAndSource( decoder - .via(deserializer) .via(new InboundHandshake(this, inControlStream = false)) .via(new InboundQuarantineCheck(this)) .to(messageDispatcherSink), Source.maybe[ByteString].via(killSwitch.flow)) } - val inboundControlFlow: Flow[ByteString, ByteString, ControlMessageSubject] = { + val inboundControlFlow: Flow[EnvelopeBuffer, ByteString, ControlMessageSubject] = { Flow.fromSinkAndSourceMat( decoder - .via(deserializer) .via(new InboundHandshake(this, inControlStream = true)) .via(new InboundQuarantineCheck(this)) .viaMat(new InboundControlJunction)(Keep.right) @@ -521,10 +488,18 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } -object ArteryTransport { +/** + * INTERNAL API + */ +private[remote] object ArteryTransport { + + val Version = 0 + val MaximumFrameSize = 1024 * 1024 + val MaximumPooledBuffers = 256 /** * Internal API + * * @return A port that is hopefully available */ private[remote] def autoSelectPort(hostname: String): Int = { 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..901391673d --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -0,0 +1,316 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ + +package akka.remote.artery + +import java.lang.reflect.Field +import java.nio.charset.Charset +import java.nio.{ ByteBuffer, ByteOrder } + +import org.agrona.concurrent.{ ManyToManyConcurrentArrayQueue, UnsafeBuffer } +import sun.misc.Cleaner + +import scala.util.control.NonFatal + +/** + * 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.tryForceDrop() + +} + +/** + * INTERNAL API + */ +private[remote] object EnvelopeBuffer { + + val TagTypeMask = 0xFF000000 + val TagValueMask = 0x0000FFFF + + val VersionOffset = 0 + val UidOffset = 4 + val SenderActorRefTagOffset = 8 + val RecipientActorRefTagOffset = 12 + val SerializerTagOffset = 16 + val ClassManifestTagOffset = 24 + + val LiteralsSectionOffset = 32 + + val UsAscii = Charset.forName("US-ASCII") + + val DeadLettersCode = 0 +} + +/** + * INTERNAL API + */ +private[remote] trait LiteralCompressionTable { + + def compressActorRef(ref: String): Int + def decompressActorRef(idx: Int): String + + def compressSerializer(serializer: String): Int + def decompressSerializer(idx: Int): String + + def compressClassManifest(manifest: String): Int + def decompressClassManifest(idx: Int): String + +} + +object HeaderBuilder { + def apply(compressionTable: LiteralCompressionTable): HeaderBuilder = new HeaderBuilderImpl(compressionTable) +} + +/** + * INTERNAL API + */ +sealed trait HeaderBuilder { + def version_=(v: Int): Unit + def version: Int + + def uid_=(u: Int): Unit + def uid: Int + + def senderActorRef_=(ref: String): Unit + def senderActorRef: String + + def setNoSender(): Unit + + def recipientActorRef_=(ref: String): Unit + def recipientActorRef: String + + def serializer_=(serializer: String): Unit + def serializer: String + + def classManifest_=(manifest: String): Unit + def classManifest: String +} + +/** + * INTERNAL API + */ +private[remote] final class HeaderBuilderImpl(val compressionTable: LiteralCompressionTable) extends HeaderBuilder { + var version: Int = _ + var uid: Int = _ + + // Fields only available for EnvelopeBuffer + var _senderActorRef: String = null + var _senderActorRefIdx: Int = -1 + var _recipientActorRef: String = null + var _recipientActorRefIdx: Int = -1 + + var _serializer: String = null + var _serializerIdx: Int = -1 + var _classManifest: String = null + var _classManifestIdx: Int = -1 + + def senderActorRef_=(ref: String): Unit = { + _senderActorRef = ref + _senderActorRefIdx = compressionTable.compressActorRef(ref) + } + + def setNoSender(): Unit = { + _senderActorRef = null + _senderActorRefIdx = EnvelopeBuffer.DeadLettersCode + } + + def senderActorRef: String = { + if (_senderActorRef ne null) _senderActorRef + else { + _senderActorRef = compressionTable.decompressActorRef(_senderActorRefIdx) + _senderActorRef + } + } + + def recipientActorRef_=(ref: String): Unit = { + _recipientActorRef = ref + _recipientActorRefIdx = compressionTable.compressActorRef(ref) + } + + def recipientActorRef: String = { + if (_recipientActorRef ne null) _recipientActorRef + else { + _recipientActorRef = compressionTable.decompressActorRef(_recipientActorRefIdx) + _recipientActorRef + } + } + + override def serializer_=(serializer: String): Unit = { + _serializer = serializer + _serializerIdx = compressionTable.compressSerializer(serializer) + } + + override def serializer: String = { + if (_serializer ne null) _serializer + else { + _serializer = compressionTable.decompressSerializer(_serializerIdx) + _serializer + } + } + + override def classManifest_=(manifest: String): Unit = { + _classManifest = manifest + _classManifestIdx = compressionTable.compressClassManifest(manifest) + } + + override def classManifest: String = { + if (_classManifest ne null) _classManifest + else { + _classManifest = compressionTable.decompressClassManifest(_classManifestIdx) + _classManifest + } + } + + override def toString = s"HeaderBuilderImpl($version, $uid, ${_senderActorRef}, ${_senderActorRefIdx}, ${_recipientActorRef}, ${_recipientActorRefIdx}, ${_serializer}, ${_serializerIdx}, ${_classManifest}, ${_classManifestIdx})" +} + +/** + * INTERNAL API + */ +private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { + import EnvelopeBuffer._ + val aeronBuffer = new UnsafeBuffer(byteBuffer) + + private val cleanerField: Field = try { + val cleaner = byteBuffer.getClass.getDeclaredField("cleaner") + cleaner.setAccessible(true) + cleaner + } catch { + case NonFatal(_) ⇒ null + } + + def tryForceDrop(): Unit = { + if (cleanerField ne null) cleanerField.get(byteBuffer) match { + case cleaner: Cleaner ⇒ cleaner.clean() + case _ ⇒ + } + } + + def writeHeader(h: HeaderBuilder): Unit = { + val header = h.asInstanceOf[HeaderBuilderImpl] + byteBuffer.clear() + + // Write fixed length parts + byteBuffer.putInt(header.version) + byteBuffer.putInt(header.uid) + + // Write compressable, variable-length parts always to the actual position of the buffer + // Write tag values explicitly in their proper offset + byteBuffer.position(LiteralsSectionOffset) + + // 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 serializer + if (header._serializerIdx != -1) + byteBuffer.putInt(SerializerTagOffset, header._serializerIdx | TagTypeMask) + else + writeLiteral(SerializerTagOffset, header._serializer) + + // Serialize class manifest + if (header._classManifestIdx != -1) + byteBuffer.putInt(ClassManifestTagOffset, header._classManifestIdx | TagTypeMask) + else + writeLiteral(ClassManifestTagOffset, header._classManifest) + } + + def parseHeader(h: HeaderBuilder): Unit = { + val header = h.asInstanceOf[HeaderBuilderImpl] + + // Read fixed length parts + header.version = byteBuffer.getInt + header.uid = byteBuffer.getInt + + // Read compressable, variable-length parts always from the actual position of the buffer + // Read tag values explicitly from their proper offset + byteBuffer.position(LiteralsSectionOffset) + + // Deserialize sender + val senderTag = byteBuffer.getInt(SenderActorRefTagOffset) + if ((senderTag & TagTypeMask) != 0) { + val idx = senderTag & TagValueMask + header._senderActorRef = null + header._senderActorRefIdx = idx + } else { + header._senderActorRef = 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 = readLiteral() + } + + // Deserialize serializer + val serializerTag = byteBuffer.getInt(SerializerTagOffset) + if ((serializerTag & TagTypeMask) != 0) { + val idx = serializerTag & TagValueMask + header._serializer = null + header._serializerIdx = idx + } else { + header._serializer = readLiteral() + } + + // Deserialize class manifest + val classManifestTag = byteBuffer.getInt(ClassManifestTagOffset) + if ((classManifestTag & TagTypeMask) != 0) { + val idx = classManifestTag & TagValueMask + header._classManifest = null + header._classManifestIdx = idx + } else { + header._classManifest = readLiteral() + } + } + + private def readLiteral(): String = { + val length = byteBuffer.getShort + val bytes = Array.ofDim[Byte](length) + byteBuffer.get(bytes) + new String(bytes, UsAscii) + } + + private def writeLiteral(tagOffset: Int, literal: String): Unit = { + if (literal.length > 65535) + throw new IllegalArgumentException("Literals longer than 65535 cannot be encoded in the envelope") + + val literalBytes = literal.getBytes(UsAscii) + byteBuffer.putInt(tagOffset, byteBuffer.position()) + byteBuffer.putShort(literalBytes.length.toShort) + byteBuffer.put(literalBytes) + } + +} \ No newline at end of file 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..db412d7862 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -0,0 +1,103 @@ +package akka.remote.artery + +import akka.actor.{ ActorRef, InternalActorRef } +import akka.remote.EndpointManager.Send +import akka.remote.{ MessageSerializer, UniqueAddress } +import akka.serialization.{ Serialization, SerializationExtension } +import akka.stream._ +import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler } + +// TODO: Long UID +class Encoder( + transport: ArteryTransport, + compressionTable: LiteralCompressionTable) + extends GraphStage[FlowShape[Send, EnvelopeBuffer]] { + + val in: Inlet[Send] = Inlet("Artery.Encoder.in") + val out: Outlet[EnvelopeBuffer] = Outlet("Artery.Encoder.out") + val shape: FlowShape[Send, EnvelopeBuffer] = FlowShape(in, out) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new GraphStageLogic(shape) with InHandler with OutHandler { + + private val pool = transport.envelopePool + private val headerBuilder = HeaderBuilder(compressionTable) + headerBuilder.version = ArteryTransport.Version + headerBuilder.uid = transport.localAddress.uid + private val localAddress = transport.localAddress.address + private val serialization = SerializationExtension(transport.system) + + override def onPush(): Unit = { + val send = grab(in) + val envelope = pool.acquire() + + headerBuilder.recipientActorRef = send.recipient.path.toSerializationFormat + send.senderOption match { + case Some(sender) ⇒ + headerBuilder.senderActorRef = sender.path.toSerializationFormatWithAddress(localAddress) + case None ⇒ + //headerBuilder.setNoSender() + headerBuilder.senderActorRef = transport.system.deadLetters.path.toSerializationFormatWithAddress(localAddress) + } + + // FIXME: Thunk allocation + Serialization.currentTransportInformation.withValue(Serialization.Information(localAddress, transport.system)) { + MessageSerializer.serializeForArtery(serialization, send.message.asInstanceOf[AnyRef], headerBuilder, envelope) + } + + //println(s"${headerBuilder.senderActorRef} --> ${headerBuilder.recipientActorRef} ${headerBuilder.classManifest}") + + envelope.byteBuffer.flip() + push(out, envelope) + } + + override def onPull(): Unit = pull(in) + + setHandlers(in, out, this) + } +} + +class Decoder( + transport: ArteryTransport, + compressionTable: LiteralCompressionTable) extends GraphStage[FlowShape[EnvelopeBuffer, InboundEnvelope]] { + 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 GraphStageLogic(shape) with InHandler with OutHandler { + private val pool = transport.envelopePool + private val localAddress = transport.localAddress.address + private val provider = transport.provider + private val headerBuilder = HeaderBuilder(compressionTable) + + override def onPush(): Unit = { + val envelope = grab(in) + envelope.parseHeader(headerBuilder) + + //println(s"${headerBuilder.recipientActorRef} <-- ${headerBuilder.senderActorRef} ${headerBuilder.classManifest}") + + // FIXME: Instead of using Strings, the headerBuilder should automatically return cached ActorRef instances + // in case of compression is enabled + // FIXME: Is localAddress really needed? + val recipient: InternalActorRef = + provider.resolveActorRefWithLocalAddress(headerBuilder.recipientActorRef, localAddress) + val sender: ActorRef = + provider.resolveActorRefWithLocalAddress(headerBuilder.senderActorRef, localAddress) + + val decoded = InboundEnvelope( + recipient, + localAddress, // FIXME: Is this needed anymore? What should we do here? + MessageSerializer.deserializeForArtery(transport.system, headerBuilder, envelope), + Some(sender), // FIXME: No need for an option, decode simply to deadLetters instead + UniqueAddress(sender.path.address, headerBuilder.uid)) + + pool.release(envelope) + push(out, decoded) + } + + override def onPull(): Unit = pull(in) + + setHandlers(in, out, this) + } +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/Compression.scala b/akka-remote/src/main/scala/akka/remote/artery/Compression.scala new file mode 100644 index 0000000000..364a7a5c9d --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/Compression.scala @@ -0,0 +1,23 @@ +package akka.remote.artery + +import akka.actor.ActorSystem + +/** + * INTERNAL API + */ +// FIXME: Dummy compression table, needs to be replaced by the real deal +// Currently disables all compression +private[remote] class Compression(system: ActorSystem) extends LiteralCompressionTable { + // FIXME: Of course it is foolish to store this as String, but this is a stub + val deadLettersString = system.deadLetters.path.toSerializationFormat + + override def compressActorRef(ref: String): Int = -1 + override def decompressActorRef(idx: Int): String = ??? + + override def compressSerializer(serializer: String): Int = -1 + override def decompressSerializer(idx: Int): String = ??? + + override def compressClassManifest(manifest: String): Int = -1 + override def decompressClassManifest(idx: Int): String = ??? + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala b/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala index 834e124a5a..26fcdb904d 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala @@ -91,6 +91,8 @@ object AeronStreamsApp { }) lazy val reporterExecutor = Executors.newFixedThreadPool(1) + lazy val pool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumFrameSize) + def stopReporter(): Unit = { reporter.halt() reporterExecutor.shutdown() @@ -140,10 +142,10 @@ object AeronStreamsApp { if (args.length != 0 && args(0) == "echo-receiver") runEchoReceiver() - if (args(0) == "debug-receiver") + if (args.length != 0 && args(0) == "debug-receiver") runDebugReceiver() - if (args(0) == "debug-sender") + if (args.length != 0 && args(0) == "debug-sender") runDebugSender() if (args.length >= 2 && args(1) == "stats") @@ -157,20 +159,21 @@ object AeronStreamsApp { var t0 = System.nanoTime() var count = 0L var payloadSize = 0L - Source.fromGraph(new AeronSource(channel1, streamId, aeron, taskRunner)) - .map { bytes ⇒ - r.onMessage(1, bytes.length) - bytes + Source.fromGraph(new AeronSource(channel1, streamId, aeron, taskRunner, pool)) + .map { envelope ⇒ + r.onMessage(1, envelope.byteBuffer.limit) + envelope } - .runForeach { bytes ⇒ + .runForeach { envelope ⇒ count += 1 if (count == 1) { t0 = System.nanoTime() - payloadSize = bytes.length + payloadSize = envelope.byteBuffer.limit } else if (count == throughputN) { exit(0) printTotal(throughputN, "receive", t0, payloadSize) } + pool.release(envelope) }.onFailure { case e ⇒ e.printStackTrace @@ -193,21 +196,24 @@ object AeronStreamsApp { } .map { _ ⇒ r.onMessage(1, payload.length) - payload + val envelope = pool.acquire() + envelope.byteBuffer.put(payload) + envelope.byteBuffer.flip() + envelope } - .runWith(new AeronSink(channel1, streamId, aeron, taskRunner)) + .runWith(new AeronSink(channel1, streamId, aeron, taskRunner, pool)) } def runEchoReceiver(): Unit = { // just echo back on channel2 reporterExecutor.execute(reporter) val r = reporter - Source.fromGraph(new AeronSource(channel1, streamId, aeron, taskRunner)) - .map { bytes ⇒ - r.onMessage(1, bytes.length) - bytes + Source.fromGraph(new AeronSource(channel1, streamId, aeron, taskRunner, pool)) + .map { envelope ⇒ + r.onMessage(1, envelope.byteBuffer.limit) + envelope } - .runWith(new AeronSink(channel2, streamId, aeron, taskRunner)) + .runWith(new AeronSink(channel2, streamId, aeron, taskRunner, pool)) } def runEchoSender(): Unit = { @@ -219,21 +225,22 @@ object AeronStreamsApp { var repeat = 3 val count = new AtomicInteger var t0 = System.nanoTime() - Source.fromGraph(new AeronSource(channel2, streamId, aeron, taskRunner)) - .map { bytes ⇒ - r.onMessage(1, bytes.length) - bytes + Source.fromGraph(new AeronSource(channel2, streamId, aeron, taskRunner, pool)) + .map { envelope ⇒ + r.onMessage(1, envelope.byteBuffer.limit) + envelope } - .runForeach { bytes ⇒ + .runForeach { envelope ⇒ val c = count.incrementAndGet() val d = System.nanoTime() - sendTimes.get(c - 1) if (c % (latencyN / 10) == 0) println(s"# receive offset $c => ${d / 1000} µs") // FIXME histogram.recordValue(d) if (c == latencyN) { - printTotal(latencyN, "ping-pong", t0, bytes.length) + printTotal(latencyN, "ping-pong", t0, envelope.byteBuffer.limit) barrier.await() // this is always the last party } + pool.release(envelope) }.onFailure { case e ⇒ e.printStackTrace @@ -252,9 +259,12 @@ object AeronStreamsApp { if (n % (latencyN / 10) == 0) println(s"# send offset $n") // FIXME sendTimes.set(n - 1, System.nanoTime()) - payload + val envelope = pool.acquire() + envelope.byteBuffer.put(payload) + envelope.byteBuffer.flip() + envelope } - .runWith(new AeronSink(channel1, streamId, aeron, taskRunner)) + .runWith(new AeronSink(channel1, streamId, aeron, taskRunner, pool)) barrier.await() } @@ -264,8 +274,13 @@ object AeronStreamsApp { def runDebugReceiver(): Unit = { import system.dispatcher - Source.fromGraph(new AeronSource(channel1, streamId, aeron, taskRunner)) - .map(bytes ⇒ new String(bytes, "utf-8")) + Source.fromGraph(new AeronSource(channel1, streamId, aeron, taskRunner, pool)) + .map { envelope ⇒ + val bytes = Array.ofDim[Byte](envelope.byteBuffer.limit) + envelope.byteBuffer.get(bytes) + pool.release(envelope) + new String(bytes, "utf-8") + } .runForeach { s ⇒ println(s) }.onFailure { @@ -283,9 +298,12 @@ object AeronStreamsApp { .map { n ⇒ val s = (fill + n.toString).takeRight(4) println(s) - s.getBytes("utf-8") + val envelope = pool.acquire() + envelope.byteBuffer.put(s.getBytes("utf-8")) + envelope.byteBuffer.flip() + envelope } - .runWith(new AeronSink(channel1, streamId, aeron, taskRunner)) + .runWith(new AeronSink(channel1, streamId, aeron, taskRunner, pool)) } def runStats(): Unit = { 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..dbd118e0ff --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala @@ -0,0 +1,169 @@ +package akka.remote.artery + +import java.nio.{ ByteBuffer, ByteOrder } + +import akka.testkit.AkkaSpec +import akka.util.ByteString + +class EnvelopeBufferSpec extends AkkaSpec { + + object TestCompressor extends LiteralCompressionTable { + val refToIdx = Map( + "compressable0" -> 0, + "compressable1" -> 1, + "reallylongcompressablestring" -> 2) + val idxToRef = 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) + + override def compressActorRef(ref: String): Int = refToIdx.getOrElse(ref, -1) + override def decompressActorRef(idx: Int): String = idxToRef(idx) + override def compressSerializer(serializer: String): Int = serializerToIdx.getOrElse(serializer, -1) + override def decompressSerializer(idx: Int): String = idxToSer(idx) + override def compressClassManifest(manifest: String): Int = manifestToIdx.getOrElse(manifest, -1) + override def decompressClassManifest(idx: Int): String = idxToManifest(idx) + } + + "EnvelopeBuffer" must { + val headerIn = HeaderBuilder(TestCompressor) + val headerOut = HeaderBuilder(TestCompressor) + + val byteBuffer = ByteBuffer.allocate(1024).order(ByteOrder.LITTLE_ENDIAN) + val envelope = new EnvelopeBuffer(byteBuffer) + + "be able to encode and decode headers with compressed literals" in { + headerIn.version = 1 + headerIn.uid = 42 + headerIn.senderActorRef = "compressable0" + headerIn.recipientActorRef = "compressable1" + headerIn.serializer = "serializer0" + headerIn.classManifest = "manifest1" + + envelope.writeHeader(headerIn) + envelope.byteBuffer.position() should ===(EnvelopeBuffer.LiteralsSectionOffset) // Fully compressed header + + envelope.byteBuffer.flip() + envelope.parseHeader(headerOut) + + headerOut.version should ===(1) + headerOut.uid should ===(42) + headerOut.senderActorRef should ===("compressable0") + headerOut.recipientActorRef should ===("compressable1") + headerOut.serializer should ===("serializer0") + headerOut.classManifest should ===("manifest1") + } + + "be able to encode and decode headers with uncompressed literals" in { + headerIn.version = 1 + headerIn.uid = 42 + headerIn.senderActorRef = "uncompressable0" + headerIn.recipientActorRef = "uncompressable11" + headerIn.serializer = "uncompressable222" + headerIn.classManifest = "uncompressable3333" + + val expectedHeaderLength = + EnvelopeBuffer.LiteralsSectionOffset + // Constant header part + 2 + headerIn.senderActorRef.length + // Length field + literal + 2 + headerIn.recipientActorRef.length + // Length field + literal + 2 + headerIn.serializer.length + // Length field + literal + 2 + headerIn.classManifest.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.senderActorRef should ===("uncompressable0") + headerOut.recipientActorRef should ===("uncompressable11") + headerOut.serializer should ===("uncompressable222") + headerOut.classManifest should ===("uncompressable3333") + } + + "be able to encode and decode headers with mixed literals" in { + headerIn.version = 1 + headerIn.uid = 42 + headerIn.senderActorRef = "reallylongcompressablestring" + headerIn.recipientActorRef = "uncompressable1" + headerIn.serializer = "longuncompressedserializer" + headerIn.classManifest = "manifest1" + + envelope.writeHeader(headerIn) + envelope.byteBuffer.position() should ===( + EnvelopeBuffer.LiteralsSectionOffset + + 2 + headerIn.recipientActorRef.length + + 2 + headerIn.serializer.length) + + envelope.byteBuffer.flip() + envelope.parseHeader(headerOut) + + headerOut.version should ===(1) + headerOut.uid should ===(42) + headerOut.senderActorRef should ===("reallylongcompressablestring") + headerOut.recipientActorRef should ===("uncompressable1") + headerOut.serializer should ===("longuncompressedserializer") + headerOut.classManifest should ===("manifest1") + + headerIn.version = 3 + headerIn.uid = Int.MinValue + headerIn.senderActorRef = "uncompressable0" + headerIn.recipientActorRef = "reallylongcompressablestring" + headerIn.serializer = "serializer0" + headerIn.classManifest = "longlonglongliteralmanifest" + + envelope.writeHeader(headerIn) + envelope.byteBuffer.position() should ===( + EnvelopeBuffer.LiteralsSectionOffset + + 2 + headerIn.senderActorRef.length + + 2 + headerIn.classManifest.length) + + envelope.byteBuffer.flip() + envelope.parseHeader(headerOut) + + headerOut.version should ===(3) + headerOut.uid should ===(Int.MinValue) + headerOut.senderActorRef should ===("uncompressable0") + headerOut.recipientActorRef should ===("reallylongcompressablestring") + headerOut.serializer should ===("serializer0") + headerOut.classManifest should ===("longlonglongliteralmanifest") + } + + "be able to encode and decode headers with mixed literals and payload" in { + val payload = ByteString("Hello Artery!") + + headerIn.version = 1 + headerIn.uid = 42 + headerIn.senderActorRef = "reallylongcompressablestring" + headerIn.recipientActorRef = "uncompressable1" + headerIn.serializer = "serializer1" + headerIn.classManifest = "manifest1" + + envelope.writeHeader(headerIn) + envelope.byteBuffer.put(payload.toByteBuffer) + envelope.byteBuffer.flip() + + envelope.parseHeader(headerOut) + + headerOut.version should ===(1) + headerOut.uid should ===(42) + headerOut.senderActorRef should ===("reallylongcompressablestring") + headerOut.recipientActorRef should ===("uncompressable1") + headerOut.serializer should ===("serializer1") + headerOut.classManifest should ===("manifest1") + + ByteString.fromByteBuffer(envelope.byteBuffer) should ===(payload) + } + + } + +} From 23e48b1b352eee3c327bd6738a6d1e527c83af34 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 20 May 2016 11:23:46 +0200 Subject: [PATCH 032/186] rename protocol to artery --- .../src/main/scala/akka/remote/artery/ArteryTransport.scala | 2 +- .../test/scala/akka/remote/artery/HandshakeFailureSpec.scala | 2 +- .../test/scala/akka/remote/artery/HandshakeRetrySpec.scala | 2 +- .../scala/akka/remote/artery/InboundControlJunctionSpec.scala | 4 ++-- .../test/scala/akka/remote/artery/InboundHandshakeSpec.scala | 4 ++-- .../akka/remote/artery/OutboundControlJunctionSpec.scala | 4 ++-- .../test/scala/akka/remote/artery/OutboundHandshakeSpec.scala | 4 ++-- 7 files changed, 11 insertions(+), 11 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 05aa1aa075..ba942b8a13 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -263,7 +263,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // TODO: Configure materializer properly // TODO: Have a supervisor actor _localAddress = UniqueAddress( - Address("akka.artery", system.name, remoteSettings.ArteryHostname, port), + Address("artery", system.name, remoteSettings.ArteryHostname, port), AddressUidExtension(system).addressUid) materializer = ActorMaterializer()(system) diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala index d350151041..9a368c8404 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala @@ -39,7 +39,7 @@ class HandshakeFailureSpec extends AkkaSpec(HandshakeFailureSpec.commonConfig) w "Artery handshake" must { "allow for timeout and later connect" in { - def sel = system.actorSelection(s"akka.artery://systemB@localhost:$portB/user/echo") + def sel = system.actorSelection(s"artery://systemB@localhost:$portB/user/echo") sel ! "hello" expectNoMsg(3.seconds) // longer than handshake-timeout diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala index a334efa642..1b90d60611 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala @@ -39,7 +39,7 @@ class HandshakeRetrySpec extends AkkaSpec(HandshakeRetrySpec.commonConfig) with "Artery handshake" must { "be retried during handshake-timeout (no message loss)" in { - def sel = system.actorSelection(s"akka.artery://systemB@localhost:$portB/user/echo") + def sel = system.actorSelection(s"artery://systemB@localhost:$portB/user/echo") sel ! "hello" expectNoMsg(1.second) diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala index 1ead987ae9..389d843b5f 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala @@ -31,8 +31,8 @@ class InboundControlJunctionSpec extends AkkaSpec with ImplicitSender { val matSettings = ActorMaterializerSettings(system).withFuzzing(true) implicit val mat = ActorMaterializer(matSettings)(system) - val addressA = UniqueAddress(Address("akka.artery", "sysA", "hostA", 1001), 1) - val addressB = UniqueAddress(Address("akka.artery", "sysB", "hostB", 1002), 2) + val addressA = UniqueAddress(Address("artery", "sysA", "hostA", 1001), 1) + val addressB = UniqueAddress(Address("artery", "sysB", "hostB", 1002), 2) "Control messages" must { diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala index bf7cfcf8bc..7266937206 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala @@ -35,8 +35,8 @@ class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { val matSettings = ActorMaterializerSettings(system).withFuzzing(true) implicit val mat = ActorMaterializer(matSettings)(system) - val addressA = UniqueAddress(Address("akka.artery", "sysA", "hostA", 1001), 1) - val addressB = UniqueAddress(Address("akka.artery", "sysB", "hostB", 1002), 2) + val addressA = UniqueAddress(Address("artery", "sysA", "hostA", 1001), 1) + val addressB = UniqueAddress(Address("artery", "sysB", "hostB", 1002), 2) private def setupStream(inboundContext: InboundContext, timeout: FiniteDuration = 5.seconds): (TestPublisher.Probe[AnyRef], TestSubscriber.Probe[Any]) = { val recipient = null.asInstanceOf[InternalActorRef] // not used diff --git a/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala index ab1eb63f0a..206a355ece 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala @@ -30,8 +30,8 @@ class OutboundControlJunctionSpec extends AkkaSpec with ImplicitSender { val matSettings = ActorMaterializerSettings(system).withFuzzing(true) implicit val mat = ActorMaterializer(matSettings)(system) - val addressA = UniqueAddress(Address("akka.artery", "sysA", "hostA", 1001), 1) - val addressB = UniqueAddress(Address("akka.artery", "sysB", "hostB", 1002), 2) + val addressA = UniqueAddress(Address("artery", "sysA", "hostA", 1001), 1) + val addressB = UniqueAddress(Address("artery", "sysB", "hostB", 1002), 2) "Control messages" must { diff --git a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala index 2580a18dec..3602d0fa66 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala @@ -28,8 +28,8 @@ class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { val matSettings = ActorMaterializerSettings(system).withFuzzing(true) implicit val mat = ActorMaterializer(matSettings)(system) - val addressA = UniqueAddress(Address("akka.artery", "sysA", "hostA", 1001), 1) - val addressB = UniqueAddress(Address("akka.artery", "sysB", "hostB", 1002), 2) + val addressA = UniqueAddress(Address("artery", "sysA", "hostA", 1001), 1) + val addressB = UniqueAddress(Address("artery", "sysB", "hostB", 1002), 2) private def setupStream(outboundContext: OutboundContext, timeout: FiniteDuration = 5.seconds, retryInterval: FiniteDuration = 10.seconds): (TestPublisher.Probe[String], TestSubscriber.Probe[Any]) = { From 8859fc35e80fb2f3c7a107f7c300dec2fde7796e Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 20 May 2016 11:24:41 +0200 Subject: [PATCH 033/186] increase Aeron liveness timeout --- .../src/main/scala/akka/remote/artery/ArteryTransport.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index ba942b8a13..22969e2bb8 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -278,9 +278,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // TODO also support external media driver val driverContext = new MediaDriver.Context // FIXME settings from config - driverContext.clientLivenessTimeoutNs(SECONDS.toNanos(10)) - driverContext.imageLivenessTimeoutNs(SECONDS.toNanos(10)) - driverContext.driverTimeoutMs(SECONDS.toNanos(10)) + driverContext.clientLivenessTimeoutNs(SECONDS.toNanos(20)) + driverContext.imageLivenessTimeoutNs(SECONDS.toNanos(20)) + driverContext.driverTimeoutMs(SECONDS.toNanos(20)) driver = MediaDriver.launchEmbedded(driverContext) } From 121840589b68277cf8a8e5b029619e33121556db Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 20 May 2016 10:33:55 +0200 Subject: [PATCH 034/186] improve codec performance * caching of actor refs in Encoder, Decoder * dynamicAccess.getClassFor in Serialization is costly, so introduced a cache for the class manifests there --- .../akka/serialization/Serialization.scala | 19 ++++-- .../scala/akka/remote/MessageSerializer.scala | 4 +- .../scala/akka/remote/artery/Codecs.scala | 67 ++++++++++++++++--- 3 files changed, 72 insertions(+), 18 deletions(-) diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index b0f6f275de..dbc1f21d1c 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -83,6 +83,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 ConcurrentHashMap[String, Option[Class[_]]] /** * Serializes the given AnyRef/java.lang.Object according to the Serialization configuration @@ -123,12 +124,18 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { 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].") + val cachedClassManifest = manifestCache.get(manifest) + if (cachedClassManifest ne null) + s1.fromBinary(bytes, cachedClassManifest) + else { + system.dynamicAccess.getClassFor[AnyRef](manifest) match { + case Success(classManifest) ⇒ + manifestCache.put(manifest, Some(classManifest)) + s1.fromBinary(bytes, Some(classManifest)) + case Failure(e) ⇒ + throw new NotSerializableException( + s"Cannot find manifest class [$manifest] for serializer with id [$serializerId].") + } } } } diff --git a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala index e11627a078..7119b96d2c 100644 --- a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala @@ -66,12 +66,12 @@ private[akka] object MessageSerializer { envelope.byteBuffer.put(serializer.toBinary(message)) } - def deserializeForArtery(system: ExtendedActorSystem, headerBuilder: HeaderBuilder, envelope: EnvelopeBuffer): AnyRef = { + def deserializeForArtery(system: ExtendedActorSystem, serialization: Serialization, headerBuilder: HeaderBuilder, envelope: EnvelopeBuffer): AnyRef = { // FIXME: Use the buffer directly val size = envelope.byteBuffer.limit - envelope.byteBuffer.position val bytes = Array.ofDim[Byte](size) envelope.byteBuffer.get(bytes) - SerializationExtension(system).deserialize( + serialization.deserialize( bytes, Integer.parseInt(headerBuilder.serializer), // FIXME: Use FQCN headerBuilder.classManifest).get diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index db412d7862..aeeb541cde 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -27,17 +27,42 @@ class Encoder( private val localAddress = transport.localAddress.address private val serialization = SerializationExtension(transport.system) + private val noSender = transport.system.deadLetters.path.toSerializationFormatWithAddress(localAddress) + private val senderCache = new java.util.HashMap[ActorRef, String] + private var recipientCache = new java.util.HashMap[ActorRef, String] + override def onPush(): Unit = { val send = grab(in) val envelope = pool.acquire() - headerBuilder.recipientActorRef = send.recipient.path.toSerializationFormat + val recipientStr = recipientCache.get(send.recipient) match { + case null ⇒ + val s = send.recipient.path.toSerializationFormat + // FIXME this cache will be replaced by compression table + if (recipientCache.size() >= 1000) + recipientCache.clear() + recipientCache.put(send.recipient, s) + s + case s ⇒ s + } + headerBuilder.recipientActorRef = recipientStr + send.senderOption match { case Some(sender) ⇒ - headerBuilder.senderActorRef = sender.path.toSerializationFormatWithAddress(localAddress) + val senderStr = senderCache.get(sender) match { + case null ⇒ + val s = sender.path.toSerializationFormatWithAddress(localAddress) + // FIXME we might need an efficient LRU cache, or replaced by compression table + if (senderCache.size() >= 1000) + senderCache.clear() + senderCache.put(sender, s) + s + case s ⇒ s + } + headerBuilder.senderActorRef = senderStr case None ⇒ //headerBuilder.setNoSender() - headerBuilder.senderActorRef = transport.system.deadLetters.path.toSerializationFormatWithAddress(localAddress) + headerBuilder.senderActorRef = noSender } // FIXME: Thunk allocation @@ -70,6 +95,10 @@ class Decoder( private val localAddress = transport.localAddress.address private val provider = transport.provider private val headerBuilder = HeaderBuilder(compressionTable) + private val serialization = SerializationExtension(transport.system) + + private val recipientCache = new java.util.HashMap[String, InternalActorRef] + private val senderCache = new java.util.HashMap[String, Option[ActorRef]] override def onPush(): Unit = { val envelope = grab(in) @@ -80,17 +109,35 @@ class Decoder( // FIXME: Instead of using Strings, the headerBuilder should automatically return cached ActorRef instances // in case of compression is enabled // FIXME: Is localAddress really needed? - val recipient: InternalActorRef = - provider.resolveActorRefWithLocalAddress(headerBuilder.recipientActorRef, localAddress) - val sender: ActorRef = - provider.resolveActorRefWithLocalAddress(headerBuilder.senderActorRef, localAddress) + val recipient: InternalActorRef = recipientCache.get(headerBuilder.recipientActorRef) match { + case null ⇒ + val ref = provider.resolveActorRefWithLocalAddress(headerBuilder.recipientActorRef, localAddress) + // FIXME we might need an efficient LRU cache, or replaced by compression table + if (recipientCache.size() >= 1000) + recipientCache.clear() + recipientCache.put(headerBuilder.recipientActorRef, ref) + ref + case ref ⇒ ref + } + + val senderOption: Option[ActorRef] = senderCache.get(headerBuilder.senderActorRef) match { + case null ⇒ + val ref = provider.resolveActorRefWithLocalAddress(headerBuilder.senderActorRef, localAddress) + // FIXME this cache will be replaced by compression table + if (senderCache.size() >= 1000) + senderCache.clear() + val refOpt = Some(ref) + senderCache.put(headerBuilder.senderActorRef, refOpt) + refOpt + case refOpt ⇒ refOpt + } val decoded = InboundEnvelope( recipient, localAddress, // FIXME: Is this needed anymore? What should we do here? - MessageSerializer.deserializeForArtery(transport.system, headerBuilder, envelope), - Some(sender), // FIXME: No need for an option, decode simply to deadLetters instead - UniqueAddress(sender.path.address, headerBuilder.uid)) + MessageSerializer.deserializeForArtery(transport.system, serialization, headerBuilder, envelope), + senderOption, // FIXME: No need for an option, decode simply to deadLetters instead + UniqueAddress(senderOption.get.path.address, headerBuilder.uid)) // FIXME see issue #20568 pool.release(envelope) push(out, decoded) From cd71643a91b0c92985b702f9bbb3a4bbb551ae2a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Fri, 20 May 2016 12:40:56 +0200 Subject: [PATCH 035/186] [WIP] Large message stream for Artery (#20545) * First stab at separate large message channel for Artery * Full actor paths, no implicit "/user/" part * Various small fixes after review * Fixes to make it work after rebasing * Use a separate EnvelopeBufferPool for the large message stream * Docs for actorSelection not sending through large message stream --- akka-remote/src/main/resources/reference.conf | 14 ++ .../akka/remote/RemoteActorRefProvider.scala | 10 ++ .../akka/remote/artery/ArteryTransport.scala | 50 +++++- .../akka/remote/artery/Association.scala | 47 +++++- .../scala/akka/remote/artery/Codecs.scala | 4 +- .../artery/LargeMessagesStreamSpec.scala | 155 ++++++++++++++++++ 6 files changed, 266 insertions(+), 14 deletions(-) create mode 100644 akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index fc81a7597a..ea692dab5c 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -91,6 +91,20 @@ akka { # InetAddress.getLocalHost.getHostName is used if # "" is specified. 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 = [] } ### General settings diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 52fe8f2ba8..987f819b88 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -438,6 +438,13 @@ private[akka] trait RemoteRef extends ActorRefScope { final def isLocal = false } +/** + * INTERNAL API + */ +private[remote] sealed abstract class LargeMessageDestinationFlag +private[remote] case object RegularDestination extends LargeMessageDestinationFlag +private[remote] case object LargeDestination extends LargeMessageDestinationFlag + /** * INTERNAL API * Remote ActorRef that is used when referencing the Actor on a different node than its "home" node. @@ -454,6 +461,9 @@ private[akka] class RemoteActorRef private[akka] ( @volatile var cachedAssociation: artery.Association = null + // used by artery to direct messages to a separate stream for large messages + @volatile var cachedLargeMessageDestinationFlag: LargeMessageDestinationFlag = null + def getChild(name: Iterator[String]): InternalActorRef = { val s = name.toStream s.headOption match { diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 05aa1aa075..df612a77a0 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -50,8 +50,7 @@ import akka.stream.scaladsl.Framing import akka.stream.scaladsl.Keep import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Source -import akka.util.ByteString -import akka.util.ByteStringBuilder +import akka.util.{ ByteString, ByteStringBuilder, WildcardTree } import akka.util.Helpers.ConfigOps import akka.util.Helpers.Requiring import io.aeron.Aeron @@ -68,6 +67,7 @@ import java.nio.channels.DatagramChannel import akka.remote.artery.OutboundControlJunction.OutboundControlIngress +import scala.collection.JavaConverters._ /** * INTERNAL API */ @@ -233,10 +233,18 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R system.settings.config.getMillisDuration("akka.remote.handshake-timeout").requiring(_ > Duration.Zero, "handshake-timeout must be > 0") + private val largeMessageDestinations = + system.settings.config.getStringList("akka.remote.artery.large-message-destinations").asScala.foldLeft(WildcardTree[NotUsed]()) { (tree, entry) ⇒ + val segments = entry.split('/').tail + tree.insert(segments.iterator, NotUsed) + } + private val largeMessageDestinationsEnabled = largeMessageDestinations.children.nonEmpty + private def inboundChannel = s"aeron:udp?endpoint=${localAddress.address.host.get}:${localAddress.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 = 3 + private val largeStreamId = 4 private val taskRunner = new TaskRunner(system) // FIXME: This does locking on putIfAbsent, we need something smarter @@ -247,6 +255,10 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val restartCounter = new RestartCounter(maxRestarts, restartTimeout) val envelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) + val largeEnvelopePool: Option[EnvelopeBufferPool] = + if (largeMessageDestinationsEnabled) Some(new EnvelopeBufferPool(ArteryTransport.MaximumLargeFrameSize, ArteryTransport.MaximumPooledBuffers)) + else None + // FIXME: Compression table must be owned by each channel instead // of having a global one val compression = new Compression(system) @@ -320,6 +332,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private def runInboundStreams(): Unit = { runInboundControlStream() runInboundOrdinaryMessagesStream() + if (largeMessageDestinationsEnabled) { + runInboundLargeMessagesStream() + } } private def runInboundControlStream(): Unit = { @@ -369,6 +384,18 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R attachStreamRestart("Inbound message stream", completed, () ⇒ runInboundOrdinaryMessagesStream()) } + private def runInboundLargeMessagesStream(): Unit = { + largeEnvelopePool.foreach { largePool ⇒ + // TODO just cargo-cult programming here + val completed = Source.fromGraph(new AeronSource(inboundChannel, largeStreamId, aeron, taskRunner, largePool)) + .async // FIXME measure + .via(inboundFlow) + .runWith(Sink.ignore)(materializer) + + 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 { @@ -422,7 +449,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R else { associations.computeIfAbsent(remoteAddress, new JFunction[Address, Association] { override def apply(remoteAddress: Address): Association = { - val newAssociation = new Association(ArteryTransport.this, materializer, remoteAddress, controlSubject) + val newAssociation = new Association(ArteryTransport.this, materializer, remoteAddress, controlSubject, largeMessageDestinations) newAssociation.associate() // This is a bit costly for this blocking method :( newAssociation } @@ -443,6 +470,17 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), ordinaryStreamId, aeron, taskRunner, envelopePool))(Keep.right) } + def outboundLarge(outboundContext: OutboundContext): Sink[Send, Future[Done]] = { + largeEnvelopePool match { + case Some(pool) ⇒ + Flow.fromGraph(killSwitch.flow[Send]) + .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval)) + .via(createEncoder(pool)) + .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), largeStreamId, aeron, taskRunner, envelopePool))(Keep.right) + case None ⇒ throw new IllegalArgumentException("Trying to create outbound stream but outbound stream not configured") + } + } + def outboundControl(outboundContext: OutboundContext): Sink[Send, (OutboundControlIngress, Future[Done])] = { Flow.fromGraph(killSwitch.flow[Send]) .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval)) @@ -457,8 +495,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // FIXME hack until real envelopes, encoding originAddress in sender :) private val dummySender = system.systemActorOf(Props.empty, "dummy") - val encoder: Flow[Send, EnvelopeBuffer, NotUsed] = - Flow.fromGraph(new Encoder(this, compression)) + def createEncoder(pool: EnvelopeBufferPool): Flow[Send, EnvelopeBuffer, NotUsed] = + Flow.fromGraph(new Encoder(this, compression, pool)) + val encoder = createEncoder(envelopePool) val messageDispatcherSink: Sink[InboundEnvelope, Future[Done]] = Sink.foreach[InboundEnvelope] { m ⇒ messageDispatcher.dispatch(m.recipient, m.recipientAddress, m.message, m.senderOption) @@ -496,6 +535,7 @@ private[remote] object ArteryTransport { val Version = 0 val MaximumFrameSize = 1024 * 1024 val MaximumPooledBuffers = 256 + val MaximumLargeFrameSize = MaximumFrameSize * 5 /** * Internal API diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 219dc15906..fe6b9e800e 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -12,8 +12,7 @@ import scala.concurrent.Promise import scala.concurrent.duration._ import scala.concurrent.duration.FiniteDuration import scala.util.Success - -import akka.Done +import akka.{ Done, NotUsed } import akka.actor.ActorRef import akka.actor.ActorSelectionMessage import akka.actor.Address @@ -21,8 +20,7 @@ import akka.actor.RootActorPath import akka.dispatch.sysmsg.SystemMessage import akka.event.Logging import akka.remote.EndpointManager.Send -import akka.remote.RemoteActorRef -import akka.remote.UniqueAddress +import akka.remote.{ LargeDestination, RegularDestination, RemoteActorRef, UniqueAddress } import akka.remote.artery.InboundControlJunction.ControlMessageSubject import akka.remote.artery.OutboundControlJunction.OutboundControlIngress import akka.remote.artery.OutboundHandshake.HandshakeTimeoutException @@ -33,7 +31,7 @@ import akka.stream.OverflowStrategy import akka.stream.scaladsl.Keep import akka.stream.scaladsl.Source import akka.stream.scaladsl.SourceQueueWithComplete -import akka.util.Unsafe +import akka.util.{ Unsafe, WildcardTree } /** * INTERNAL API @@ -45,7 +43,8 @@ private[akka] class Association( val transport: ArteryTransport, val materializer: Materializer, override val remoteAddress: Address, - override val controlSubject: ControlMessageSubject) + override val controlSubject: ControlMessageSubject, + largeMessageDestinations: WildcardTree[NotUsed]) extends AbstractAssociation with OutboundContext { private val log = Logging(transport.system, getClass.getName) @@ -54,8 +53,10 @@ private[akka] class Association( private val restartTimeout: FiniteDuration = 5.seconds // FIXME config private val maxRestarts = 5 // FIXME config private val restartCounter = new RestartCounter(maxRestarts, restartTimeout) + private val largeMessageChannelEnabled = largeMessageDestinations.children.nonEmpty @volatile private[this] var queue: SourceQueueWithComplete[Send] = _ + @volatile private[this] var largeQueue: SourceQueueWithComplete[Send] = _ @volatile private[this] var controlQueue: SourceQueueWithComplete[Send] = _ @volatile private[this] var _outboundControlIngress: OutboundControlIngress = _ @volatile private[this] var materializing = new CountDownLatch(1) @@ -136,12 +137,32 @@ private[akka] class Association( quarantine(reason = s"Due to overflow of control queue, size [$controlQueueSize]") } case _ ⇒ - queue.offer(Send(message, senderOption, recipient, None)) + val send = Send(message, senderOption, recipient, None) + if (largeMessageChannelEnabled && isLargeMessageDestination(recipient)) + largeQueue.offer(send) + else + queue.offer(send) } } else if (log.isDebugEnabled) log.debug("Dropping message to quarantined system {}", remoteAddress) } + private def isLargeMessageDestination(recipient: ActorRef): Boolean = { + recipient match { + case r: RemoteActorRef if r.cachedLargeMessageDestinationFlag ne null ⇒ r.cachedLargeMessageDestinationFlag == LargeDestination + case r: RemoteActorRef ⇒ + if (largeMessageDestinations.find(r.path.elements.iterator).data.isEmpty) { + r.cachedLargeMessageDestinationFlag = RegularDestination + false + } else { + log.debug("Using large message stream for {}", r.path) + r.cachedLargeMessageDestinationFlag = LargeDestination + true + } + case _ ⇒ false + } + } + // FIXME we should be able to Send without a recipient ActorRef override val dummyRecipient: RemoteActorRef = transport.provider.resolveActorRef(RootActorPath(remoteAddress) / "system" / "dummy").asInstanceOf[RemoteActorRef] @@ -195,6 +216,10 @@ private[akka] class Association( // so that outboundControlIngress is ready when stages for all streams start runOutboundControlStream() runOutboundOrdinaryMessagesStream() + + if (largeMessageChannelEnabled) { + runOutboundLargeMessagesStream() + } } } @@ -225,6 +250,14 @@ private[akka] class Association( attachStreamRestart("Outbound message stream", completed, _ ⇒ runOutboundOrdinaryMessagesStream()) } + private def runOutboundLargeMessagesStream(): Unit = { + val (q, completed) = Source.queue(256, OverflowStrategy.dropBuffer) + .toMat(transport.outboundLarge(this))(Keep.both) + .run()(materializer) + largeQueue = q + attachStreamRestart("Outbound large message stream", completed, _ ⇒ runOutboundLargeMessagesStream()) + } + private def attachStreamRestart(streamName: String, streamCompleted: Future[Done], restart: Throwable ⇒ Unit): Unit = { implicit val ec = materializer.executionContext streamCompleted.onFailure { diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index db412d7862..e853046131 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -10,7 +10,8 @@ import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler } // TODO: Long UID class Encoder( transport: ArteryTransport, - compressionTable: LiteralCompressionTable) + compressionTable: LiteralCompressionTable, + pool: EnvelopeBufferPool) extends GraphStage[FlowShape[Send, EnvelopeBuffer]] { val in: Inlet[Send] = Inlet("Artery.Encoder.in") @@ -20,7 +21,6 @@ class Encoder( override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) with InHandler with OutHandler { - private val pool = transport.envelopePool private val headerBuilder = HeaderBuilder(compressionTable) headerBuilder.version = ArteryTransport.Version headerBuilder.uid = transport.localAddress.uid 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..b56b3ff795 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala @@ -0,0 +1,155 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.actor.{ Actor, ActorRef, ActorSelection, ActorSystem, ExtendedActorSystem, Props, RootActorPath } +import akka.remote.{ LargeDestination, RegularDestination, RemoteActorRef } +import akka.testkit.{ SocketUtil, TestKit, TestProbe } +import akka.util.ByteString +import com.typesafe.config.ConfigFactory +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.{ ShouldMatchers, WordSpec } + +import scala.concurrent.Await +import scala.concurrent.duration._ + +object LargeMessagesStreamSpec { + case class Ping(payload: ByteString = ByteString.empty) + case class Pong(bytesReceived: Long) + class EchoSize extends Actor { + def receive = { + case Ping(bytes) ⇒ sender() ! Pong(bytes.size) + } + } +} + +class LargeMessagesStreamSpec extends WordSpec with ShouldMatchers with ScalaFutures { + import LargeMessagesStreamSpec._ + + val config = ConfigFactory.parseString( + s""" + akka { + loglevel = ERROR + actor { + provider = "akka.remote.RemoteActorRefProvider" + } + remote.artery { + enabled = on + hostname = localhost + port = 0 + large-message-destinations = [ + "/user/large" + ] + } + } + + """) + + "The large message support" should { + + "not affect regular communication" in { + val systemA = ActorSystem("systemA", config) + val systemB = ActorSystem("systemB", config) + + try { + 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), "regular") + large.tell(Ping(), senderProbeB.ref) + senderProbeB.expectMsg(Pong(0)) + + // communicate with it from the other system + val addressB = systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + val rootB = RootActorPath(addressB) + val largeRemote = awaitResolve(systemA.actorSelection(rootB / "user" / "regular")) + largeRemote.tell(Ping(), senderProbeA.ref) + senderProbeA.expectMsg(Pong(0)) + + // flag should be cached now + largeRemote.asInstanceOf[RemoteActorRef].cachedLargeMessageDestinationFlag should ===(RegularDestination) + + } finally { + TestKit.shutdownActorSystem(systemA) + TestKit.shutdownActorSystem(systemB) + } + } + + "pass small regular messages over the large-message stream" in { + val systemA = ActorSystem("systemA", config) + val systemB = ActorSystem("systemB", config) + + try { + 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 = systemB.asInstanceOf[ExtendedActorSystem].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].cachedLargeMessageDestinationFlag should ===(LargeDestination) + + } finally { + TestKit.shutdownActorSystem(systemA) + TestKit.shutdownActorSystem(systemB) + } + } + + "allow for normal communication while simultaneously sending large messages" in { + val systemA = ActorSystem("systemA", config) + val systemB = ActorSystem("systemB", config) + + try { + + 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 = systemB.asInstanceOf[ExtendedActorSystem].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].cachedLargeMessageDestinationFlag should ===(LargeDestination) + regularRemote.asInstanceOf[RemoteActorRef].cachedLargeMessageDestinationFlag should ===(RegularDestination) + + } finally { + TestKit.shutdownActorSystem(systemA) + TestKit.shutdownActorSystem(systemB) + } + } + } + + def awaitResolve(selection: ActorSelection): ActorRef = Await.result(selection.resolveOne(3.seconds), 3.seconds) +} From 6603ed52363cac94a10c3269f612502e1c081ce5 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 20 May 2016 12:46:29 +0200 Subject: [PATCH 036/186] temporary workaround for failing RemoteRestartedQuarantinedSpec --- .../akka/remote/artery/RemoteRestartedQuarantinedSpec.scala | 5 +++++ 1 file changed, 5 insertions(+) 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 index b22d35c0fb..f2402fae57 100644 --- 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 @@ -82,6 +82,11 @@ abstract class RemoteRestartedQuarantinedSpec runOn(first) { val secondAddress = node(second).address + // FIXME this should not be needed, see issue #20566 + within(30.seconds) { + identifyWithUid(second, "subject", 1.seconds) + } + val (uid, ref) = identifyWithUid(second, "subject", 5.seconds) enterBarrier("before-quarantined") From c90121485fcfc44a3cee62a0c638e1982d13d812 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 19 May 2016 08:24:27 +0200 Subject: [PATCH 037/186] give up sending after a while, #20317 --- .../artery/AeronStreamConcistencySpec.scala | 7 +- .../artery/AeronStreamLatencySpec.scala | 7 +- .../artery/AeronStreamMaxThroughputSpec.scala | 3 +- .../scala/akka/remote/artery/AeronSink.scala | 51 +++++++---- .../akka/remote/artery/ArteryTransport.scala | 28 ++++--- .../akka/remote/artery/Association.scala | 23 +++-- .../scala/akka/remote/artery/Control.scala | 7 +- .../akka/remote/artery/StageLogging.scala | 34 ++++++++ .../remote/artery/SystemMessageDelivery.scala | 1 + .../akka/remote/artery/AeronSinkSpec.scala | 84 +++++++++++++++++++ .../akka/remote/artery/AeronStreamsApp.scala | 9 +- 11 files changed, 203 insertions(+), 51 deletions(-) create mode 100644 akka-remote/src/main/scala/akka/remote/artery/StageLogging.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala 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 index 526f321bd0..880dafd727 100644 --- 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 @@ -81,6 +81,7 @@ abstract class AeronStreamConsistencySpec } val streamId = 1 + val giveUpSendAfter = 30.seconds override def afterAll(): Unit = { taskRunner.stop() @@ -96,7 +97,7 @@ abstract class AeronStreamConsistencySpec runOn(second) { // just echo back Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner, pool)) - .runWith(new AeronSink(channel(first), streamId, aeron, taskRunner, pool)) + .runWith(new AeronSink(channel(first), streamId, aeron, taskRunner, pool, giveUpSendAfter)) } enterBarrier("echo-started") } @@ -137,7 +138,7 @@ abstract class AeronStreamConsistencySpec envelope } .throttle(1, 200.milliseconds, 1, ThrottleMode.Shaping) - .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter)) started.expectMsg(Done) } @@ -149,7 +150,7 @@ abstract class AeronStreamConsistencySpec envelope.byteBuffer.flip() envelope } - .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter)) Await.ready(done, 20.seconds) killSwitch.shutdown() 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 index 6ce14f0b3b..27159795b1 100644 --- 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 @@ -108,6 +108,7 @@ abstract class AeronStreamLatencySpec } val streamId = 1 + val giveUpSendAfter = 30.seconds lazy val reporterExecutor = Executors.newFixedThreadPool(1) def reporter(name: String): TestRateReporter = { @@ -227,7 +228,7 @@ abstract class AeronStreamLatencySpec envelope } .throttle(1, 200.milliseconds, 1, ThrottleMode.Shaping) - .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter)) started.expectMsg(Done) } @@ -245,7 +246,7 @@ abstract class AeronStreamLatencySpec sendTimes.set(n - 1, System.nanoTime()) envelope } - .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter)) barrier.await((totalMessages / messageRate) + 10, SECONDS) } @@ -264,7 +265,7 @@ abstract class AeronStreamLatencySpec runOn(second) { // just echo back Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner, pool)) - .runWith(new AeronSink(channel(first), streamId, aeron, taskRunner, pool)) + .runWith(new AeronSink(channel(first), streamId, aeron, taskRunner, pool, giveUpSendAfter)) } enterBarrier("echo-started") } 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 index e4db9dca62..110ce66554 100644 --- 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 @@ -114,6 +114,7 @@ abstract class AeronStreamMaxThroughputSpec } val streamId = 1 + val giveUpSendAfter = 30.seconds lazy val reporterExecutor = Executors.newFixedThreadPool(1) def reporter(name: String): TestRateReporter = { @@ -211,7 +212,7 @@ abstract class AeronStreamMaxThroughputSpec envelope.byteBuffer.flip() envelope } - .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter)) printStats("sender") enterBarrier(testName + "-done") diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala index 611809a449..a9a490579d 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala @@ -13,6 +13,7 @@ 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 @@ -28,26 +29,38 @@ import org.agrona.concurrent.UnsafeBuffer object AeronSink { - class OfferTask(pub: Publication, var buffer: UnsafeBuffer, msgSize: AtomicInteger, onOfferSuccess: AsyncCallback[Unit]) - extends (() ⇒ Boolean) { + final class GaveUpSendingException(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]) + extends (() ⇒ Boolean) { + val giveUpAfterNanos = giveUpAfter match { + case f: FiniteDuration ⇒ f.toNanos + case _ ⇒ -1L + } var n = 0L - var localMsgSize = -1 + 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 - if (localMsgSize == -1) - localMsgSize = msgSize.get - val result = pub.offer(buffer, 0, localMsgSize) + val result = pub.offer(buffer, 0, msgSize) if (result >= 0) { - n = 0 - localMsgSize = -1 + n = 0L onOfferSuccess.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 { - // FIXME drop after too many attempts? - if (n > 1000000 && n % 100000 == 0) - println(s"# offer not accepted after $n") // FIXME false } } @@ -57,7 +70,7 @@ object AeronSink { /** * @param channel eg. "aeron:udp?endpoint=localhost:40123" */ -class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRunner, pool: EnvelopeBufferPool) +class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRunner, pool: EnvelopeBufferPool, giveUpSendAfter: Duration) extends GraphStageWithMaterializedValue[SinkShape[EnvelopeBuffer], Future[Done]] { import AeronSink._ import TaskRunner._ @@ -77,8 +90,8 @@ class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRu private val spinning = 1000 private var backoffCount = spinning private var lastMsgSize = 0 - private val lastMsgSizeRef = new AtomicInteger // used in the external backoff task - private val offerTask = new OfferTask(pub, null, lastMsgSizeRef, getAsyncCallback(_ ⇒ onOfferSuccess())) + private val offerTask = new OfferTask(pub, null, lastMsgSize, getAsyncCallback(_ ⇒ onOfferSuccess()), + giveUpSendAfter, getAsyncCallback(_ ⇒ onGiveUp())) private val addOfferTask: Add = Add(offerTask) private var offerTaskInProgress = false @@ -112,9 +125,10 @@ class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRu publish() // recursive } else { // delegate backoff to shared TaskRunner - lastMsgSizeRef.set(lastMsgSize) offerTaskInProgress = true + // visibility of these assignments are ensured by adding the task to the command queue offerTask.buffer = envelopeInFlight.aeronBuffer + offerTask.msgSize = lastMsgSize taskRunner.command(addOfferTask) } } else { @@ -134,6 +148,13 @@ class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRu pull(in) } + private def onGiveUp(): Unit = { + offerTaskInProgress = false + val cause = new GaveUpSendingException(s"Gave up sending message to $channel after $giveUpSendAfter.") + completedValue = Failure(cause) + failStage(cause) + } + override def onUpstreamFinish(): Unit = { // flush outstanding offer before completing stage if (!offerTaskInProgress) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 7a35e2913e..ece54fa867 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -236,6 +236,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val handshakeTimeout: FiniteDuration = system.settings.config.getMillisDuration("akka.remote.handshake-timeout").requiring(_ > Duration.Zero, "handshake-timeout must be > 0") + private val giveUpSendAfter: FiniteDuration = 60.seconds private val largeMessageDestinations = system.settings.config.getStringList("akka.remote.artery.large-message-destinations").asScala.foldLeft(WildcardTree[NotUsed]()) { (tree, entry) ⇒ @@ -416,17 +417,17 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private def attachStreamRestart(streamName: String, streamCompleted: Future[Done], restart: () ⇒ Unit): Unit = { implicit val ec = materializer.executionContext streamCompleted.onFailure { + case _ if isShutdown ⇒ // don't restart after shutdown case _: AbruptTerminationException ⇒ // ActorSystem shutdown case cause ⇒ - if (!isShutdown) - if (restartCounter.restart()) { - log.error(cause, "{} failed. Restarting it.", streamName) - restart() - } else { - log.error(cause, "{} failed and restarted {} times within {} seconds. Terminating system.", - streamName, maxRestarts, restartTimeout.toSeconds) - system.terminate() - } + 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, maxRestarts, restartTimeout.toSeconds, cause.getMessage) + system.terminate() + } } } @@ -485,7 +486,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R Flow.fromGraph(killSwitch.flow[Send]) .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval)) .via(encoder) - .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), ordinaryStreamId, aeron, taskRunner, envelopePool))(Keep.right) + .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), ordinaryStreamId, aeron, taskRunner, + envelopePool, giveUpSendAfter))(Keep.right) } def outboundLarge(outboundContext: OutboundContext): Sink[Send, Future[Done]] = { @@ -494,7 +496,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R Flow.fromGraph(killSwitch.flow[Send]) .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval)) .via(createEncoder(pool)) - .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), largeStreamId, aeron, taskRunner, envelopePool))(Keep.right) + .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), largeStreamId, aeron, taskRunner, + envelopePool, giveUpSendAfter))(Keep.right) case None ⇒ throw new IllegalArgumentException("Trying to create outbound stream but outbound stream not configured") } } @@ -505,7 +508,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .via(new SystemMessageDelivery(outboundContext, systemMessageResendInterval, remoteSettings.SysMsgBufferSize)) .viaMat(new OutboundControlJunction(outboundContext))(Keep.right) .via(encoder) - .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), controlStreamId, aeron, taskRunner, envelopePool))(Keep.both) + .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), controlStreamId, aeron, taskRunner, + envelopePool, Duration.Inf))(Keep.both) // FIXME we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index fe6b9e800e..cbc3abc118 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -21,6 +21,7 @@ import akka.dispatch.sysmsg.SystemMessage import akka.event.Logging import akka.remote.EndpointManager.Send import akka.remote.{ LargeDestination, RegularDestination, RemoteActorRef, UniqueAddress } +import akka.remote.artery.AeronSink.GaveUpSendingException import akka.remote.artery.InboundControlJunction.ControlMessageSubject import akka.remote.artery.OutboundControlJunction.OutboundControlIngress import akka.remote.artery.OutboundHandshake.HandshakeTimeoutException @@ -261,17 +262,21 @@ private[akka] class Association( private def attachStreamRestart(streamName: String, streamCompleted: Future[Done], restart: Throwable ⇒ Unit): Unit = { implicit val ec = materializer.executionContext streamCompleted.onFailure { + case _ if transport.isShutdown ⇒ // don't restart after shutdown case _: AbruptTerminationException ⇒ // ActorSystem shutdown + case cause: GaveUpSendingException ⇒ + log.error(cause, "{} failed. Restarting it. {}", streamName, cause.getMessage) + // restart unconditionally, without counting restarts + restart(cause) case cause ⇒ - if (!transport.isShutdown) - if (restartCounter.restart()) { - log.error(cause, "{} failed. Restarting it.", streamName) - restart(cause) - } else { - log.error(cause, "{} failed and restarted {} times within {} seconds. Terminating system.", - streamName, maxRestarts, restartTimeout.toSeconds) - transport.system.terminate() - } + if (restartCounter.restart()) { + log.error(cause, "{} failed. Restarting it. {}", streamName, cause.getMessage) + restart(cause) + } else { + log.error(cause, "{} failed and restarted {} times within {} seconds. Terminating system. {}", + streamName, maxRestarts, restartTimeout.toSeconds, cause.getMessage) + transport.system.terminate() + } } } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Control.scala b/akka-remote/src/main/scala/akka/remote/artery/Control.scala index b371ae2650..7d32d0889b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Control.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Control.scala @@ -141,7 +141,7 @@ private[akka] class InboundControlJunction * INTERNAL API */ private[akka] object OutboundControlJunction { - trait OutboundControlIngress { + private[akka] trait OutboundControlIngress { def sendControlMessage(message: ControlMessage): Unit } } @@ -158,7 +158,7 @@ private[akka] class OutboundControlJunction(outboundContext: OutboundContext) 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 { + val logic = new GraphStageLogic(shape) with CallbackWrapper[ControlMessage] with InHandler with OutHandler with StageLogging { import OutboundControlJunction._ private val sendControlMessageCallback = getAsyncCallback[ControlMessage](internalSendControlMessage) @@ -192,8 +192,7 @@ private[akka] class OutboundControlJunction(outboundContext: OutboundContext) buffer.offer(wrap(message)) else { // it's alright to drop control messages - // FIXME we need that stage logging support - println(s"dropping control message ${message.getClass.getName} due to full buffer") + log.debug("Dropping control message [{}] due to full buffer.", message.getClass.getName) } } 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 index 9fa61c25dd..747fcad084 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -107,6 +107,7 @@ private[akka] class SystemMessageDelivery( } if (!unacknowledged.isEmpty) scheduleOnce(ResendTick, resendInterval) + // FIXME give up resending after a long while, i.e. config property quarantine-after-silence } // ControlMessageObserver, external call 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..313ac9e979 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala @@ -0,0 +1,84 @@ +/** + * 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.GaveUpSendingException +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 taskRunner = { + val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem]) + r.start() + r + } + + val pool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) + + 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)) + // 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)) + + // without the give up timeout the stream would not complete/fail + intercept[GaveUpSendingException] { + Await.result(done, 5.seconds) + } + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala b/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala index 26fcdb904d..12e664d732 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala @@ -38,6 +38,7 @@ object AeronStreamsApp { val latencyRate = 10000 // per second val latencyN = 10 * latencyRate val payload = ("0" * 100).getBytes("utf-8") + val giveUpSendAfter = 60.seconds lazy val sendTimes = new AtomicLongArray(latencyN) lazy val driver = { @@ -201,7 +202,7 @@ object AeronStreamsApp { envelope.byteBuffer.flip() envelope } - .runWith(new AeronSink(channel1, streamId, aeron, taskRunner, pool)) + .runWith(new AeronSink(channel1, streamId, aeron, taskRunner, pool, giveUpSendAfter)) } def runEchoReceiver(): Unit = { @@ -213,7 +214,7 @@ object AeronStreamsApp { r.onMessage(1, envelope.byteBuffer.limit) envelope } - .runWith(new AeronSink(channel2, streamId, aeron, taskRunner, pool)) + .runWith(new AeronSink(channel2, streamId, aeron, taskRunner, pool, giveUpSendAfter)) } def runEchoSender(): Unit = { @@ -264,7 +265,7 @@ object AeronStreamsApp { envelope.byteBuffer.flip() envelope } - .runWith(new AeronSink(channel1, streamId, aeron, taskRunner, pool)) + .runWith(new AeronSink(channel1, streamId, aeron, taskRunner, pool, giveUpSendAfter)) barrier.await() } @@ -303,7 +304,7 @@ object AeronStreamsApp { envelope.byteBuffer.flip() envelope } - .runWith(new AeronSink(channel1, streamId, aeron, taskRunner, pool)) + .runWith(new AeronSink(channel1, streamId, aeron, taskRunner, pool, giveUpSendAfter)) } def runStats(): Unit = { From e9e65c463f89c26f3579abc2e72fd53172bc13f9 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 19 May 2016 21:12:47 +0200 Subject: [PATCH 038/186] improve restart logging --- .../src/main/scala/akka/remote/artery/Association.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index cbc3abc118..7a3d8b2f66 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -265,16 +265,16 @@ private[akka] class Association( case _ if transport.isShutdown ⇒ // don't restart after shutdown case _: AbruptTerminationException ⇒ // ActorSystem shutdown case cause: GaveUpSendingException ⇒ - log.error(cause, "{} failed. Restarting it. {}", streamName, cause.getMessage) + log.debug("{} to {} failed. Restarting it. {}", streamName, remoteAddress, cause.getMessage) // restart unconditionally, without counting restarts restart(cause) case cause ⇒ if (restartCounter.restart()) { - log.error(cause, "{} failed. Restarting it. {}", streamName, cause.getMessage) + log.error(cause, "{} to {} failed. Restarting it. {}", streamName, remoteAddress, cause.getMessage) restart(cause) } else { - log.error(cause, "{} failed and restarted {} times within {} seconds. Terminating system. {}", - streamName, maxRestarts, restartTimeout.toSeconds, cause.getMessage) + log.error(cause, s"{} to {} failed and restarted {} times within {} seconds. Terminating system. ${cause.getMessage}", + streamName, remoteAddress, maxRestarts, restartTimeout.toSeconds) transport.system.terminate() } } From 5b7c978844fd0d84fa10cbdd4f7dc69ca8285df5 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 26 May 2016 10:42:08 +0200 Subject: [PATCH 039/186] add JMH benchmark for encoder decoder stage * CodecBenchmark that tests encode, decode and combined encode + decode * refactoring of codec stages to make it possible to run them without real ArteryTransport * also fixed a bug in inbound stream for large messages, it was using wrong envelope pool --- .../akka/remote/artery/BenchTestSource.scala | 63 ++++++ .../akka/remote/artery/CodecBenchmark.scala | 198 ++++++++++++++++++ .../scala/akka/remote/artery/LatchSink.scala | 36 ++++ .../akka/remote/artery/ArteryTransport.scala | 65 +++--- .../akka/remote/artery/Association.scala | 2 +- .../scala/akka/remote/artery/Codecs.scala | 34 +-- 6 files changed, 353 insertions(+), 45 deletions(-) create mode 100644 akka-bench-jmh/src/main/scala/akka/remote/artery/BenchTestSource.scala create mode 100644 akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala create mode 100644 akka-bench-jmh/src/main/scala/akka/remote/artery/LatchSink.scala 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..39e1de3ab9 --- /dev/null +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -0,0 +1,198 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.nio.ByteBuffer +import java.nio.ByteOrder + +import java.util.concurrent.CountDownLatch +import java.util.concurrent.TimeUnit + +import scala.concurrent.Await +import scala.concurrent.duration._ + +import akka.NotUsed +import akka.actor.ActorSystem +import akka.actor.ExtendedActorSystem +import akka.actor.InternalActorRef +import akka.actor.Props +import akka.actor.RootActorPath +import akka.remote.AddressUidExtension +import akka.remote.EndpointManager.Send +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._ + +@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 = "akka.remote.RemoteActorRefProvider" + 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) + + val envelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) + val compression = new Compression(system) + val headerIn = HeaderBuilder(compression) + val envelopeTemplateBuffer = ByteBuffer.allocate(ArteryTransport.MaximumFrameSize).order(ByteOrder.LITTLE_ENDIAN) + + val uniqueLocalAddress = UniqueAddress(system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress, + AddressUidExtension(system).addressUid) + val payload = Array.ofDim[Byte](1000) + + 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.version = 1 + headerIn.uid = 42 + headerIn.senderActorRef = senderStringA + headerIn.recipientActorRef = recipientStringB + headerIn.serializer = "4" + headerIn.classManifest = "" + 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) + + latch.await(30, TimeUnit.SECONDS) + } + + @Benchmark + @OperationsPerInvocation(100000) + def encode(): Unit = { + val latch = new CountDownLatch(1) + val N = 100000 + + val encoder: Flow[Send, EnvelopeBuffer, NotUsed] = + Flow.fromGraph(new Encoder(uniqueLocalAddress, system, compression, envelopePool)) + + Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) + .map(_ ⇒ Send(payload, None, remoteRefB, None)) + .via(encoder) + .map(envelope => envelopePool.release(envelope)) + .runWith(new LatchSink(N, latch))(materializer) + + latch.await(30, TimeUnit.SECONDS) + } + + @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(uniqueLocalAddress, system.asInstanceOf[ExtendedActorSystem], + resolveActorRefWithLocalAddress, compression, envelopePool)) + + Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) + .map { _ => + val envelope = envelopePool.acquire() + envelopeTemplateBuffer.rewind() + envelope.byteBuffer.put(envelopeTemplateBuffer) + envelope.byteBuffer.flip() + envelope + } + .via(decoder) + .runWith(new LatchSink(N, latch))(materializer) + + latch.await(30, TimeUnit.SECONDS) + } + + @Benchmark + @OperationsPerInvocation(100000) + def encode_decode(): Unit = { + val latch = new CountDownLatch(1) + val N = 100000 + + val encoder: Flow[Send, EnvelopeBuffer, NotUsed] = + Flow.fromGraph(new Encoder(uniqueLocalAddress, system, compression, envelopePool)) + + 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(uniqueLocalAddress, system.asInstanceOf[ExtendedActorSystem], + resolveActorRefWithLocalAddress, compression, envelopePool)) + + Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) + .map(_ ⇒ Send(payload, None, remoteRefB, None)) + .via(encoder) + .via(decoder) + .runWith(new LatchSink(N, latch))(materializer) + + latch.await(30, TimeUnit.SECONDS) + } + +} 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..c60bead1ce --- /dev/null +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/LatchSink.scala @@ -0,0 +1,36 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.util.concurrent.CountDownLatch + +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) + } +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index ece54fa867..d399264299 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -260,9 +260,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val restartCounter = new RestartCounter(maxRestarts, restartTimeout) val envelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) - val largeEnvelopePool: Option[EnvelopeBufferPool] = - if (largeMessageDestinationsEnabled) Some(new EnvelopeBufferPool(ArteryTransport.MaximumLargeFrameSize, ArteryTransport.MaximumPooledBuffers)) - else None + val largeEnvelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumLargeFrameSize, ArteryTransport.MaximumPooledBuffers) // FIXME: Compression table must be owned by each channel instead // of having a global one @@ -403,15 +401,12 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundLargeMessagesStream(): Unit = { - largeEnvelopePool.foreach { largePool ⇒ - // TODO just cargo-cult programming here - val completed = Source.fromGraph(new AeronSource(inboundChannel, largeStreamId, aeron, taskRunner, largePool)) - .async // FIXME measure - .via(inboundFlow) - .runWith(Sink.ignore)(materializer) + val completed = Source.fromGraph(new AeronSource(inboundChannel, largeStreamId, aeron, taskRunner, largeEnvelopePool)) + .async // FIXME measure + .via(inboundLargeFlow) + .runWith(Sink.ignore)(materializer) - attachStreamRestart("Inbound large message stream", completed, () ⇒ runInboundLargeMessagesStream()) - } + attachStreamRestart("Inbound large message stream", completed, () ⇒ runInboundLargeMessagesStream()) } private def attachStreamRestart(streamName: String, streamCompleted: Future[Done], restart: () ⇒ Unit): Unit = { @@ -491,15 +486,11 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } def outboundLarge(outboundContext: OutboundContext): Sink[Send, Future[Done]] = { - largeEnvelopePool match { - case Some(pool) ⇒ - Flow.fromGraph(killSwitch.flow[Send]) - .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval)) - .via(createEncoder(pool)) - .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), largeStreamId, aeron, taskRunner, - envelopePool, giveUpSendAfter))(Keep.right) - case None ⇒ throw new IllegalArgumentException("Trying to create outbound stream but outbound stream not configured") - } + Flow.fromGraph(killSwitch.flow[Send]) + .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval)) + .via(createEncoder(largeEnvelopePool)) + .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), largeStreamId, aeron, taskRunner, + envelopePool, giveUpSendAfter))(Keep.right) } def outboundControl(outboundContext: OutboundContext): Sink[Send, (OutboundControlIngress, Future[Done])] = { @@ -518,25 +509,41 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val dummySender = system.systemActorOf(Props.empty, "dummy") def createEncoder(pool: EnvelopeBufferPool): Flow[Send, EnvelopeBuffer, NotUsed] = - Flow.fromGraph(new Encoder(this, compression, pool)) - val encoder = createEncoder(envelopePool) + Flow.fromGraph(new Encoder(localAddress, system, compression, pool)) + + def encoder: Flow[Send, EnvelopeBuffer, NotUsed] = createEncoder(envelopePool) val messageDispatcherSink: Sink[InboundEnvelope, Future[Done]] = Sink.foreach[InboundEnvelope] { m ⇒ messageDispatcher.dispatch(m.recipient, m.recipientAddress, m.message, m.senderOption) } - val decoder = Flow.fromGraph(new Decoder(this, compression)) + def createDecoder(pool: EnvelopeBufferPool): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { + val resolveActorRefWithLocalAddress: String ⇒ InternalActorRef = + recipient ⇒ provider.resolveActorRefWithLocalAddress(recipient, localAddress.address) + Flow.fromGraph(new Decoder(localAddress, system, resolveActorRefWithLocalAddress, compression, pool)) + } - val inboundFlow: Flow[EnvelopeBuffer, ByteString, NotUsed] = { + def decoder: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = createDecoder(envelopePool) + + def inboundSink: Sink[InboundEnvelope, NotUsed] = + Flow[InboundEnvelope] + .via(new InboundHandshake(this, inControlStream = false)) + .via(new InboundQuarantineCheck(this)) + .to(messageDispatcherSink) + + def inboundFlow: Flow[EnvelopeBuffer, ByteString, NotUsed] = { Flow.fromSinkAndSource( - decoder - .via(new InboundHandshake(this, inControlStream = false)) - .via(new InboundQuarantineCheck(this)) - .to(messageDispatcherSink), + decoder.to(inboundSink), Source.maybe[ByteString].via(killSwitch.flow)) } - val inboundControlFlow: Flow[EnvelopeBuffer, ByteString, ControlMessageSubject] = { + def inboundLargeFlow: Flow[EnvelopeBuffer, ByteString, NotUsed] = { + Flow.fromSinkAndSource( + createDecoder(largeEnvelopePool).to(inboundSink), + Source.maybe[ByteString].via(killSwitch.flow)) + } + + def inboundControlFlow: Flow[EnvelopeBuffer, ByteString, ControlMessageSubject] = { Flow.fromSinkAndSourceMat( decoder .via(new InboundHandshake(this, inControlStream = true)) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 7a3d8b2f66..32482ff07f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -150,7 +150,7 @@ private[akka] class Association( private def isLargeMessageDestination(recipient: ActorRef): Boolean = { recipient match { - case r: RemoteActorRef if r.cachedLargeMessageDestinationFlag ne null ⇒ r.cachedLargeMessageDestinationFlag == LargeDestination + case r: RemoteActorRef if r.cachedLargeMessageDestinationFlag ne null ⇒ r.cachedLargeMessageDestinationFlag eq LargeDestination case r: RemoteActorRef ⇒ if (largeMessageDestinations.find(r.path.elements.iterator).data.isEmpty) { r.cachedLargeMessageDestinationFlag = RegularDestination diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 25e6fdba3a..fe30e36920 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -6,10 +6,13 @@ import akka.remote.{ MessageSerializer, UniqueAddress } import akka.serialization.{ Serialization, SerializationExtension } import akka.stream._ import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler } +import akka.actor.ActorSystem +import akka.actor.ExtendedActorSystem // TODO: Long UID class Encoder( - transport: ArteryTransport, + uniqueLocalAddress: UniqueAddress, + system: ActorSystem, compressionTable: LiteralCompressionTable, pool: EnvelopeBufferPool) extends GraphStage[FlowShape[Send, EnvelopeBuffer]] { @@ -23,11 +26,11 @@ class Encoder( private val headerBuilder = HeaderBuilder(compressionTable) headerBuilder.version = ArteryTransport.Version - headerBuilder.uid = transport.localAddress.uid - private val localAddress = transport.localAddress.address - private val serialization = SerializationExtension(transport.system) + headerBuilder.uid = uniqueLocalAddress.uid + private val localAddress = uniqueLocalAddress.address + private val serialization = SerializationExtension(system) - private val noSender = transport.system.deadLetters.path.toSerializationFormatWithAddress(localAddress) + private val noSender = system.deadLetters.path.toSerializationFormatWithAddress(localAddress) private val senderCache = new java.util.HashMap[ActorRef, String] private var recipientCache = new java.util.HashMap[ActorRef, String] @@ -66,7 +69,7 @@ class Encoder( } // FIXME: Thunk allocation - Serialization.currentTransportInformation.withValue(Serialization.Information(localAddress, transport.system)) { + Serialization.currentTransportInformation.withValue(Serialization.Information(localAddress, system)) { MessageSerializer.serializeForArtery(serialization, send.message.asInstanceOf[AnyRef], headerBuilder, envelope) } @@ -83,19 +86,20 @@ class Encoder( } class Decoder( - transport: ArteryTransport, - compressionTable: LiteralCompressionTable) extends GraphStage[FlowShape[EnvelopeBuffer, InboundEnvelope]] { + uniqueLocalAddress: UniqueAddress, + system: ExtendedActorSystem, + resolveActorRefWithLocalAddress: String ⇒ InternalActorRef, + compressionTable: LiteralCompressionTable, + pool: EnvelopeBufferPool) extends GraphStage[FlowShape[EnvelopeBuffer, InboundEnvelope]] { 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 GraphStageLogic(shape) with InHandler with OutHandler { - private val pool = transport.envelopePool - private val localAddress = transport.localAddress.address - private val provider = transport.provider + private val localAddress = uniqueLocalAddress.address private val headerBuilder = HeaderBuilder(compressionTable) - private val serialization = SerializationExtension(transport.system) + private val serialization = SerializationExtension(system) private val recipientCache = new java.util.HashMap[String, InternalActorRef] private val senderCache = new java.util.HashMap[String, Option[ActorRef]] @@ -111,7 +115,7 @@ class Decoder( // FIXME: Is localAddress really needed? val recipient: InternalActorRef = recipientCache.get(headerBuilder.recipientActorRef) match { case null ⇒ - val ref = provider.resolveActorRefWithLocalAddress(headerBuilder.recipientActorRef, localAddress) + val ref = resolveActorRefWithLocalAddress(headerBuilder.recipientActorRef) // FIXME we might need an efficient LRU cache, or replaced by compression table if (recipientCache.size() >= 1000) recipientCache.clear() @@ -122,7 +126,7 @@ class Decoder( val senderOption: Option[ActorRef] = senderCache.get(headerBuilder.senderActorRef) match { case null ⇒ - val ref = provider.resolveActorRefWithLocalAddress(headerBuilder.senderActorRef, localAddress) + val ref = resolveActorRefWithLocalAddress(headerBuilder.senderActorRef) // FIXME this cache will be replaced by compression table if (senderCache.size() >= 1000) senderCache.clear() @@ -135,7 +139,7 @@ class Decoder( val decoded = InboundEnvelope( recipient, localAddress, // FIXME: Is this needed anymore? What should we do here? - MessageSerializer.deserializeForArtery(transport.system, serialization, headerBuilder, envelope), + MessageSerializer.deserializeForArtery(system, serialization, headerBuilder, envelope), senderOption, // FIXME: No need for an option, decode simply to deadLetters instead UniqueAddress(senderOption.get.path.address, headerBuilder.uid)) // FIXME see issue #20568 From d759f738ea01fe8cf8ca99b582d64a54922b6301 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 26 May 2016 11:58:13 +0200 Subject: [PATCH 040/186] add serialization api based on ByteBuffer, #20324 * new trait ByteBufferSerializer with fromBinary and toBinary methods that takes ByteBuffer, this can be mixed in to existing serializer without breaking compatibility * implement the ByteBufferSerializer in the ByteArraySerializer * minor adjustment of the class manifest cache --- .../akka/serialization/SerializeSpec.scala | 23 +++++- akka-actor/src/main/resources/reference.conf | 2 +- .../akka/serialization/Serialization.scala | 67 +++++++++++++----- .../scala/akka/serialization/Serializer.scala | 70 ++++++++++++++++++- .../scala/akka/remote/MessageSerializer.scala | 34 ++++----- 5 files changed, 159 insertions(+), 37 deletions(-) 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 fa76783b5c..95f643d602 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 object SerializationTests { @@ -247,7 +248,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 b3aaf7eafd..a58c8e0c21 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -598,7 +598,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/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index dbc1f21d1c..a2633d05a2 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -14,6 +14,9 @@ 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 object Serialization { @@ -83,7 +86,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 ConcurrentHashMap[String, Option[Class[_]]] + 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 @@ -118,28 +121,60 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { 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 { - val cachedClassManifest = manifestCache.get(manifest) - if (cachedClassManifest ne null) - s1.fromBinary(bytes, cachedClassManifest) - else { + 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) ⇒ - manifestCache.put(manifest, Some(classManifest)) - s1.fromBinary(bytes, Some(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 [$serializerId].") + 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 serializerByIdentity(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. diff --git a/akka-actor/src/main/scala/akka/serialization/Serializer.scala b/akka-actor/src/main/scala/akka/serialization/Serializer.scala index 51bcc545ad..d550a769ca 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`. @@ -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) @@ -274,7 +325,22 @@ class ByteArraySerializer(val system: ExtendedActorSystem) extends BaseSerialize def toBinary(o: AnyRef) = 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-remote/src/main/scala/akka/remote/MessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala index 7119b96d2c..38699a04c9 100644 --- a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala @@ -9,6 +9,7 @@ import akka.protobuf.ByteString import akka.actor.ExtendedActorSystem import akka.remote.artery.{ EnvelopeBuffer, HeaderBuilder } import akka.serialization.{ Serialization, SerializationExtension, SerializerWithStringManifest } +import akka.serialization.ByteBufferSerializer /** * INTERNAL API @@ -53,27 +54,28 @@ private[akka] object MessageSerializer { // FIXME: This should be a FQCN instead headerBuilder.serializer = serializer.identifier.toString - serializer match { - case ser2: SerializerWithStringManifest ⇒ - val manifest = ser2.manifest(message) - headerBuilder.classManifest = manifest - case _ ⇒ - headerBuilder.classManifest = message.getClass.getName + + def manifest: String = serializer match { + case ser: SerializerWithStringManifest ⇒ ser.manifest(message) + case _ ⇒ if (serializer.includeManifest) message.getClass.getName else "" } - envelope.writeHeader(headerBuilder) - // FIXME: This should directly write to the buffer instead - envelope.byteBuffer.put(serializer.toBinary(message)) + serializer match { + case ser: ByteBufferSerializer ⇒ + headerBuilder.classManifest = manifest + envelope.writeHeader(headerBuilder) + ser.toBinary(message, envelope.byteBuffer) + case _ ⇒ + headerBuilder.classManifest = manifest + envelope.writeHeader(headerBuilder) + envelope.byteBuffer.put(serializer.toBinary(message)) + } } def deserializeForArtery(system: ExtendedActorSystem, serialization: Serialization, headerBuilder: HeaderBuilder, envelope: EnvelopeBuffer): AnyRef = { - // FIXME: Use the buffer directly - val size = envelope.byteBuffer.limit - envelope.byteBuffer.position - val bytes = Array.ofDim[Byte](size) - envelope.byteBuffer.get(bytes) - serialization.deserialize( - bytes, + serialization.deserializeByteBuffer( + envelope.byteBuffer, Integer.parseInt(headerBuilder.serializer), // FIXME: Use FQCN - headerBuilder.classManifest).get + headerBuilder.classManifest) } } From e74e1da6cca299043dba30d6193849ebe758e845 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 27 May 2016 08:50:41 +0200 Subject: [PATCH 041/186] optimize currentTransportInformation --- .../src/main/scala/akka/remote/artery/Codecs.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index fe30e36920..da662e9fcd 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -29,6 +29,7 @@ class Encoder( headerBuilder.uid = uniqueLocalAddress.uid private val localAddress = uniqueLocalAddress.address private val serialization = SerializationExtension(system) + private val serializationInfo = Serialization.Information(localAddress, system) private val noSender = system.deadLetters.path.toSerializationFormatWithAddress(localAddress) private val senderCache = new java.util.HashMap[ActorRef, String] @@ -68,10 +69,13 @@ class Encoder( headerBuilder.senderActorRef = noSender } - // FIXME: Thunk allocation - Serialization.currentTransportInformation.withValue(Serialization.Information(localAddress, system)) { + // avoiding currentTransportInformation.withValue due to thunk allocation + val oldValue = Serialization.currentTransportInformation.value + try { + Serialization.currentTransportInformation.value = serializationInfo MessageSerializer.serializeForArtery(serialization, send.message.asInstanceOf[AnyRef], headerBuilder, envelope) - } + } finally + Serialization.currentTransportInformation.value = oldValue //println(s"${headerBuilder.senderActorRef} --> ${headerBuilder.recipientActorRef} ${headerBuilder.classManifest}") From aa2c4fe7bf8cba8371da627bdda0623752f023f2 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 27 May 2016 11:24:08 +0200 Subject: [PATCH 042/186] handle serialization errors, #20324 --- .../scala/akka/remote/MessageSerializer.scala | 3 +- .../scala/akka/remote/artery/Codecs.scala | 77 +++++++++++----- .../artery/SerializationErrorSpec.scala | 87 +++++++++++++++++++ 3 files changed, 142 insertions(+), 25 deletions(-) create mode 100644 akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala diff --git a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala index 38699a04c9..09625d7fa5 100644 --- a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala @@ -72,7 +72,8 @@ private[akka] object MessageSerializer { } } - def deserializeForArtery(system: ExtendedActorSystem, serialization: Serialization, headerBuilder: HeaderBuilder, envelope: EnvelopeBuffer): AnyRef = { + def deserializeForArtery(system: ExtendedActorSystem, serialization: Serialization, headerBuilder: HeaderBuilder, + envelope: EnvelopeBuffer): AnyRef = { serialization.deserializeByteBuffer( envelope.byteBuffer, Integer.parseInt(headerBuilder.serializer), // FIXME: Use FQCN diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index da662e9fcd..454c01b663 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -1,13 +1,16 @@ package akka.remote.artery +import scala.util.control.NonFatal + import akka.actor.{ ActorRef, InternalActorRef } -import akka.remote.EndpointManager.Send +import akka.actor.ActorSystem +import akka.actor.ExtendedActorSystem import akka.remote.{ MessageSerializer, UniqueAddress } +import akka.remote.EndpointManager.Send +import akka.remote.artery.SystemMessageDelivery.SystemMessageEnvelope import akka.serialization.{ Serialization, SerializationExtension } import akka.stream._ import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler } -import akka.actor.ActorSystem -import akka.actor.ExtendedActorSystem // TODO: Long UID class Encoder( @@ -22,7 +25,7 @@ class Encoder( val shape: FlowShape[Send, EnvelopeBuffer] = FlowShape(in, out) override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = - new GraphStageLogic(shape) with InHandler with OutHandler { + new GraphStageLogic(shape) with InHandler with OutHandler with StageLogging { private val headerBuilder = HeaderBuilder(compressionTable) headerBuilder.version = ArteryTransport.Version @@ -35,6 +38,8 @@ class Encoder( private val senderCache = new java.util.HashMap[ActorRef, String] private var recipientCache = new java.util.HashMap[ActorRef, String] + override protected def logSource = classOf[Encoder] + override def onPush(): Unit = { val send = grab(in) val envelope = pool.acquire() @@ -69,18 +74,31 @@ class Encoder( headerBuilder.senderActorRef = noSender } - // avoiding currentTransportInformation.withValue due to thunk allocation - val oldValue = Serialization.currentTransportInformation.value try { - Serialization.currentTransportInformation.value = serializationInfo - MessageSerializer.serializeForArtery(serialization, send.message.asInstanceOf[AnyRef], headerBuilder, envelope) - } finally - Serialization.currentTransportInformation.value = oldValue + // avoiding currentTransportInformation.withValue due to thunk allocation + val oldValue = Serialization.currentTransportInformation.value + try { + Serialization.currentTransportInformation.value = serializationInfo + MessageSerializer.serializeForArtery(serialization, send.message.asInstanceOf[AnyRef], headerBuilder, envelope) + } finally + Serialization.currentTransportInformation.value = oldValue - //println(s"${headerBuilder.senderActorRef} --> ${headerBuilder.recipientActorRef} ${headerBuilder.classManifest}") + envelope.byteBuffer.flip() + push(out, envelope) + + } catch { + case NonFatal(e) ⇒ + pool.release(envelope) + send.message match { + case _: SystemMessageEnvelope ⇒ + log.error(e, "Failed to serialize system message [{}].", send.message.getClass.getName) + throw e + case _ ⇒ + log.error(e, "Failed to serialize message [{}].", send.message.getClass.getName) + pull(in) + } + } - envelope.byteBuffer.flip() - push(out, envelope) } override def onPull(): Unit = pull(in) @@ -100,7 +118,7 @@ class Decoder( val shape: FlowShape[EnvelopeBuffer, InboundEnvelope] = FlowShape(in, out) override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = - new GraphStageLogic(shape) with InHandler with OutHandler { + new GraphStageLogic(shape) with InHandler with OutHandler with StageLogging { private val localAddress = uniqueLocalAddress.address private val headerBuilder = HeaderBuilder(compressionTable) private val serialization = SerializationExtension(system) @@ -108,12 +126,12 @@ class Decoder( private val recipientCache = new java.util.HashMap[String, InternalActorRef] private val senderCache = new java.util.HashMap[String, Option[ActorRef]] + override protected def logSource = classOf[Decoder] + override def onPush(): Unit = { val envelope = grab(in) envelope.parseHeader(headerBuilder) - //println(s"${headerBuilder.recipientActorRef} <-- ${headerBuilder.senderActorRef} ${headerBuilder.classManifest}") - // FIXME: Instead of using Strings, the headerBuilder should automatically return cached ActorRef instances // in case of compression is enabled // FIXME: Is localAddress really needed? @@ -140,15 +158,26 @@ class Decoder( case refOpt ⇒ refOpt } - val decoded = InboundEnvelope( - recipient, - localAddress, // FIXME: Is this needed anymore? What should we do here? - MessageSerializer.deserializeForArtery(system, serialization, headerBuilder, envelope), - senderOption, // FIXME: No need for an option, decode simply to deadLetters instead - UniqueAddress(senderOption.get.path.address, headerBuilder.uid)) // FIXME see issue #20568 + try { + val deserializedMessage = MessageSerializer.deserializeForArtery( + system, serialization, headerBuilder, envelope) - pool.release(envelope) - push(out, decoded) + val decoded = InboundEnvelope( + recipient, + localAddress, // FIXME: Is this needed anymore? What should we do here? + deserializedMessage, + senderOption, // FIXME: No need for an option, decode simply to deadLetters instead + UniqueAddress(senderOption.get.path.address, headerBuilder.uid)) // FIXME see issue #20568 + + push(out, decoded) + } catch { + case NonFatal(e) ⇒ + log.warning("Failed to deserialize message with serializer id [{}] and manifest [{}]. {}", + headerBuilder.serializer, headerBuilder.classManifest, e.getMessage) + pull(in) + } finally { + pool.release(envelope) + } } override def onPull(): Unit = pull(in) 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..6b104257fe --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala @@ -0,0 +1,87 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.duration._ +import akka.actor.{ ActorIdentity, ActorSystem, ExtendedActorSystem, Identify, RootActorPath } +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 = "akka.remote.RemoteActorRefProvider" + remote.artery.enabled = on + remote.artery.hostname = localhost + remote.artery.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 = systemB.asInstanceOf[ExtendedActorSystem].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") + } + + } + +} From a0f8f19e8687b034a35beb5b9553418a0d368729 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 27 May 2016 11:47:34 +0200 Subject: [PATCH 043/186] change header to use serializer id instead of FQCN * because FQCN can become a problem for rolling upgrade scenarios where you want to rename serializer classes * also renamed classManifest to manifest because it doesn't have to be class names --- .../akka/remote/artery/CodecBenchmark.scala | 4 +- .../scala/akka/remote/MessageSerializer.scala | 11 +-- .../scala/akka/remote/artery/BufferPool.scala | 98 +++++++------------ .../scala/akka/remote/artery/Codecs.scala | 2 +- .../akka/remote/artery/Compression.scala | 3 - .../remote/artery/EnvelopeBufferSpec.scala | 50 +++++----- 6 files changed, 68 insertions(+), 100 deletions(-) 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 index 39e1de3ab9..03d73b1fde 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -86,10 +86,10 @@ class CodecBenchmark { val envelope = new EnvelopeBuffer(envelopeTemplateBuffer) headerIn.version = 1 headerIn.uid = 42 + headerIn.serializer = 4 headerIn.senderActorRef = senderStringA headerIn.recipientActorRef = recipientStringB - headerIn.serializer = "4" - headerIn.classManifest = "" + headerIn.manifest = "" envelope.writeHeader(headerIn) envelope.byteBuffer.put(payload) envelope.byteBuffer.flip() diff --git a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala index 09625d7fa5..a52fb00c50 100644 --- a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala @@ -52,8 +52,7 @@ private[akka] object MessageSerializer { def serializeForArtery(serialization: Serialization, message: AnyRef, headerBuilder: HeaderBuilder, envelope: EnvelopeBuffer): Unit = { val serializer = serialization.findSerializerFor(message) - // FIXME: This should be a FQCN instead - headerBuilder.serializer = serializer.identifier.toString + headerBuilder.serializer = serializer.identifier def manifest: String = serializer match { case ser: SerializerWithStringManifest ⇒ ser.manifest(message) @@ -62,11 +61,11 @@ private[akka] object MessageSerializer { serializer match { case ser: ByteBufferSerializer ⇒ - headerBuilder.classManifest = manifest + headerBuilder.manifest = manifest envelope.writeHeader(headerBuilder) ser.toBinary(message, envelope.byteBuffer) case _ ⇒ - headerBuilder.classManifest = manifest + headerBuilder.manifest = manifest envelope.writeHeader(headerBuilder) envelope.byteBuffer.put(serializer.toBinary(message)) } @@ -76,7 +75,7 @@ private[akka] object MessageSerializer { envelope: EnvelopeBuffer): AnyRef = { serialization.deserializeByteBuffer( envelope.byteBuffer, - Integer.parseInt(headerBuilder.serializer), // FIXME: Use FQCN - headerBuilder.classManifest) + headerBuilder.serializer, + headerBuilder.manifest) } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index 901391673d..60662dee0f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -50,10 +50,10 @@ private[remote] object EnvelopeBuffer { val VersionOffset = 0 val UidOffset = 4 - val SenderActorRefTagOffset = 8 - val RecipientActorRefTagOffset = 12 - val SerializerTagOffset = 16 - val ClassManifestTagOffset = 24 + val SerializerOffset = 8 + val SenderActorRefTagOffset = 12 + val RecipientActorRefTagOffset = 16 + val ClassManifestTagOffset = 20 val LiteralsSectionOffset = 32 @@ -70,9 +70,6 @@ private[remote] trait LiteralCompressionTable { def compressActorRef(ref: String): Int def decompressActorRef(idx: Int): String - def compressSerializer(serializer: String): Int - def decompressSerializer(idx: Int): String - def compressClassManifest(manifest: String): Int def decompressClassManifest(idx: Int): String @@ -100,11 +97,11 @@ sealed trait HeaderBuilder { def recipientActorRef_=(ref: String): Unit def recipientActorRef: String - def serializer_=(serializer: String): Unit - def serializer: String + def serializer_=(serializer: Int): Unit + def serializer: Int - def classManifest_=(manifest: String): Unit - def classManifest: String + def manifest_=(manifest: String): Unit + def manifest: String } /** @@ -120,10 +117,9 @@ private[remote] final class HeaderBuilderImpl(val compressionTable: LiteralCompr var _recipientActorRef: String = null var _recipientActorRefIdx: Int = -1 - var _serializer: String = null - var _serializerIdx: Int = -1 - var _classManifest: String = null - var _classManifestIdx: Int = -1 + var _serializer: Int = _ + var _manifest: String = null + var _manifestIdx: Int = -1 def senderActorRef_=(ref: String): Unit = { _senderActorRef = ref @@ -156,33 +152,27 @@ private[remote] final class HeaderBuilderImpl(val compressionTable: LiteralCompr } } - override def serializer_=(serializer: String): Unit = { + override def serializer_=(serializer: Int): Unit = { _serializer = serializer - _serializerIdx = compressionTable.compressSerializer(serializer) } - override def serializer: String = { - if (_serializer ne null) _serializer + override def serializer: Int = + _serializer + + override def manifest_=(manifest: String): Unit = { + _manifest = manifest + _manifestIdx = compressionTable.compressClassManifest(manifest) + } + + override def manifest: String = { + if (_manifest ne null) _manifest else { - _serializer = compressionTable.decompressSerializer(_serializerIdx) - _serializer + _manifest = compressionTable.decompressClassManifest(_manifestIdx) + _manifest } } - override def classManifest_=(manifest: String): Unit = { - _classManifest = manifest - _classManifestIdx = compressionTable.compressClassManifest(manifest) - } - - override def classManifest: String = { - if (_classManifest ne null) _classManifest - else { - _classManifest = compressionTable.decompressClassManifest(_classManifestIdx) - _classManifest - } - } - - override def toString = s"HeaderBuilderImpl($version, $uid, ${_senderActorRef}, ${_senderActorRefIdx}, ${_recipientActorRef}, ${_recipientActorRefIdx}, ${_serializer}, ${_serializerIdx}, ${_classManifest}, ${_classManifestIdx})" + override def toString = s"HeaderBuilderImpl($version, $uid, ${_senderActorRef}, ${_senderActorRefIdx}, ${_recipientActorRef}, ${_recipientActorRefIdx}, ${_serializer}, ${_manifest}, ${_manifestIdx})" } /** @@ -214,6 +204,7 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { // Write fixed length parts byteBuffer.putInt(header.version) byteBuffer.putInt(header.uid) + byteBuffer.putInt(header.serializer) // Write compressable, variable-length parts always to the actual position of the buffer // Write tag values explicitly in their proper offset @@ -231,17 +222,11 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { else writeLiteral(RecipientActorRefTagOffset, header._recipientActorRef) - // Serialize serializer - if (header._serializerIdx != -1) - byteBuffer.putInt(SerializerTagOffset, header._serializerIdx | TagTypeMask) - else - writeLiteral(SerializerTagOffset, header._serializer) - // Serialize class manifest - if (header._classManifestIdx != -1) - byteBuffer.putInt(ClassManifestTagOffset, header._classManifestIdx | TagTypeMask) + if (header._manifestIdx != -1) + byteBuffer.putInt(ClassManifestTagOffset, header._manifestIdx | TagTypeMask) else - writeLiteral(ClassManifestTagOffset, header._classManifest) + writeLiteral(ClassManifestTagOffset, header._manifest) } def parseHeader(h: HeaderBuilder): Unit = { @@ -250,6 +235,7 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { // Read fixed length parts header.version = byteBuffer.getInt header.uid = byteBuffer.getInt + header.serializer = byteBuffer.getInt // Read compressable, variable-length parts always from the actual position of the buffer // Read tag values explicitly from their proper offset @@ -275,24 +261,14 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { header._recipientActorRef = readLiteral() } - // Deserialize serializer - val serializerTag = byteBuffer.getInt(SerializerTagOffset) - if ((serializerTag & TagTypeMask) != 0) { - val idx = serializerTag & TagValueMask - header._serializer = null - header._serializerIdx = idx - } else { - header._serializer = readLiteral() - } - // Deserialize class manifest - val classManifestTag = byteBuffer.getInt(ClassManifestTagOffset) - if ((classManifestTag & TagTypeMask) != 0) { - val idx = classManifestTag & TagValueMask - header._classManifest = null - header._classManifestIdx = idx + val manifestTag = byteBuffer.getInt(ClassManifestTagOffset) + if ((manifestTag & TagTypeMask) != 0) { + val idx = manifestTag & TagValueMask + header._manifest = null + header._manifestIdx = idx } else { - header._classManifest = readLiteral() + header._manifest = readLiteral() } } @@ -313,4 +289,4 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { byteBuffer.put(literalBytes) } -} \ No newline at end of file +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 454c01b663..db557259c3 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -173,7 +173,7 @@ class Decoder( } catch { case NonFatal(e) ⇒ log.warning("Failed to deserialize message with serializer id [{}] and manifest [{}]. {}", - headerBuilder.serializer, headerBuilder.classManifest, e.getMessage) + headerBuilder.serializer, headerBuilder.manifest, e.getMessage) pull(in) } finally { pool.release(envelope) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Compression.scala b/akka-remote/src/main/scala/akka/remote/artery/Compression.scala index 364a7a5c9d..50206833a1 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Compression.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Compression.scala @@ -14,9 +14,6 @@ private[remote] class Compression(system: ActorSystem) extends LiteralCompressio override def compressActorRef(ref: String): Int = -1 override def decompressActorRef(idx: Int): String = ??? - override def compressSerializer(serializer: String): Int = -1 - override def decompressSerializer(idx: Int): String = ??? - override def compressClassManifest(manifest: String): Int = -1 override def decompressClassManifest(idx: Int): String = ??? diff --git a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala index dbd118e0ff..a8301dcb84 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala @@ -26,8 +26,6 @@ class EnvelopeBufferSpec extends AkkaSpec { override def compressActorRef(ref: String): Int = refToIdx.getOrElse(ref, -1) override def decompressActorRef(idx: Int): String = idxToRef(idx) - override def compressSerializer(serializer: String): Int = serializerToIdx.getOrElse(serializer, -1) - override def decompressSerializer(idx: Int): String = idxToSer(idx) override def compressClassManifest(manifest: String): Int = manifestToIdx.getOrElse(manifest, -1) override def decompressClassManifest(idx: Int): String = idxToManifest(idx) } @@ -42,10 +40,10 @@ class EnvelopeBufferSpec extends AkkaSpec { "be able to encode and decode headers with compressed literals" in { headerIn.version = 1 headerIn.uid = 42 + headerIn.serializer = 4 headerIn.senderActorRef = "compressable0" headerIn.recipientActorRef = "compressable1" - headerIn.serializer = "serializer0" - headerIn.classManifest = "manifest1" + headerIn.manifest = "manifest1" envelope.writeHeader(headerIn) envelope.byteBuffer.position() should ===(EnvelopeBuffer.LiteralsSectionOffset) // Fully compressed header @@ -55,26 +53,25 @@ class EnvelopeBufferSpec extends AkkaSpec { headerOut.version should ===(1) headerOut.uid should ===(42) + headerOut.serializer should ===(4) headerOut.senderActorRef should ===("compressable0") headerOut.recipientActorRef should ===("compressable1") - headerOut.serializer should ===("serializer0") - headerOut.classManifest should ===("manifest1") + headerOut.manifest should ===("manifest1") } "be able to encode and decode headers with uncompressed literals" in { headerIn.version = 1 headerIn.uid = 42 + headerIn.serializer = 4 headerIn.senderActorRef = "uncompressable0" headerIn.recipientActorRef = "uncompressable11" - headerIn.serializer = "uncompressable222" - headerIn.classManifest = "uncompressable3333" + headerIn.manifest = "uncompressable3333" val expectedHeaderLength = EnvelopeBuffer.LiteralsSectionOffset + // Constant header part 2 + headerIn.senderActorRef.length + // Length field + literal 2 + headerIn.recipientActorRef.length + // Length field + literal - 2 + headerIn.serializer.length + // Length field + literal - 2 + headerIn.classManifest.length // Length field + literal + 2 + headerIn.manifest.length // Length field + literal envelope.writeHeader(headerIn) envelope.byteBuffer.position() should ===(expectedHeaderLength) @@ -84,58 +81,57 @@ class EnvelopeBufferSpec extends AkkaSpec { headerOut.version should ===(1) headerOut.uid should ===(42) + headerOut.serializer should ===(4) headerOut.senderActorRef should ===("uncompressable0") headerOut.recipientActorRef should ===("uncompressable11") - headerOut.serializer should ===("uncompressable222") - headerOut.classManifest should ===("uncompressable3333") + headerOut.manifest should ===("uncompressable3333") } "be able to encode and decode headers with mixed literals" in { headerIn.version = 1 headerIn.uid = 42 + headerIn.serializer = 4 headerIn.senderActorRef = "reallylongcompressablestring" headerIn.recipientActorRef = "uncompressable1" - headerIn.serializer = "longuncompressedserializer" - headerIn.classManifest = "manifest1" + headerIn.manifest = "manifest1" envelope.writeHeader(headerIn) envelope.byteBuffer.position() should ===( EnvelopeBuffer.LiteralsSectionOffset + - 2 + headerIn.recipientActorRef.length + - 2 + headerIn.serializer.length) + 2 + headerIn.recipientActorRef.length) envelope.byteBuffer.flip() envelope.parseHeader(headerOut) headerOut.version should ===(1) headerOut.uid should ===(42) + headerOut.serializer should ===(4) headerOut.senderActorRef should ===("reallylongcompressablestring") headerOut.recipientActorRef should ===("uncompressable1") - headerOut.serializer should ===("longuncompressedserializer") - headerOut.classManifest should ===("manifest1") + headerOut.manifest should ===("manifest1") headerIn.version = 3 headerIn.uid = Int.MinValue + headerIn.serializer = -1 headerIn.senderActorRef = "uncompressable0" headerIn.recipientActorRef = "reallylongcompressablestring" - headerIn.serializer = "serializer0" - headerIn.classManifest = "longlonglongliteralmanifest" + headerIn.manifest = "longlonglongliteralmanifest" envelope.writeHeader(headerIn) envelope.byteBuffer.position() should ===( EnvelopeBuffer.LiteralsSectionOffset + 2 + headerIn.senderActorRef.length + - 2 + headerIn.classManifest.length) + 2 + headerIn.manifest.length) envelope.byteBuffer.flip() envelope.parseHeader(headerOut) headerOut.version should ===(3) headerOut.uid should ===(Int.MinValue) + headerOut.serializer should ===(-1) headerOut.senderActorRef should ===("uncompressable0") headerOut.recipientActorRef should ===("reallylongcompressablestring") - headerOut.serializer should ===("serializer0") - headerOut.classManifest should ===("longlonglongliteralmanifest") + headerOut.manifest should ===("longlonglongliteralmanifest") } "be able to encode and decode headers with mixed literals and payload" in { @@ -143,10 +139,10 @@ class EnvelopeBufferSpec extends AkkaSpec { headerIn.version = 1 headerIn.uid = 42 + headerIn.serializer = 4 headerIn.senderActorRef = "reallylongcompressablestring" headerIn.recipientActorRef = "uncompressable1" - headerIn.serializer = "serializer1" - headerIn.classManifest = "manifest1" + headerIn.manifest = "manifest1" envelope.writeHeader(headerIn) envelope.byteBuffer.put(payload.toByteBuffer) @@ -156,10 +152,10 @@ class EnvelopeBufferSpec extends AkkaSpec { headerOut.version should ===(1) headerOut.uid should ===(42) + headerOut.serializer should ===(4) headerOut.senderActorRef should ===("reallylongcompressablestring") headerOut.recipientActorRef should ===("uncompressable1") - headerOut.serializer should ===("serializer1") - headerOut.classManifest should ===("manifest1") + headerOut.manifest should ===("manifest1") ByteString.fromByteBuffer(envelope.byteBuffer) should ===(payload) } From 7505393c891d2152e48392b20f7af158f516fad1 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 25 May 2016 12:28:44 +0200 Subject: [PATCH 044/186] initiate new handshake after restart of receiving system, #20568 * we don't want to include the full origin address in each message, only the UID * that means that the restarted receiving system can't initate a new handshake immediately when it sees message from unknown origin * instead we inject HandshakeReq from the sending system once in a while (1 per second) which will trigger the new handshake * any messages that arrives before the HandshakeReq are dropped, but that is fine since the system was just restarted anyway * note that the injected handshake is only done for active connections, when a message is sent * also changed the UID to a Long, but there are more places in old remoting that must be changed before we actually can use a Long value fix lost first message, #20566 * the first message was sometimes dropped by the InboundHandshake stage because it came from unknown origin, i.e. the handshake had not completed * that happended because the ordinary messagage arrived before the first HandshakeReq, which may happen since we sent the HandshakeReq over the control stream * this changes so that HandshakeReq is sent over the same stream, not only on the control stream and thereby the HandshakeReq will arrive before any other message * always send HandshakeReq as first message * also when the handshake on sender side has been completed at startup * moved code from preStart to onPull --- .../artery/HandshakeRestartReceiverSpec.scala | 1 - .../akka/remote/AddressUidExtension.scala | 11 +- .../scala/akka/remote/UniqueAddress.scala | 7 +- .../akka/remote/artery/ArteryTransport.scala | 63 ++++---- .../akka/remote/artery/Association.scala | 47 +++++- .../scala/akka/remote/artery/BufferPool.scala | 24 +-- .../scala/akka/remote/artery/Codecs.scala | 2 +- .../scala/akka/remote/artery/Handshake.scala | 138 +++++++++++------- .../artery/InboundQuarantineCheck.scala | 21 ++- .../remote/artery/SystemMessageDelivery.scala | 7 +- .../remote/artery/EnvelopeBufferSpec.scala | 4 +- .../artery/InboundControlJunctionSpec.scala | 2 +- .../remote/artery/InboundHandshakeSpec.scala | 14 +- .../artery/LargeMessagesStreamSpec.scala | 10 +- .../remote/artery/OutboundHandshakeSpec.scala | 68 +++++++-- .../artery/SystemMessageDeliverySpec.scala | 2 +- .../akka/remote/artery/TestContext.scala | 22 ++- 17 files changed, 298 insertions(+), 145 deletions(-) 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 index 5fea761616..3c934011d3 100644 --- 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 @@ -91,7 +91,6 @@ abstract class HandshakeRestartReceiverSpec val (secondUid2, subject2) = identifyWithUid(secondRootPath, "subject2") secondUid2 should !==(secondUid) val secondUniqueRemoteAddress2 = Await.result(secondAssociation.associationState.uniqueRemoteAddress, 3.seconds) - println(s"# ${secondAssociation.associationState} secondUid $secondUid $secondUid2") // FIXME secondUniqueRemoteAddress2.uid should ===(secondUid2) secondUniqueRemoteAddress2.address should ===(secondAddress) secondUniqueRemoteAddress2 should !==(secondUniqueRemoteAddress) diff --git a/akka-remote/src/main/scala/akka/remote/AddressUidExtension.scala b/akka-remote/src/main/scala/akka/remote/AddressUidExtension.scala index 027f703a24..e83d6f7f42 100644 --- a/akka-remote/src/main/scala/akka/remote/AddressUidExtension.scala +++ b/akka-remote/src/main/scala/akka/remote/AddressUidExtension.scala @@ -25,5 +25,12 @@ object AddressUidExtension extends ExtensionId[AddressUidExtension] with Extensi } class AddressUidExtension(val system: ExtendedActorSystem) extends Extension { - val addressUid: Int = ThreadLocalRandom.current.nextInt() -} \ No newline at end of file + val longAddressUid: Long = { + // FIXME we should use a long here, but then we need to change in Cluster and RemoteWatcher also + //ThreadLocalRandom.current.nextLong() + ThreadLocalRandom.current.nextInt() + } + + @deprecated("Use longAddressUid instead", "2.4.x") + val addressUid: Int = longAddressUid.toInt +} diff --git a/akka-remote/src/main/scala/akka/remote/UniqueAddress.scala b/akka-remote/src/main/scala/akka/remote/UniqueAddress.scala index 7bb76716c5..d8ababfd03 100644 --- a/akka-remote/src/main/scala/akka/remote/UniqueAddress.scala +++ b/akka-remote/src/main/scala/akka/remote/UniqueAddress.scala @@ -6,12 +6,15 @@ package akka.remote import akka.actor.Address @SerialVersionUID(1L) -final case class UniqueAddress(address: Address, uid: Int) extends Ordered[UniqueAddress] { - override def hashCode = uid +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/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index d399264299..701d26bf00 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -5,8 +5,6 @@ package akka.remote.artery import java.io.File import java.nio.ByteOrder -import java.util.concurrent.ConcurrentHashMap -import java.util.function.{ Function ⇒ JFunction } import scala.concurrent.Future import scala.concurrent.Promise @@ -79,7 +77,7 @@ private[akka] final case class InboundEnvelope( recipientAddress: Address, message: AnyRef, senderOption: Option[ActorRef], - originAddress: UniqueAddress) + originUid: Long) /** * INTERNAL API @@ -103,6 +101,16 @@ private[akka] trait InboundContext { */ def association(remoteAddress: Address): OutboundContext + /** + * Lookup the outbound association for a given UID. + * Will return `null` if the UID is unknown, i.e. + * handshake not completed. `null` is used instead of `Optional` + * to avoid allocations. + */ + def association(uid: Long): OutboundContext + + def completeHandshake(peer: UniqueAddress): Unit + } /** @@ -150,7 +158,7 @@ private[akka] final class AssociationState( } def isQuarantined(uid: Long): Boolean = { - // FIXME does this mean boxing (allocation) because of Set[Long]? Use specialized Set. LongMap? + // FIXME does this mean boxing (allocation) because of Set[Long]? Use specialized Set. org.agrona.collections.LongHashSet? quarantined(uid) } @@ -183,8 +191,6 @@ private[akka] trait OutboundContext { def associationState: AssociationState - def completeHandshake(peer: UniqueAddress): Unit - def quarantine(reason: String): Unit /** @@ -236,6 +242,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val handshakeTimeout: FiniteDuration = system.settings.config.getMillisDuration("akka.remote.handshake-timeout").requiring(_ > Duration.Zero, "handshake-timeout must be > 0") + private val injectHandshakeInterval: FiniteDuration = 1.second private val giveUpSendAfter: FiniteDuration = 60.seconds private val largeMessageDestinations = @@ -252,9 +259,6 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val largeStreamId = 4 private val taskRunner = new TaskRunner(system) - // FIXME: This does locking on putIfAbsent, we need something smarter - private[this] val associations = new ConcurrentHashMap[Address, Association]() - private val restartTimeout: FiniteDuration = 5.seconds // FIXME config private val maxRestarts = 5 // FIXME config private val restartCounter = new RestartCounter(maxRestarts, restartTimeout) @@ -266,6 +270,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // of having a global one val compression = new Compression(system) + private val associationRegistry = new AssociationRegistry( + remoteAddress ⇒ new Association(this, materializer, remoteAddress, controlSubject, largeMessageDestinations)) + override def start(): Unit = { startMediaDriver() startAeron() @@ -280,7 +287,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // TODO: Have a supervisor actor _localAddress = UniqueAddress( Address("artery", system.name, remoteSettings.ArteryHostname, port), - AddressUidExtension(system).addressUid) + AddressUidExtension(system).longAddressUid) materializer = ActorMaterializer()(system) messageDispatcher = new MessageDispatcher(system, provider) @@ -457,29 +464,28 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R a.send(message, senderOption, recipient) } - override def association(remoteAddress: Address): Association = { - val current = associations.get(remoteAddress) - if (current ne null) current - else { - associations.computeIfAbsent(remoteAddress, new JFunction[Address, Association] { - override def apply(remoteAddress: Address): Association = { - val newAssociation = new Association(ArteryTransport.this, materializer, remoteAddress, controlSubject, largeMessageDestinations) - newAssociation.associate() // This is a bit costly for this blocking method :( - newAssociation - } - }) - } + override def association(remoteAddress: Address): Association = + associationRegistry.association(remoteAddress) + + override def association(uid: Long): Association = + associationRegistry.association(uid) + + override def completeHandshake(peer: UniqueAddress): Unit = { + val a = associationRegistry.setUID(peer) + a.completeHandshake(peer) } private def publishLifecycleEvent(event: RemotingLifecycleEvent): Unit = eventPublisher.notifyListeners(event) - override def quarantine(remoteAddress: Address, uid: Option[Int]): Unit = - association(remoteAddress).quarantine(reason = "", uid) // FIXME change the method signature (old remoting) to include reason? + override def quarantine(remoteAddress: Address, uid: Option[Int]): Unit = { + // FIXME change the method signature (old remoting) to include reason and use Long uid? + association(remoteAddress).quarantine(reason = "", uid.map(_.toLong)) + } def outbound(outboundContext: OutboundContext): Sink[Send, Future[Done]] = { Flow.fromGraph(killSwitch.flow[Send]) - .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval)) + .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) .via(encoder) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), ordinaryStreamId, aeron, taskRunner, envelopePool, giveUpSendAfter))(Keep.right) @@ -487,7 +493,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R def outboundLarge(outboundContext: OutboundContext): Sink[Send, Future[Done]] = { Flow.fromGraph(killSwitch.flow[Send]) - .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval)) + .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) .via(createEncoder(largeEnvelopePool)) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), largeStreamId, aeron, taskRunner, envelopePool, giveUpSendAfter))(Keep.right) @@ -495,7 +501,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R def outboundControl(outboundContext: OutboundContext): Sink[Send, (OutboundControlIngress, Future[Done])] = { Flow.fromGraph(killSwitch.flow[Send]) - .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval)) + .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) .via(new SystemMessageDelivery(outboundContext, systemMessageResendInterval, remoteSettings.SysMsgBufferSize)) .viaMat(new OutboundControlJunction(outboundContext))(Keep.right) .via(encoder) @@ -505,9 +511,6 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // FIXME we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages } - // FIXME hack until real envelopes, encoding originAddress in sender :) - private val dummySender = system.systemActorOf(Props.empty, "dummy") - def createEncoder(pool: EnvelopeBufferPool): Flow[Send, EnvelopeBuffer, NotUsed] = Flow.fromGraph(new Encoder(localAddress, system, compression, pool)) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 32482ff07f..8e9db7092d 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -3,8 +3,10 @@ */ package akka.remote.artery +import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.CountDownLatch import java.util.concurrent.TimeUnit +import java.util.function.{ Function ⇒ JFunction } import scala.annotation.tailrec import scala.concurrent.Future @@ -12,6 +14,7 @@ import scala.concurrent.Promise import scala.concurrent.duration._ import scala.concurrent.duration.FiniteDuration import scala.util.Success + import akka.{ Done, NotUsed } import akka.actor.ActorRef import akka.actor.ActorSelectionMessage @@ -19,8 +22,8 @@ import akka.actor.Address import akka.actor.RootActorPath import akka.dispatch.sysmsg.SystemMessage import akka.event.Logging -import akka.remote.EndpointManager.Send import akka.remote.{ LargeDestination, RegularDestination, RemoteActorRef, UniqueAddress } +import akka.remote.EndpointManager.Send import akka.remote.artery.AeronSink.GaveUpSendingException import akka.remote.artery.InboundControlJunction.ControlMessageSubject import akka.remote.artery.OutboundControlJunction.OutboundControlIngress @@ -99,7 +102,7 @@ private[akka] class Association( def associationState: AssociationState = Unsafe.instance.getObjectVolatile(this, AbstractAssociation.sharedStateOffset).asInstanceOf[AssociationState] - override def completeHandshake(peer: UniqueAddress): Unit = { + def completeHandshake(peer: UniqueAddress): Unit = { require(remoteAddress == peer.address, s"wrong remote address in completeHandshake, got ${peer.address}, expected ${remoteAddress}") val current = associationState @@ -177,7 +180,7 @@ private[akka] class Association( quarantine(reason, uid) } - @tailrec final def quarantine(reason: String, uid: Option[Int]): Unit = { + @tailrec final def quarantine(reason: String, uid: Option[Long]): Unit = { uid match { case Some(u) ⇒ val current = associationState @@ -279,4 +282,42 @@ private[akka] class Association( } } } + + override def toString(): String = + s"Association($localAddress -> $remoteAddress with $associationState)" + +} + +/** + * INTERNAL API + */ +private[remote] class AssociationRegistry(createAssociation: Address ⇒ Association) { + // FIXME: This does locking on putIfAbsent, we need something smarter + private[this] val associationsByAddress = new ConcurrentHashMap[Address, Association]() + private[this] val associationsByUid = new ConcurrentHashMap[Long, Association]() + + def association(remoteAddress: Address): Association = { + val current = associationsByAddress.get(remoteAddress) + if (current ne null) current + else { + associationsByAddress.computeIfAbsent(remoteAddress, new JFunction[Address, Association] { + override def apply(remoteAddress: Address): Association = { + val newAssociation = createAssociation(remoteAddress) + newAssociation.associate() // This is a bit costly for this blocking method :( + newAssociation + } + }) + } + } + + def association(uid: Long): Association = + associationsByUid.get(uid) + + def setUID(peer: UniqueAddress): Association = { + val a = association(peer.address) + val previous = associationsByUid.put(peer.uid, a) + if ((previous ne null) && (previous ne a)) + throw new IllegalArgumentException(s"UID collision old [$previous] new [$a]") + a + } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index 60662dee0f..858309b10e 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -48,14 +48,14 @@ private[remote] object EnvelopeBuffer { val TagTypeMask = 0xFF000000 val TagValueMask = 0x0000FFFF - val VersionOffset = 0 - val UidOffset = 4 - val SerializerOffset = 8 - val SenderActorRefTagOffset = 12 - val RecipientActorRefTagOffset = 16 - val ClassManifestTagOffset = 20 + val VersionOffset = 0 // Int + val UidOffset = 4 // Long + val SerializerOffset = 12 // Int + val SenderActorRefTagOffset = 16 // Int + val RecipientActorRefTagOffset = 20 // Int + val ClassManifestTagOffset = 24 // Int - val LiteralsSectionOffset = 32 + val LiteralsSectionOffset = 28 val UsAscii = Charset.forName("US-ASCII") @@ -86,8 +86,8 @@ sealed trait HeaderBuilder { def version_=(v: Int): Unit def version: Int - def uid_=(u: Int): Unit - def uid: Int + def uid_=(u: Long): Unit + def uid: Long def senderActorRef_=(ref: String): Unit def senderActorRef: String @@ -109,7 +109,7 @@ sealed trait HeaderBuilder { */ private[remote] final class HeaderBuilderImpl(val compressionTable: LiteralCompressionTable) extends HeaderBuilder { var version: Int = _ - var uid: Int = _ + var uid: Long = _ // Fields only available for EnvelopeBuffer var _senderActorRef: String = null @@ -203,7 +203,7 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { // Write fixed length parts byteBuffer.putInt(header.version) - byteBuffer.putInt(header.uid) + byteBuffer.putLong(header.uid) byteBuffer.putInt(header.serializer) // Write compressable, variable-length parts always to the actual position of the buffer @@ -234,7 +234,7 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { // Read fixed length parts header.version = byteBuffer.getInt - header.uid = byteBuffer.getInt + header.uid = byteBuffer.getLong header.serializer = byteBuffer.getInt // Read compressable, variable-length parts always from the actual position of the buffer diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index db557259c3..1457373146 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -167,7 +167,7 @@ class Decoder( localAddress, // FIXME: Is this needed anymore? What should we do here? deserializedMessage, senderOption, // FIXME: No need for an option, decode simply to deadLetters instead - UniqueAddress(senderOption.get.path.address, headerBuilder.uid)) // FIXME see issue #20568 + headerBuilder.uid) push(out, decoded) } catch { diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala index cb2fe64f24..8be3029670 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -4,7 +4,6 @@ package akka.remote.artery import scala.concurrent.duration._ -import scala.util.Success import scala.util.control.NoStackTrace import akka.remote.EndpointManager.Send @@ -41,13 +40,15 @@ private[akka] object OutboundHandshake { private case object HandshakeTimeout private case object HandshakeRetryTick + private case object InjectHandshakeTick } /** * INTERNAL API */ -private[akka] class OutboundHandshake(outboundContext: OutboundContext, timeout: FiniteDuration, retryInterval: FiniteDuration) +private[akka] class OutboundHandshake(outboundContext: OutboundContext, timeout: FiniteDuration, + retryInterval: FiniteDuration, injectHandshakeInterval: FiniteDuration) extends GraphStage[FlowShape[Send, Send]] { val in: Inlet[Send] = Inlet("OutboundHandshake.in") @@ -59,50 +60,71 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext, timeout: import OutboundHandshake._ private var handshakeState: HandshakeState = Start - - override def preStart(): Unit = { - val uniqueRemoteAddress = outboundContext.associationState.uniqueRemoteAddress - if (uniqueRemoteAddress.isCompleted) { - handshakeState = Completed - } else { - // 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 - } - } - } + private var pendingMessage: Send = null + private var injectHandshakeTickScheduled = false // InHandler override def onPush(): Unit = { if (handshakeState != Completed) throw new IllegalStateException(s"onPush before handshake completed, was [$handshakeState]") - push(out, grab(in)) + + // 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 ⇒ pull(in) + case Completed ⇒ + if (pendingMessage eq null) + pull(in) + else { + push(out, pendingMessage) + pendingMessage = null + } + case Start ⇒ - // will pull when handshake reply is received (uniqueRemoteAddress completed) - handshakeState = ReqInProgress - scheduleOnce(HandshakeTimeout, timeout) - schedulePeriodically(HandshakeRetryTick, retryInterval) - sendHandshakeReq() + 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 sendHandshakeReq(): Unit = - outboundContext.sendControl(HandshakeReq(outboundContext.localAddress)) + private def pushHandshakeReq(): Unit = { + injectHandshakeTickScheduled = true + scheduleOnce(InjectHandshakeTick, injectHandshakeInterval) + push(out, Send(HandshakeReq(outboundContext.localAddress), None, outboundContext.dummyRecipient, None)) + } private def handshakeCompleted(): Unit = { handshakeState = Completed @@ -112,10 +134,13 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext, timeout: override protected def onTimer(timerKey: Any): Unit = timerKey match { + case InjectHandshakeTick ⇒ + // next onPush message will trigger sending of HandshakeReq + injectHandshakeTickScheduled = false case HandshakeRetryTick ⇒ - sendHandshakeReq() + if (isAvailable(out)) + pushHandshakeReq() case HandshakeTimeout ⇒ - // FIXME would it make sense to retry a few times before failing? failStage(new HandshakeTimeoutException( s"Handshake with [${outboundContext.remoteAddress}] did not complete within ${timeout.toMillis} ms")) } @@ -134,7 +159,7 @@ private[akka] class InboundHandshake(inboundContext: InboundContext, inControlSt override val shape: FlowShape[InboundEnvelope, InboundEnvelope] = FlowShape(in, out) override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = - new TimerGraphStageLogic(shape) with OutHandler { + new TimerGraphStageLogic(shape) with OutHandler with StageLogging { import OutboundHandshake._ // InHandler @@ -143,11 +168,9 @@ private[akka] class InboundHandshake(inboundContext: InboundContext, inControlSt override def onPush(): Unit = { grab(in) match { case InboundEnvelope(_, _, HandshakeReq(from), _, _) ⇒ - inboundContext.association(from.address).completeHandshake(from) - inboundContext.sendControl(from.address, HandshakeRsp(inboundContext.localAddress)) - pull(in) + onHandshakeReq(from) case InboundEnvelope(_, _, HandshakeRsp(from), _, _) ⇒ - inboundContext.association(from.address).completeHandshake(from) + inboundContext.completeHandshake(from) pull(in) case other ⇒ onMessage(other) } @@ -155,29 +178,42 @@ private[akka] class InboundHandshake(inboundContext: InboundContext, inControlSt }) else setHandler(in, new InHandler { - override def onPush(): Unit = onMessage(grab(in)) + override def onPush(): Unit = { + grab(in) match { + case InboundEnvelope(_, _, HandshakeReq(from), _, _) ⇒ + onHandshakeReq(from) + case other ⇒ onMessage(other) + } + } }) + private def onHandshakeReq(from: UniqueAddress): Unit = { + inboundContext.completeHandshake(from) + inboundContext.sendControl(from.address, HandshakeRsp(inboundContext.localAddress)) + pull(in) + } + private def onMessage(env: InboundEnvelope): Unit = { - if (isKnownOrigin(env.originAddress)) + if (isKnownOrigin(env.originUid)) push(out, env) else { - inboundContext.sendControl(env.originAddress.address, HandshakeReq(inboundContext.localAddress)) - // FIXME Note that we have the originAddress that would be needed to complete the handshake - // but it is not done here because the handshake might exchange more information. - // Is that a valid thought? - // drop message from unknown, this system was probably restarted + // FIXME remove, only debug + log.warning(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, inboundContext.localAddress.uid, env.originUid) + 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, inboundContext.localAddress.uid, env.originUid) pull(in) } } - private def isKnownOrigin(originAddress: UniqueAddress): Boolean = { + private def isKnownOrigin(originUid: Long): Boolean = { // FIXME these association lookups are probably too costly for each message, need local cache or something - val associationState = inboundContext.association(originAddress.address).associationState - associationState.uniqueRemoteAddressValue() match { - case Some(Success(a)) if a.uid == originAddress.uid ⇒ true - case x ⇒ false - } + (inboundContext.association(originUid) ne null) } // OutHandler diff --git a/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala b/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala index cf0a916ee9..2cdb2e3542 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala @@ -7,11 +7,11 @@ 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 /** * INTERNAL API @@ -27,13 +27,18 @@ private[akka] class InboundQuarantineCheck(inboundContext: InboundContext) exten // InHandler override def onPush(): Unit = { val env = grab(in) - val association = inboundContext.association(env.originAddress.address) - if (association.associationState.isQuarantined(env.originAddress.uid)) { - inboundContext.sendControl(env.originAddress.address, - Quarantined(inboundContext.localAddress, env.originAddress)) - pull(in) - } else - push(out, env) + inboundContext.association(env.originUid) match { + case null ⇒ + // unknown, handshake not completed + push(out, env) + case association ⇒ + if (association.associationState.isQuarantined(env.originUid)) { + inboundContext.sendControl(association.remoteAddress, + Quarantined(inboundContext.localAddress, UniqueAddress(association.remoteAddress, env.originUid))) + pull(in) + } else + push(out, env) + } } // OutHandler diff --git a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala index 747fcad084..472ce89fe7 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -4,13 +4,11 @@ package akka.remote.artery 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.EndpointManager.Send import akka.remote.UniqueAddress @@ -24,6 +22,7 @@ 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 /** * INTERNAL API @@ -159,6 +158,10 @@ private[akka] class SystemMessageDelivery( // InHandler override def onPush(): Unit = { grab(in) match { + case s @ Send(_: HandshakeReq, _, _, _) ⇒ + // pass on HandshakeReq + if (isAvailable(out)) + push(out, s) case s @ Send(ClearSystemMessageDelivery, _, _, _) ⇒ clear() pull(in) diff --git a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala index a8301dcb84..67b5ce8a3a 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala @@ -111,7 +111,7 @@ class EnvelopeBufferSpec extends AkkaSpec { headerOut.manifest should ===("manifest1") headerIn.version = 3 - headerIn.uid = Int.MinValue + headerIn.uid = Long.MinValue headerIn.serializer = -1 headerIn.senderActorRef = "uncompressable0" headerIn.recipientActorRef = "reallylongcompressablestring" @@ -127,7 +127,7 @@ class EnvelopeBufferSpec extends AkkaSpec { envelope.parseHeader(headerOut) headerOut.version should ===(3) - headerOut.uid should ===(Int.MinValue) + headerOut.uid should ===(Long.MinValue) headerOut.serializer should ===(-1) headerOut.senderActorRef should ===("uncompressable0") headerOut.recipientActorRef should ===("reallylongcompressablestring") diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala index 389d843b5f..0981ade341 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala @@ -42,7 +42,7 @@ class InboundControlJunctionSpec extends AkkaSpec with ImplicitSender { val recipient = null.asInstanceOf[InternalActorRef] // not used val ((upstream, controlSubject), downstream) = TestSource.probe[AnyRef] - .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, None, addressA)) + .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, None, addressA.uid)) .viaMat(new InboundControlJunction)(Keep.both) .map { case InboundEnvelope(_, _, msg, _, _) ⇒ msg } .toMat(TestSink.probe[Any])(Keep.both) diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala index 7266937206..4a1774a7bc 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala @@ -41,7 +41,7 @@ class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { private def setupStream(inboundContext: InboundContext, timeout: FiniteDuration = 5.seconds): (TestPublisher.Probe[AnyRef], TestSubscriber.Probe[Any]) = { val recipient = null.asInstanceOf[InternalActorRef] // not used TestSource.probe[AnyRef] - .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, None, addressA)) + .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, None, addressA.uid)) .via(new InboundHandshake(inboundContext, inControlStream = true)) .map { case InboundEnvelope(_, _, msg, _, _) ⇒ msg } .toMat(TestSink.probe[Any])(Keep.both) @@ -77,7 +77,7 @@ class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { downstream.cancel() } - "send HandshakeReq when receiving message from unknown (receiving system restarted)" in { + "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) @@ -85,8 +85,16 @@ class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { downstream.request(10) // no HandshakeReq upstream.sendNext("msg17") - replyProbe.expectMsg(HandshakeReq(addressB)) downstream.expectNoMsg(200.millis) // messages from unknown are dropped + + // and accept messages after handshake + upstream.sendNext(HandshakeReq(addressA)) + 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 index b56b3ff795..7a8343099b 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala @@ -57,19 +57,19 @@ class LargeMessagesStreamSpec extends WordSpec with ShouldMatchers with ScalaFut val senderProbeB = TestProbe()(systemB) // start actor and make sure it is up and running - val large = systemB.actorOf(Props(new EchoSize), "regular") - large.tell(Ping(), senderProbeB.ref) + 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 addressB = systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress val rootB = RootActorPath(addressB) - val largeRemote = awaitResolve(systemA.actorSelection(rootB / "user" / "regular")) - largeRemote.tell(Ping(), senderProbeA.ref) + val regularRemote = awaitResolve(systemA.actorSelection(rootB / "user" / "regular")) + regularRemote.tell(Ping(), senderProbeA.ref) senderProbeA.expectMsg(Pong(0)) // flag should be cached now - largeRemote.asInstanceOf[RemoteActorRef].cachedLargeMessageDestinationFlag should ===(RegularDestination) + regularRemote.asInstanceOf[RemoteActorRef].cachedLargeMessageDestinationFlag should ===(RegularDestination) } finally { TestKit.shutdownActorSystem(systemA) diff --git a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala index 3602d0fa66..df5f2d8189 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala @@ -31,12 +31,15 @@ class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { val addressA = UniqueAddress(Address("artery", "sysA", "hostA", 1001), 1) val addressB = UniqueAddress(Address("artery", "sysB", "hostB", 1002), 2) - private def setupStream(outboundContext: OutboundContext, timeout: FiniteDuration = 5.seconds, - retryInterval: FiniteDuration = 10.seconds): (TestPublisher.Probe[String], TestSubscriber.Probe[Any]) = { + private def setupStream( + outboundContext: OutboundContext, timeout: FiniteDuration = 5.seconds, + retryInterval: FiniteDuration = 10.seconds, + injectHandshakeInterval: FiniteDuration = 10.seconds): (TestPublisher.Probe[String], TestSubscriber.Probe[Any]) = { + val destination = null.asInstanceOf[RemoteActorRef] // not used TestSource.probe[String] .map(msg ⇒ Send(msg, None, destination, None)) - .via(new OutboundHandshake(outboundContext, timeout, retryInterval)) + .via(new OutboundHandshake(outboundContext, timeout, retryInterval, injectHandshakeInterval)) .map { case Send(msg, _, _, _) ⇒ msg } .toMat(TestSink.probe[Any])(Keep.both) .run() @@ -44,13 +47,25 @@ class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { "OutboundHandshake stage" must { "send HandshakeReq when first pulled" in { - val controlProbe = TestProbe() - val inboundContext = new TestInboundContext(localAddress = addressA, controlProbe = Some(controlProbe.ref)) + val inboundContext = new TestInboundContext(localAddress = addressA) val outboundContext = inboundContext.association(addressB.address) val (upstream, downstream) = setupStream(outboundContext) downstream.request(10) - controlProbe.expectMsg(HandshakeReq(addressA)) + downstream.expectNext(HandshakeReq(addressA)) + 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)) + downstream.expectNext("msg1") downstream.cancel() } @@ -60,40 +75,63 @@ class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { val (upstream, downstream) = setupStream(outboundContext, timeout = 200.millis) downstream.request(1) + downstream.expectNext(HandshakeReq(addressA)) downstream.expectError().getClass should be(classOf[HandshakeTimeoutException]) } "retry HandshakeReq" in { - val controlProbe = TestProbe() - val inboundContext = new TestInboundContext(localAddress = addressA, controlProbe = Some(controlProbe.ref)) + val inboundContext = new TestInboundContext(localAddress = addressA) val outboundContext = inboundContext.association(addressB.address) val (upstream, downstream) = setupStream(outboundContext, retryInterval = 100.millis) downstream.request(10) - controlProbe.expectMsg(HandshakeReq(addressA)) - controlProbe.expectMsg(HandshakeReq(addressA)) - controlProbe.expectMsg(HandshakeReq(addressA)) + downstream.expectNext(HandshakeReq(addressA)) + downstream.expectNext(HandshakeReq(addressA)) + downstream.expectNext(HandshakeReq(addressA)) downstream.cancel() } "not deliver messages from upstream until handshake completed" in { - val controlProbe = TestProbe() - val inboundContext = new TestInboundContext(localAddress = addressA, controlProbe = Some(controlProbe.ref)) + val inboundContext = new TestInboundContext(localAddress = addressA) val outboundContext = inboundContext.association(addressB.address) val (upstream, downstream) = setupStream(outboundContext) downstream.request(10) - controlProbe.expectMsg(HandshakeReq(addressA)) + downstream.expectNext(HandshakeReq(addressA)) upstream.sendNext("msg1") downstream.expectNoMsg(200.millis) // InboundHandshake stage will complete the handshake when receiving HandshakeRsp - inboundContext.association(addressB.address).completeHandshake(addressB) + 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)) + inboundContext.completeHandshake(addressB) + downstream.expectNext("msg1") + + downstream.expectNoMsg(600.millis) + upstream.sendNext("msg2") + upstream.sendNext("msg3") + upstream.sendNext("msg4") + downstream.expectNext(HandshakeReq(addressA)) + 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/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index 55fe0b941f..f9173f04e3 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -77,7 +77,7 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi Flow[Send] .map { case Send(sysEnv: SystemMessageEnvelope, _, _, _) ⇒ - InboundEnvelope(recipient, addressB.address, sysEnv, None, addressA) + InboundEnvelope(recipient, addressB.address, sysEnv, None, addressA.uid) } .async .via(new SystemMessageAcker(inboundContext)) diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala index 30ed97698f..561150f8fc 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -25,7 +25,8 @@ private[akka] class TestInboundContext( val controlProbe: Option[ActorRef] = None, val replyDropRate: Double = 0.0) extends InboundContext { - private val associations = new ConcurrentHashMap[Address, OutboundContext] + 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) @@ -33,17 +34,26 @@ private[akka] class TestInboundContext( } override def association(remoteAddress: Address): OutboundContext = - associations.get(remoteAddress) match { + associationsByAddress.get(remoteAddress) match { case null ⇒ val a = createAssociation(remoteAddress) - associations.putIfAbsent(remoteAddress, a) match { + associationsByAddress.putIfAbsent(remoteAddress, a) match { case null ⇒ a case existing ⇒ existing } case existing ⇒ existing } - protected def createAssociation(remoteAddress: Address): OutboundContext = + override def association(uid: Long): OutboundContext = + associationsByUid.get(uid) + + override def completeHandshake(peer: UniqueAddress): Unit = { + val a = association(peer.address).asInstanceOf[TestOutboundContext] + a.completeHandshake(peer) + associationsByUid.put(peer.uid, a) + } + + protected def createAssociation(remoteAddress: Address): TestOutboundContext = new TestOutboundContext(localAddress, remoteAddress, controlSubject, controlProbe) } @@ -60,7 +70,7 @@ private[akka] class TestOutboundContext( _associationState } - override def completeHandshake(peer: UniqueAddress): Unit = synchronized { + def completeHandshake(peer: UniqueAddress): Unit = synchronized { _associationState.uniqueRemoteAddressPromise.trySuccess(peer) _associationState.uniqueRemoteAddress.value match { case Some(Success(`peer`)) ⇒ // our value @@ -75,7 +85,7 @@ private[akka] class TestOutboundContext( override def sendControl(message: ControlMessage) = { controlProbe.foreach(_ ! message) - controlSubject.sendControl(InboundEnvelope(null, remoteAddress, message, None, localAddress)) + controlSubject.sendControl(InboundEnvelope(null, remoteAddress, message, None, localAddress.uid)) } // FIXME we should be able to Send without a recipient ActorRef From c6464ce49337f5e51b2fd64125265721be8977c0 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 30 May 2016 10:42:49 +0200 Subject: [PATCH 045/186] remove async, #20562 (#20645) --- .../src/main/scala/akka/remote/artery/ArteryTransport.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 701d26bf00..fb6e7aa5dc 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -362,7 +362,6 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private def runInboundControlStream(): Unit = { val (c, completed) = Source.fromGraph(new AeronSource(inboundChannel, controlStreamId, aeron, taskRunner, envelopePool)) - .async // FIXME measure .viaMat(inboundControlFlow)(Keep.right) .toMat(Sink.ignore)(Keep.both) .run()(materializer) @@ -400,7 +399,6 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private def runInboundOrdinaryMessagesStream(): Unit = { val completed = Source.fromGraph(new AeronSource(inboundChannel, ordinaryStreamId, aeron, taskRunner, envelopePool)) - .async // FIXME measure .via(inboundFlow) .runWith(Sink.ignore)(materializer) @@ -409,7 +407,6 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private def runInboundLargeMessagesStream(): Unit = { val completed = Source.fromGraph(new AeronSource(inboundChannel, largeStreamId, aeron, taskRunner, largeEnvelopePool)) - .async // FIXME measure .via(inboundLargeFlow) .runWith(Sink.ignore)(materializer) From fc6a3356100a8e9b25c4613132df42ef21499589 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 1 Jun 2016 11:55:25 +0200 Subject: [PATCH 046/186] improve performance of literal encoding (#20655) * improve performance of literal encoding * use char array instead of getBytes * JMH benchmark * use Unsafe to get the char array --- .../artery/LiteralEncodingBenchmark.scala | 107 ++++++++++++++++++ .../scala/akka/remote/artery/BufferPool.scala | 54 +++++++-- 2 files changed, 154 insertions(+), 7 deletions(-) create mode 100644 akka-bench-jmh/src/main/scala/akka/remote/artery/LiteralEncodingBenchmark.scala 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..105dee92a3 --- /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 = "artery://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-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index 858309b10e..6830202df4 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -10,6 +10,7 @@ import java.nio.{ ByteBuffer, ByteOrder } import org.agrona.concurrent.{ ManyToManyConcurrentArrayQueue, UnsafeBuffer } import sun.misc.Cleaner +import akka.util.Unsafe import scala.util.control.NonFatal @@ -60,6 +61,9 @@ private[remote] object EnvelopeBuffer { val UsAscii = Charset.forName("US-ASCII") val DeadLettersCode = 0 + + // accessing the internal char array of String when writing literal strings to ByteBuffer + val StringValueFieldOffset = Unsafe.instance.objectFieldOffset(classOf[String].getDeclaredField("value")) } /** @@ -182,6 +186,9 @@ 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) + private val cleanerField: Field = try { val cleaner = byteBuffer.getClass.getDeclaredField("cleaner") cleaner.setAccessible(true) @@ -274,19 +281,52 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { private def readLiteral(): String = { val length = byteBuffer.getShort - val bytes = Array.ofDim[Byte](length) - byteBuffer.get(bytes) - new String(bytes, UsAscii) + 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 = { - if (literal.length > 65535) + val length = literal.length + if (length > 65535) throw new IllegalArgumentException("Literals longer than 65535 cannot be encoded in the envelope") - val literalBytes = literal.getBytes(UsAscii) byteBuffer.putInt(tagOffset, byteBuffer.position()) - byteBuffer.putShort(literalBytes.length.toShort) - byteBuffer.put(literalBytes) + + if (length == 0) { + byteBuffer.putShort(0) + } else { + byteBuffer.putShort(literal.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) + } } } From 8fb7727526f4e5dedd342b53d403d21150f16d16 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 1 Jun 2016 11:56:18 +0200 Subject: [PATCH 047/186] make it possible to use external Aeron media driver, #20588 (#20653) * make it possible to use external Aeron media driver, #20588 * on my machine the MaxThroughputSpec maxed out all 8 cores completely, and when using external media driver it is much better and easier to find the actual bottlenecks * aeron.properties for external media driver --- .../remote/artery/MaxThroughputSpec.scala | 6 ++++ akka-remote/src/main/resources/reference.conf | 13 +++++++ .../scala/akka/remote/RemoteSettings.scala | 3 ++ .../akka/remote/artery/ArteryTransport.scala | 35 ++++++++++++------- .../src/test/resources/aeron.properties | 16 +++++++++ 5 files changed, 61 insertions(+), 12 deletions(-) create mode 100644 akka-remote/src/test/resources/aeron.properties 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 index 45282e4d6f..ade8827976 100644 --- 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 @@ -36,6 +36,12 @@ object MaxThroughputSpec extends MultiNodeConfig { } 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" } } """))) diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index ea692dab5c..09c50044ef 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -105,6 +105,19 @@ akka { # stream, to pass such messages through the large message stream the selections # but must be resolved to ActorRefs first. large-message-destinations = [] + + advanced { + # 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 = "" + } } ### General settings diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index 346671a457..a92f44424b 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -26,6 +26,9 @@ final class RemoteSettings(val config: Config) { case "" ⇒ InetAddress.getLocalHost.getHostName case other ⇒ other } + val EmbeddedMediaDriver = getBoolean("akka.remote.artery.advanced.embedded-media-driver") + val AeronDirectoryName = getString("akka.remote.artery.advanced.aeron-dir") requiring (dir ⇒ + EmbeddedMediaDriver || dir.nonEmpty, "aeron-dir must be defined when using external media driver") val LogReceive: Boolean = getBoolean("akka.remote.log-received-messages") diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index fb6e7aa5dc..91d0651f1c 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -222,7 +222,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R @volatile private[this] var materializer: Materializer = _ @volatile private[this] var controlSubject: ControlMessageSubject = _ @volatile private[this] var messageDispatcher: MessageDispatcher = _ - @volatile private[this] var driver: MediaDriver = _ + @volatile private[this] var mediaDriver: Option[MediaDriver] = None @volatile private[this] var aeron: Aeron = _ @volatile private[this] var aeronErrorLogTask: Cancellable = _ @@ -298,13 +298,23 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def startMediaDriver(): Unit = { - // TODO also support external media driver - val driverContext = new MediaDriver.Context - // FIXME settings from config - driverContext.clientLivenessTimeoutNs(SECONDS.toNanos(20)) - driverContext.imageLivenessTimeoutNs(SECONDS.toNanos(20)) - driverContext.driverTimeoutMs(SECONDS.toNanos(20)) - driver = MediaDriver.launchEmbedded(driverContext) + if (remoteSettings.EmbeddedMediaDriver) { + val driverContext = new MediaDriver.Context + if (remoteSettings.AeronDirectoryName.nonEmpty) + driverContext.aeronDirectoryName(remoteSettings.AeronDirectoryName) + // FIXME settings from config + driverContext.clientLivenessTimeoutNs(SECONDS.toNanos(20)) + driverContext.imageLivenessTimeoutNs(SECONDS.toNanos(20)) + driverContext.driverTimeoutMs(SECONDS.toNanos(20)) + val driver = MediaDriver.launchEmbedded(driverContext) + log.debug("Started embedded media driver in directory [{}]", driver.aeronDirectoryName) + mediaDriver = Some(driver) + } + } + + private def aeronDir: String = mediaDriver match { + case Some(driver) ⇒ driver.aeronDirectoryName + case None ⇒ remoteSettings.AeronDirectoryName } private def startAeron(): Unit = { @@ -336,12 +346,12 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } }) - ctx.aeronDirectoryName(driver.aeronDirectoryName) + ctx.aeronDirectoryName(aeronDir) aeron = Aeron.connect(ctx) } private def startAeronErrorLog(): Unit = { - val errorLog = new AeronErrorLog(new File(driver.aeronDirectoryName, CncFileDescriptor.CNC_FILE)) + val errorLog = new AeronErrorLog(new File(aeronDir, CncFileDescriptor.CNC_FILE)) val lastTimestamp = new AtomicLong(0L) import system.dispatcher // FIXME perhaps use another dispatcher for this aeronErrorLogTask = system.scheduler.schedule(3.seconds, 5.seconds) { @@ -436,9 +446,10 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R if (taskRunner != null) taskRunner.stop() if (aeronErrorLogTask != null) aeronErrorLogTask.cancel() if (aeron != null) aeron.close() - if (driver != null) { + mediaDriver.foreach { driver ⇒ + // this is only for embedded media driver driver.close() - // FIXME only delete files for embedded media driver, and it should also be configurable + // FIXME it should also be configurable to not delete dir IoUtil.delete(new File(driver.aeronDirectoryName), true) } Future.successful(Done) diff --git a/akka-remote/src/test/resources/aeron.properties b/akka-remote/src/test/resources/aeron.properties new file mode 100644 index 0000000000..db195e1075 --- /dev/null +++ b/akka-remote/src/test/resources/aeron.properties @@ -0,0 +1,16 @@ +# 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 + +# use same director in akka.remote.artery.advanced.aeron-dir config +# of the Akka application +aeron.dir=target/aeron From e3afe6107dd342fa091960f8994b64a31ae0690f Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Sun, 29 May 2016 19:41:09 +0200 Subject: [PATCH 048/186] configuration of Artery materializer and dispatcher * also increased the parallelism-max to 4 for default-remote-dispatcher --- .../scala/akka/remote/artery/MaxThroughputSpec.scala | 2 +- akka-remote/src/main/resources/reference.conf | 6 ++++++ .../main/scala/akka/remote/artery/ArteryTransport.scala | 8 +++++--- 3 files changed, 12 insertions(+), 4 deletions(-) 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 index ade8827976..5e395187ed 100644 --- 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 @@ -54,7 +54,7 @@ object MaxThroughputSpec extends MultiNodeConfig { final case class FlowControl(burstStartTime: Long) extends Echo def receiverProps(reporter: RateReporter, payloadSize: Int): Props = - Props(new Receiver(reporter, payloadSize)) + Props(new Receiver(reporter, payloadSize)).withDispatcher("akka.remote.default-remote-dispatcher") class Receiver(reporter: RateReporter, payloadSize: Int) extends Actor { var c = 0L diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 09c50044ef..59da3085ef 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -107,6 +107,12 @@ akka { large-message-destinations = [] advanced { + # Settings for the materializer that is used for the remote streams. + materializer = ${akka.stream.materializer} + materializer { + dispatcher = "akka.remote.default-remote-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'. diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 91d0651f1c..f846360d6d 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -5,7 +5,6 @@ package akka.remote.artery import java.io.File import java.nio.ByteOrder - import scala.concurrent.Future import scala.concurrent.Promise import scala.concurrent.duration._ @@ -62,13 +61,12 @@ import org.agrona.IoUtil import java.io.File import java.net.InetSocketAddress import java.nio.channels.DatagramChannel - import akka.remote.artery.OutboundControlJunction.OutboundControlIngress import io.aeron.CncFileDescriptor import java.util.concurrent.atomic.AtomicLong import akka.actor.Cancellable - import scala.collection.JavaConverters._ +import akka.stream.ActorMaterializerSettings /** * INTERNAL API */ @@ -288,6 +286,10 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R _localAddress = UniqueAddress( Address("artery", system.name, remoteSettings.ArteryHostname, port), AddressUidExtension(system).longAddressUid) + + val materializerSettings = ActorMaterializerSettings( + remoteSettings.config.getConfig("akka.remote.artery.advanced.materializer")) + materializer = ActorMaterializer(materializerSettings)(system) materializer = ActorMaterializer()(system) messageDispatcher = new MessageDispatcher(system, provider) From aab46199fd49b2d0a7ed05d65e9ec481fe26b9be Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 2 Jun 2016 07:21:47 +0200 Subject: [PATCH 049/186] port of some remote multi-node tests --- .../remote/AttemptSysMsgRedeliverySpec.scala | 36 ++++-- .../akka/remote/LookupRemoteActorSpec.scala | 34 +++-- .../akka/remote/NewRemoteActorSpec.scala | 46 ++++--- .../PiercingShouldKeepQuarantineSpec.scala | 31 +++-- .../RemoteDeploymentDeathWatchSpec.scala | 62 +++++---- .../remote/RemoteQuarantinePiercingSpec.scala | 47 ++++--- .../PiercingShouldKeepQuarantineSpec.scala | 84 ------------- .../artery/RemoteQuarantinePiercingSpec.scala | 119 ------------------ 8 files changed, 166 insertions(+), 293 deletions(-) delete mode 100644 akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/PiercingShouldKeepQuarantineSpec.scala delete mode 100644 akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteQuarantinePiercingSpec.scala 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..f0a5327991 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,39 @@ 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 + """))) 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)) + +// FIXME this test is failing for Artery, a DeathWatchNotification is not delivered as expected? +//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 +53,11 @@ object AttemptSysMsgRedeliveryMultiJvmSpec extends MultiNodeConfig { } } -class AttemptSysMsgRedeliveryMultiJvmNode1 extends AttemptSysMsgRedeliverySpec -class AttemptSysMsgRedeliveryMultiJvmNode2 extends AttemptSysMsgRedeliverySpec -class AttemptSysMsgRedeliveryMultiJvmNode3 extends AttemptSysMsgRedeliverySpec - -class AttemptSysMsgRedeliverySpec extends MultiNodeSpec(AttemptSysMsgRedeliveryMultiJvmSpec) +abstract class AttemptSysMsgRedeliverySpec(multiNodeConfig: AttemptSysMsgRedeliveryMultiJvmSpec) + extends MultiNodeSpec(multiNodeConfig) with STMultiNodeSpec with ImplicitSender with DefaultTimeout { - import AttemptSysMsgRedeliveryMultiJvmSpec._ + 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..67635e1b17 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 + """))) 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) +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 MultiNodeSpec(multiNodeConfig) with STMultiNodeSpec with ImplicitSender with DefaultTimeout { - import LookupRemoteActorMultiJvmSpec._ + 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..91a959dacd 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 + """))) val master = role("master") val slave = role("slave") @@ -43,12 +34,31 @@ 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) +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 MultiNodeSpec(multiNodeConfig) with STMultiNodeSpec with ImplicitSender with DefaultTimeout { - import NewRemoteActorMultiJvmSpec._ + 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..2166bdac0d 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,41 @@ 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 - """))) + akka.remote.artery.enabled = $artery + """))) +} + +class PiercingShouldKeepQuarantineSpecMultiJvmNode1 extends PiercingShouldKeepQuarantineSpec( + new PiercingShouldKeepQuarantineConfig(artery = false)) +class PiercingShouldKeepQuarantineSpecMultiJvmNode2 extends PiercingShouldKeepQuarantineSpec( + new PiercingShouldKeepQuarantineConfig(artery = false)) + +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).addressUid } } - } -class PiercingShouldKeepQuarantineSpecMultiJvmNode1 extends PiercingShouldKeepQuarantineSpec -class PiercingShouldKeepQuarantineSpecMultiJvmNode2 extends PiercingShouldKeepQuarantineSpec - -abstract class PiercingShouldKeepQuarantineSpec extends MultiNodeSpec(PiercingShouldKeepQuarantineSpec) +abstract class PiercingShouldKeepQuarantineSpec(multiNodeConfig: PiercingShouldKeepQuarantineConfig) + extends MultiNodeSpec(multiNodeConfig) with STMultiNodeSpec with ImplicitSender { - + import multiNodeConfig._ import PiercingShouldKeepQuarantineSpec._ override def initialParticipants = roles.size 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..649e340b6e 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,62 @@ 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 """))) 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) +abstract class RemoteDeploymentDeathWatchSpec(multiNodeConfig: RemoteDeploymentDeathWatchMultiJvmSpec) + extends MultiNodeSpec(multiNodeConfig) with STMultiNodeSpec with ImplicitSender { - - import RemoteDeploymentDeathWatchMultiJvmSpec._ + 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/RemoteQuarantinePiercingSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteQuarantinePiercingSpec.scala index 84dbf87593..2c624471bf 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,52 @@ 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 - """))) + akka.remote.artery.enabled = $artery + """))) +} + +class RemoteQuarantinePiercingMultiJvmNode1 extends RemoteQuarantinePiercingSpec( + new RemoteQuarantinePiercingConfig(artery = false)) +class RemoteQuarantinePiercingMultiJvmNode2 extends RemoteQuarantinePiercingSpec( + new RemoteQuarantinePiercingConfig(artery = false)) + +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).addressUid -> self) } } - } -class RemoteQuarantinePiercingMultiJvmNode1 extends RemoteQuarantinePiercingSpec -class RemoteQuarantinePiercingMultiJvmNode2 extends RemoteQuarantinePiercingSpec - -abstract class RemoteQuarantinePiercingSpec extends MultiNodeSpec(RemoteQuarantinePiercingSpec) +abstract class RemoteQuarantinePiercingSpec(multiNodeConfig: RemoteQuarantinePiercingConfig) + extends MultiNodeSpec(multiNodeConfig) with STMultiNodeSpec with ImplicitSender { - + 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): (Int, ActorRef) = { + within(timeout) { + system.actorSelection(node(role) / "user" / actorName) ! "identify" + expectMsgType[(Int, ActorRef)] + } } "RemoteNodeShutdownAndComesBack" must { @@ -61,7 +74,7 @@ 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 @@ -101,11 +114,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.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/artery/PiercingShouldKeepQuarantineSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/PiercingShouldKeepQuarantineSpec.scala deleted file mode 100644 index cb17da59d2..0000000000 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/PiercingShouldKeepQuarantineSpec.scala +++ /dev/null @@ -1,84 +0,0 @@ -/** - * Copyright (C) 2009-2016 Lightbend Inc. - */ -package akka.remote.artery - -import scala.concurrent.duration._ -import com.typesafe.config.ConfigFactory -import akka.actor._ -import akka.testkit._ -import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec } -import akka.remote.testconductor.RoleName -import akka.remote.AddressUidExtension -import akka.remote.RARP - -object PiercingShouldKeepQuarantineSpec 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 - akka.remote.retry-gate-closed-for = 0.5s - - akka.remote.artery.enabled = on - """))) - - class Subject extends Actor { - def receive = { - case "getuid" ⇒ sender() ! AddressUidExtension(context.system).addressUid - } - } - -} - -class PiercingShouldKeepQuarantineSpecMultiJvmNode1 extends PiercingShouldKeepQuarantineSpec -class PiercingShouldKeepQuarantineSpecMultiJvmNode2 extends PiercingShouldKeepQuarantineSpec - -abstract class PiercingShouldKeepQuarantineSpec extends MultiNodeSpec(PiercingShouldKeepQuarantineSpec) - with STMultiNodeSpec - with ImplicitSender { - - import PiercingShouldKeepQuarantineSpec._ - - override def initialParticipants = roles.size - - "While probing through the quarantine remoting" must { - - "not lose existing quarantine marker" taggedAs LongRunningTest in { - runOn(first) { - enterBarrier("actors-started") - - // Communicate with second system - system.actorSelection(node(second) / "user" / "subject") ! "getuid" - val uid = expectMsgType[Int](10.seconds) - enterBarrier("actor-identified") - - // Manually Quarantine the other system - RARP(system).provider.transport.quarantine(node(second).address, Some(uid)) - - // Quarantining is not immediate - Thread.sleep(1000) - - // Quarantine is up -- Should not be able to communicate with remote system any more - for (_ ← 1 to 4) { - system.actorSelection(node(second) / "user" / "subject") ! "getuid" - expectNoMsg(2.seconds) - } - - enterBarrier("quarantine-intact") - - } - - runOn(second) { - system.actorOf(Props[Subject], "subject") - enterBarrier("actors-started") - enterBarrier("actor-identified") - enterBarrier("quarantine-intact") - } - - } - - } -} diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteQuarantinePiercingSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteQuarantinePiercingSpec.scala deleted file mode 100644 index 299857cfc5..0000000000 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteQuarantinePiercingSpec.scala +++ /dev/null @@ -1,119 +0,0 @@ -/** - * Copyright (C) 2009-2016 Lightbend Inc. - */ -package akka.remote.artery - -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.{ ForceDisassociate, Direction } -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 -import akka.remote.RARP - -object RemoteQuarantinePiercingSpec 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 - akka.remote.artery.enabled = on - """))) - - class Subject extends Actor { - def receive = { - case "shutdown" ⇒ context.system.terminate() - case "identify" ⇒ sender() ! (AddressUidExtension(context.system).addressUid -> self) - } - } - -} - -class RemoteQuarantinePiercingSpecMultiJvmNode1 extends RemoteQuarantinePiercingSpec -class RemoteQuarantinePiercingSpecMultiJvmNode2 extends RemoteQuarantinePiercingSpec - -abstract class RemoteQuarantinePiercingSpec extends MultiNodeSpec(RemoteQuarantinePiercingSpec) - with STMultiNodeSpec - with ImplicitSender { - - import RemoteQuarantinePiercingSpec._ - - override def initialParticipants = roles.size - - def identifyWithUid(role: RoleName, actorName: String, timeout: FiniteDuration = remainingOrDefault): (Int, ActorRef) = { - within(timeout) { - system.actorSelection(node(role) / "user" / actorName) ! "identify" - expectMsgType[(Int, ActorRef)] - } - } - - "RemoteNodeShutdownAndComesBack" must { - - "allow piercing through the quarantine when remote UID is new" taggedAs LongRunningTest in { - runOn(first) { - val secondAddress = node(second).address - enterBarrier("actors-started") - - // Acquire ActorRef from first system - 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)) - - // Quarantine is up -- Cannot communicate with remote system any more - system.actorSelection(RootActorPath(secondAddress) / "user" / "subject") ! "identify" - expectNoMsg(2.seconds) - - // Shut down the other system -- which results in restart (see runOn(second)) - Await.result(testConductor.shutdown(second), 30.seconds) - - // Now wait until second system becomes alive again - within(30.seconds) { - // 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) - uidSecond should not be (uidFirst) - subjectSecond should not be (subjectFirst) - } - } - - // If we got here the Quarantine was successfully pierced since it is configured to last 1 day - - system.actorSelection(RootActorPath(secondAddress) / "user" / "subject") ! "shutdown" - - } - - runOn(second) { - val addr = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress - system.actorOf(Props[Subject], "subject") - enterBarrier("actors-started") - - enterBarrier("actor-identified") - - Await.ready(system.whenTerminated, 30.seconds) - - val freshSystem = ActorSystem(system.name, ConfigFactory.parseString(s""" - akka.remote.artery.port = ${addr.port.get} - """).withFallback(system.settings.config)) - freshSystem.actorOf(Props[Subject], "subject") - - Await.ready(freshSystem.whenTerminated, 30.seconds) - } - - } - - } -} From 37af15b05046214c8bfba3520157380a093acbc7 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 2 Jun 2016 13:32:08 +0200 Subject: [PATCH 050/186] tryCleanDirectByteBuffer in Artery BufferPool --- .../scala/akka/remote/artery/BufferPool.scala | 40 +++++++++++-------- 1 file changed, 24 insertions(+), 16 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index 6830202df4..450099c99f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -37,7 +37,7 @@ private[remote] class EnvelopeBufferPool(maximumPayload: Int, maximumBuffers: In } } - def release(buffer: EnvelopeBuffer) = if (!availableBuffers.offer(buffer)) buffer.tryForceDrop() + def release(buffer: EnvelopeBuffer) = if (!availableBuffers.offer(buffer)) buffer.tryCleanDirectByteBuffer() } @@ -189,21 +189,6 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { private var literalChars = Array.ofDim[Char](64) private var literalBytes = Array.ofDim[Byte](64) - private val cleanerField: Field = try { - val cleaner = byteBuffer.getClass.getDeclaredField("cleaner") - cleaner.setAccessible(true) - cleaner - } catch { - case NonFatal(_) ⇒ null - } - - def tryForceDrop(): Unit = { - if (cleanerField ne null) cleanerField.get(byteBuffer) match { - case cleaner: Cleaner ⇒ cleaner.clean() - case _ ⇒ - } - } - def writeHeader(h: HeaderBuilder): Unit = { val header = h.asInstanceOf[HeaderBuilderImpl] byteBuffer.clear() @@ -329,4 +314,27 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { } } + /** + * DirectByteBuffers are garbage collected by using a phantom reference and a + * reference queue. Every once a while, the JVM checks the reference queue and + * cleans the DirectByteBuffers. However, as this doesn't happen + * immediately after discarding all references to a DirectByteBuffer, it's + * easy to OutOfMemoryError yourself using DirectByteBuffers. This function + * explicitly calls the Cleaner method of a DirectByteBuffer. + * + * Utilizes reflection to avoid dependency to `sun.misc.Cleaner`. + */ + def tryCleanDirectByteBuffer(): Unit = try { + if (byteBuffer.isDirect) { + val cleanerMethod = byteBuffer.getClass().getMethod("cleaner") + cleanerMethod.setAccessible(true) + val cleaner = cleanerMethod.invoke(byteBuffer) + val cleanMethod = cleaner.getClass().getMethod("clean") + cleanMethod.setAccessible(true) + cleanMethod.invoke(cleaner) + } + } catch { + case NonFatal(_) ⇒ // attempt failed, ok + } + } From 70a72d58bf5ec42c75ad20d82fbb9e59045231c2 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 27 May 2016 15:14:13 +0200 Subject: [PATCH 051/186] adjust sample for Artery --- .../src/main/resources/application.conf | 15 +++++++-------- .../cluster/factorial/FactorialBackend.scala | 2 +- .../sample/cluster/simple/SimpleClusterApp.scala | 2 +- .../scala/sample/cluster/stats/StatsSample.scala | 2 +- .../cluster/stats/StatsSampleOneMaster.scala | 2 +- .../transformation/TransformationBackend.scala | 2 +- .../transformation/TransformationFrontend.scala | 2 +- 7 files changed, 13 insertions(+), 14 deletions(-) diff --git a/akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf b/akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf index 676a223350..58195b14b4 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf +++ b/akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf @@ -3,18 +3,16 @@ akka { actor { provider = "akka.cluster.ClusterActorRefProvider" } - remote { - log-remote-lifecycle-events = off - netty.tcp { - hostname = "127.0.0.1" - port = 0 - } + remote.artery { + enabled = on + hostname = 127.0.0.1 + port = 0 } cluster { seed-nodes = [ - "akka.tcp://ClusterSystem@127.0.0.1:2551", - "akka.tcp://ClusterSystem@127.0.0.1:2552"] + "artery://ClusterSystem@127.0.0.1:2551", + "artery://ClusterSystem@127.0.0.1:2552"] #//#snippet # excluded from snippet @@ -37,3 +35,4 @@ akka.extensions=["akka.cluster.metrics.ClusterMetricsExtension"] # Note: use per-jvm-instance folder when running multiple jvm on one host. akka.cluster.metrics.native-library-extract-folder=${user.dir}/target/native #//#snippet + diff --git a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/FactorialBackend.scala b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/FactorialBackend.scala index 11ee8c2074..273b2eaa58 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/FactorialBackend.scala +++ b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/FactorialBackend.scala @@ -34,7 +34,7 @@ object FactorialBackend { def main(args: Array[String]): Unit = { // Override the configuration of the port when specified as program argument val port = if (args.isEmpty) "0" else args(0) - val config = ConfigFactory.parseString(s"akka.remote.netty.tcp.port=$port"). + val config = ConfigFactory.parseString(s"akka.remote.artery.port=$port"). withFallback(ConfigFactory.parseString("akka.cluster.roles = [backend]")). withFallback(ConfigFactory.load("factorial")) diff --git a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/simple/SimpleClusterApp.scala b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/simple/SimpleClusterApp.scala index 1e87f49d25..343e56d86f 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/simple/SimpleClusterApp.scala +++ b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/simple/SimpleClusterApp.scala @@ -15,7 +15,7 @@ object SimpleClusterApp { def startup(ports: Seq[String]): Unit = { ports foreach { port => // Override the configuration of the port - val config = ConfigFactory.parseString("akka.remote.netty.tcp.port=" + port). + val config = ConfigFactory.parseString("akka.remote.artery.port=" + port). withFallback(ConfigFactory.load()) // Create an Akka system diff --git a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSample.scala b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSample.scala index b32dd961a8..99741f1403 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSample.scala +++ b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSample.scala @@ -28,7 +28,7 @@ object StatsSample { ports foreach { port => // Override the configuration of the port when specified as program argument val config = - ConfigFactory.parseString(s"akka.remote.netty.tcp.port=" + port).withFallback( + ConfigFactory.parseString(s"akka.remote.artery.port=" + port).withFallback( ConfigFactory.parseString("akka.cluster.roles = [compute]")). withFallback(ConfigFactory.load("stats1")) diff --git a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSampleOneMaster.scala b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSampleOneMaster.scala index f4db5403be..3962f968db 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSampleOneMaster.scala +++ b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSampleOneMaster.scala @@ -23,7 +23,7 @@ object StatsSampleOneMaster { ports foreach { port => // Override the configuration of the port when specified as program argument val config = - ConfigFactory.parseString(s"akka.remote.netty.tcp.port=" + port).withFallback( + ConfigFactory.parseString(s"akka.remote.artery.port=" + port).withFallback( ConfigFactory.parseString("akka.cluster.roles = [compute]")). withFallback(ConfigFactory.load("stats2")) diff --git a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationBackend.scala b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationBackend.scala index 2d164b4bb0..6d8cbbcd59 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationBackend.scala +++ b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationBackend.scala @@ -42,7 +42,7 @@ object TransformationBackend { def main(args: Array[String]): Unit = { // Override the configuration of the port when specified as program argument val port = if (args.isEmpty) "0" else args(0) - val config = ConfigFactory.parseString(s"akka.remote.netty.tcp.port=$port"). + val config = ConfigFactory.parseString(s"akka.remote.artery.port=$port"). withFallback(ConfigFactory.parseString("akka.cluster.roles = [backend]")). withFallback(ConfigFactory.load()) diff --git a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationFrontend.scala b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationFrontend.scala index deb34d5158..e6679cd7c6 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationFrontend.scala +++ b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationFrontend.scala @@ -40,7 +40,7 @@ object TransformationFrontend { def main(args: Array[String]): Unit = { // Override the configuration of the port when specified as program argument val port = if (args.isEmpty) "0" else args(0) - val config = ConfigFactory.parseString(s"akka.remote.netty.tcp.port=$port"). + val config = ConfigFactory.parseString(s"akka.remote.artery.port=$port"). withFallback(ConfigFactory.parseString("akka.cluster.roles = [frontend]")). withFallback(ConfigFactory.load()) From 60111c4886bed37a7da90a5852c6569a9d775d96 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 27 May 2016 16:45:48 +0200 Subject: [PATCH 052/186] detect wrong protocol * used wrong protocol by mistake and got weird errors and it was not obvious that the reason was wrong protocol, e.g. it created association to itself * and also set the cachedAssociation --- .../akka/remote/RemoteActorRefProvider.scala | 37 ++++++++--- .../akka/remote/artery/ArteryTransport.scala | 17 +++-- .../test/scala/akka/remote/RemotingSpec.scala | 2 +- .../artery/RemoteActorRefProviderSpec.scala | 63 +++++++++++++++++++ 4 files changed, 105 insertions(+), 14 deletions(-) create mode 100644 akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 987f819b88..490b2c85a8 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -342,10 +342,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 @@ -355,9 +362,14 @@ private[akka] class RemoteActorRefProvider( path match { case ActorPathExtractor(address, elems) ⇒ if (hasAddress(address)) local.resolveActorRef(rootGuardian, elems) - else + 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 @@ -374,7 +386,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) } } @@ -390,7 +402,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) } } @@ -459,10 +471,19 @@ private[akka] class RemoteActorRef private[akka] ( deploy: Option[Deploy]) extends InternalActorRef with RemoteRef { - @volatile var cachedAssociation: artery.Association = null + 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 a separate stream for large messages - @volatile var cachedLargeMessageDestinationFlag: LargeMessageDestinationFlag = null + @volatile private[remote] var cachedLargeMessageDestinationFlag: LargeMessageDestinationFlag = null def getChild(name: Iterator[String]): InternalActorRef = { val s = name.toStream diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index f846360d6d..62bf1e9ecf 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -216,7 +216,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // these vars are initialized once in the start method @volatile private[this] var _localAddress: UniqueAddress = _ - override def localAddress: UniqueAddress = _localAddress + @volatile private[this] var _addresses: Set[Address] = _ @volatile private[this] var materializer: Materializer = _ @volatile private[this] var controlSubject: ControlMessageSubject = _ @volatile private[this] var messageDispatcher: MessageDispatcher = _ @@ -224,8 +224,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R @volatile private[this] var aeron: Aeron = _ @volatile private[this] var aeronErrorLogTask: Cancellable = _ + override def localAddress: UniqueAddress = _localAddress override def defaultAddress: Address = localAddress.address - override def addresses: Set[Address] = Set(defaultAddress) + override def addresses: Set[Address] = _addresses override def localAddressForRemote(remote: Address): Address = defaultAddress override val log: LoggingAdapter = Logging(system, getClass.getName) private val eventPublisher = new EventPublisher(system, log, remoteSettings.RemoteLifecycleEventsLogLevel) @@ -284,8 +285,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // TODO: Configure materializer properly // TODO: Have a supervisor actor _localAddress = UniqueAddress( - Address("artery", system.name, remoteSettings.ArteryHostname, port), + Address(ArteryTransport.ProtocolName, system.name, remoteSettings.ArteryHostname, port), AddressUidExtension(system).longAddressUid) + _addresses = Set(_localAddress.address) val materializerSettings = ActorMaterializerSettings( remoteSettings.config.getConfig("akka.remote.artery.advanced.materializer")) @@ -465,11 +467,14 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R override def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = { val cached = recipient.cachedAssociation - val remoteAddress = recipient.path.address val a = if (cached ne null) cached - else association(remoteAddress) + else { + val a2 = association(recipient.path.address) + recipient.cachedAssociation = a2 + a2 + } a.send(message, senderOption, recipient) } @@ -574,6 +579,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R */ private[remote] object ArteryTransport { + val ProtocolName = "artery" + val Version = 0 val MaximumFrameSize = 1024 * 1024 val MaximumPooledBuffers = 256 diff --git a/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala b/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala index dbd188d377..ea7435cff5 100644 --- a/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala @@ -560,7 +560,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) 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..3d0614dc79 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala @@ -0,0 +1,63 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.duration._ +import akka.actor.{ ActorIdentity, ActorSystem, ExtendedActorSystem, Identify, RootActorPath } +import akka.testkit.{ AkkaSpec, ImplicitSender } +import akka.testkit.TestActors +import com.typesafe.config.ConfigFactory +import akka.testkit.EventFilter +import akka.actor.InternalActorRef +import akka.remote.RemoteActorRef +import akka.actor.EmptyLocalActorRef + +object RemoteActorRefProviderSpec { + + val config = ConfigFactory.parseString(s""" + akka { + actor.provider = "akka.remote.RemoteActorRefProvider" + remote.artery.enabled = on + remote.artery.hostname = localhost + remote.artery.port = 0 + } + """) + +} + +class RemoteActorRefProviderSpec extends AkkaSpec(RemoteActorRefProviderSpec.config) with ImplicitSender { + import RemoteActorRefProviderSpec._ + + val addressA = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + system.actorOf(TestActors.echoActorProps, "echo") + + val systemB = ActorSystem("systemB", system.settings.config) + val addressB = systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + systemB.actorOf(TestActors.echoActorProps, "echo") + + override def afterTermination(): Unit = shutdown(systemB) + + "RemoteActorRefProvider" must { + + "resolve local actor selection" in { + val sel = system.actorSelection(s"artery://${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"artery://${systemB.name}@${addressB.host.get}:${addressB.port.get}/user/echo") + sel.anchor.getClass should ===(classOf[RemoteActorRef]) + sel.anchor.asInstanceOf[InternalActorRef].isLocal should be(false) + } + + "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]) + } + } + + } + +} From 3465a221f0ab799e5ad068c9b1234f47cd8092e3 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 3 Jun 2016 11:59:00 +0200 Subject: [PATCH 053/186] format with new Scalariform version * and fix mima issue --- .../akka/serialization/Serialization.scala | 6 ++--- .../akka/remote/artery/CodecBenchmark.scala | 9 +++++--- .../scala/akka/cluster/QuickRestartSpec.scala | 6 +++-- .../akka/remote/testkit/MultiNodeSpec.scala | 10 ++++---- .../artery/AeronStreamLatencySpec.scala | 4 ++-- .../artery/AeronStreamMaxThroughputSpec.scala | 4 ++-- .../artery/HandshakeRestartReceiverSpec.scala | 2 +- .../akka/remote/artery/LatencySpec.scala | 4 ++-- .../remote/artery/MaxThroughputSpec.scala | 11 +++++---- .../scala/akka/remote/artery/PlotResult.scala | 2 +- .../RemoteRestartedQuarantinedSpec.scala | 2 +- .../akka/remote/artery/TestRateReporter.scala | 3 ++- .../akka/remote/artery/ArteryTransport.scala | 15 ++++++------ .../akka/remote/artery/Association.scala | 23 +++++++++++-------- .../scala/akka/remote/artery/Codecs.scala | 17 +++++++------- .../scala/akka/remote/artery/Handshake.scala | 14 ++++++----- .../artery/InboundQuarantineCheck.scala | 3 ++- .../remote/artery/MessageDispatcher.scala | 22 ++++++++++-------- .../remote/artery/SystemMessageDelivery.scala | 4 ++-- .../remote/artery/EnvelopeBufferSpec.scala | 14 +++++------ .../remote/artery/OutboundHandshakeSpec.scala | 2 +- .../artery/SerializationErrorSpec.scala | 4 ++-- .../akka/remote/artery/TestContext.scala | 16 ++++++------- project/MiMa.scala | 8 ++++--- 24 files changed, 114 insertions(+), 91 deletions(-) diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index 8762e1d940..673fee3706 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -139,8 +139,8 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { else { val cache = manifestCache.get cache.get(manifest) match { - case Some(cachedClassManifest) => s1.fromBinary(bytes, cachedClassManifest) - case None => + case Some(cachedClassManifest) ⇒ s1.fromBinary(bytes, cachedClassManifest) + case None ⇒ system.dynamicAccess.getClassFor[AnyRef](manifest) match { case Success(classManifest) ⇒ val classManifestOption: Option[Class[_]] = Some(classManifest) @@ -167,7 +167,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { "akka.actor.serializers is not in synch between the two systems.") } serializer match { - case ser: ByteBufferSerializer => + case ser: ByteBufferSerializer ⇒ ser.fromBinary(buf, manifest) case _ ⇒ val bytes = Array.ofDim[Byte](buf.remaining()) 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 index 03d73b1fde..8c09e80f39 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -46,7 +46,8 @@ class CodecBenchmark { remote.artery.hostname = localhost remote.artery.port = 0 } - """) + """ + ) implicit val system = ActorSystem("CodecBenchmark", config) val systemB = ActorSystem("systemB", system.settings.config) @@ -56,8 +57,10 @@ class CodecBenchmark { val headerIn = HeaderBuilder(compression) val envelopeTemplateBuffer = ByteBuffer.allocate(ArteryTransport.MaximumFrameSize).order(ByteOrder.LITTLE_ENDIAN) - val uniqueLocalAddress = UniqueAddress(system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress, - AddressUidExtension(system).addressUid) + val uniqueLocalAddress = UniqueAddress( + system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress, + AddressUidExtension(system).addressUid + ) val payload = Array.ofDim[Byte](1000) private var materializer: ActorMaterializer = _ 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 0a4e68143a..713c02160a 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/QuickRestartSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/QuickRestartSpec.scala @@ -65,11 +65,13 @@ abstract class QuickRestartSpec runOn(second) { restartingSystem = if (restartingSystem == null) - ActorSystem(system.name, + ActorSystem( + system.name, ConfigFactory.parseString(s"akka.cluster.roles = [round-$n]") .withFallback(system.settings.config)) else - ActorSystem(system.name, + ActorSystem( + system.name, ConfigFactory.parseString(s""" akka.cluster.roles = [round-$n] akka.remote.netty.tcp.port = ${Cluster(restartingSystem).selfAddress.port.get}""") // same port 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 ef76581368..bd0446bce2 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 @@ -195,11 +195,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.remote.artery.hostname" -> selfName, - "akka.remote.netty.tcp.hostname" -> selfName, - "akka.remote.netty.tcp.port" -> selfPort, - "akka.remote.artery.port" -> selfPort)) + "akka.actor.provider" → "akka.remote.RemoteActorRefProvider", + "akka.remote.artery.hostname" → selfName, + "akka.remote.netty.tcp.hostname" → selfName, + "akka.remote.netty.tcp.port" → selfPort, + "akka.remote.artery.port" → selfPort)) private[testkit] val baseConfig: Config = ConfigFactory.parseString(""" akka { 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 index 27159795b1..8dfd030aef 100644 --- 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 @@ -57,10 +57,10 @@ object AeronStreamLatencySpec extends MultiNodeConfig { """))) final case class TestSettings( - testName: String, + testName: String, messageRate: Int, // msg/s payloadSize: Int, - repeat: Int) + repeat: Int) } 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 index 110ce66554..e8460d8b4e 100644 --- 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 @@ -50,9 +50,9 @@ object AeronStreamMaxThroughputSpec extends MultiNodeConfig { """))) final case class TestSettings( - testName: String, + testName: String, totalMessages: Long, - payloadSize: Int) + payloadSize: Int) def iterate(start: Long, end: Long): Iterator[Long] = new AbstractIterator[Long] { private[this] var first = true 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 index 3c934011d3..7084c138ce 100644 --- 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 @@ -34,7 +34,7 @@ object HandshakeRestartReceiverSpec extends MultiNodeConfig { class Subject extends Actor { def receive = { case "shutdown" ⇒ context.system.terminate() - case "identify" ⇒ sender() ! (AddressUidExtension(context.system).addressUid -> self) + case "identify" ⇒ sender() ! (AddressUidExtension(context.system).addressUid → self) } } 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 index 1e2a2bb39b..fa0ad212ad 100644 --- 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 @@ -122,10 +122,10 @@ object LatencySpec extends MultiNodeConfig { } final case class TestSettings( - testName: String, + testName: String, messageRate: Int, // msg/s payloadSize: Int, - repeat: Int) + repeat: Int) } 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 index 5e395187ed..77e0039614 100644 --- 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 @@ -147,10 +147,10 @@ object MaxThroughputSpec extends MultiNodeConfig { } final case class TestSettings( - testName: String, - totalMessages: Long, - burstSize: Int, - payloadSize: Int, + testName: String, + totalMessages: Long, + burstSize: Int, + payloadSize: Int, senderReceiverPairs: Int) } @@ -240,7 +240,8 @@ abstract class MaxThroughputSpec 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), + val snd = system.actorOf( + senderProps(receiver, testSettings, plotProbe.ref), testName + "-snd" + n) val terminationProbe = TestProbe() terminationProbe.watch(snd) 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 index 08858e62f1..01033b679c 100644 --- 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 @@ -6,7 +6,7 @@ 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)) + copy(values = values :+ (key → value)) def addAll(p: PlotResult): PlotResult = copy(values ++ p.values) 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 index f2402fae57..980aa51908 100644 --- 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 @@ -48,7 +48,7 @@ object RemoteRestartedQuarantinedSpec extends MultiNodeConfig { class Subject extends Actor { def receive = { case "shutdown" ⇒ context.system.terminate() - case "identify" ⇒ sender() ! (AddressUidExtension(context.system).addressUid -> self) + case "identify" ⇒ sender() ! (AddressUidExtension(context.system).addressUid → self) } } 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 index 2cef0d0ec6..6446aa7404 100644 --- 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 @@ -6,7 +6,8 @@ package akka.remote.artery import java.util.concurrent.TimeUnit.SECONDS import java.util.concurrent.Executors -class TestRateReporter(name: String) extends RateReporter(SECONDS.toNanos(1), +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 + diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 62bf1e9ecf..a5d5338395 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -71,11 +71,11 @@ import akka.stream.ActorMaterializerSettings * INTERNAL API */ private[akka] final case class InboundEnvelope( - recipient: InternalActorRef, + recipient: InternalActorRef, recipientAddress: Address, - message: AnyRef, - senderOption: Option[ActorRef], - originUid: Long) + message: AnyRef, + senderOption: Option[ActorRef], + originUid: Long) /** * INTERNAL API @@ -123,9 +123,9 @@ private[akka] object AssociationState { * INTERNAL API */ private[akka] final class AssociationState( - val incarnation: Int, + val incarnation: Int, val uniqueRemoteAddressPromise: Promise[UniqueAddress], - val quarantined: Set[Long]) { + val quarantined: Set[Long]) { /** * Full outbound address with UID for this association. @@ -239,7 +239,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val systemMessageResendInterval: FiniteDuration = 1.second private val handshakeRetryInterval: FiniteDuration = 1.second private val handshakeTimeout: FiniteDuration = - system.settings.config.getMillisDuration("akka.remote.handshake-timeout").requiring(_ > Duration.Zero, + system.settings.config.getMillisDuration("akka.remote.handshake-timeout").requiring( + _ > Duration.Zero, "handshake-timeout must be > 0") private val injectHandshakeInterval: FiniteDuration = 1.second private val giveUpSendAfter: FiniteDuration = 60.seconds diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 8e9db7092d..2064c67cdd 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -44,11 +44,11 @@ import akka.util.{ Unsafe, WildcardTree } * remote address. */ private[akka] class Association( - val transport: ArteryTransport, - val materializer: Materializer, - override val remoteAddress: Address, + val transport: ArteryTransport, + val materializer: Materializer, + override val remoteAddress: Address, override val controlSubject: ControlMessageSubject, - largeMessageDestinations: WildcardTree[NotUsed]) + largeMessageDestinations: WildcardTree[NotUsed]) extends AbstractAssociation with OutboundContext { private val log = Logging(transport.system, getClass.getName) @@ -103,7 +103,8 @@ private[akka] class Association( Unsafe.instance.getObjectVolatile(this, AbstractAssociation.sharedStateOffset).asInstanceOf[AssociationState] def completeHandshake(peer: UniqueAddress): Unit = { - require(remoteAddress == peer.address, + require( + remoteAddress == peer.address, s"wrong remote address in completeHandshake, got ${peer.address}, expected ${remoteAddress}") val current = associationState current.uniqueRemoteAddressPromise.trySuccess(peer) @@ -114,7 +115,8 @@ private[akka] class Association( if (swapState(current, newState)) { current.uniqueRemoteAddressValue() match { case Some(Success(old)) ⇒ - log.debug("Incarnation {} of association to [{}] with new UID [{}] (old UID [{}])", + log.debug( + "Incarnation {} of association to [{}] with new UID [{}] (old UID [{}])", newState.incarnation, peer.address, peer.uid, old.uid) case _ ⇒ // Failed, nothing to do @@ -190,7 +192,8 @@ private[akka] class Association( val newState = current.newQuarantined() if (swapState(current, newState)) { // quarantine state change was performed - log.warning("Association to [{}] with UID [{}] is irrecoverably failed. Quarantining address. {}", + log.warning( + "Association to [{}] with UID [{}] is irrecoverably failed. Quarantining address. {}", remoteAddress, u, reason) // end delivery of system messages to that incarnation after this point send(ClearSystemMessageDelivery, None, dummyRecipient) @@ -200,10 +203,12 @@ private[akka] class Association( quarantine(reason, uid) // recursive } case Some(Success(peer)) ⇒ - log.debug("Quarantine of [{}] ignored due to non-matching UID, quarantine requested for [{}] but current is [{}]. {}", + 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. {}", + log.debug( + "Quarantine of [{}] ignored because handshake not completed, quarantine request was for old incarnation. {}", remoteAddress, reason) } case None ⇒ diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 1457373146..eb6470e73b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -15,9 +15,9 @@ import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler } // TODO: Long UID class Encoder( uniqueLocalAddress: UniqueAddress, - system: ActorSystem, - compressionTable: LiteralCompressionTable, - pool: EnvelopeBufferPool) + system: ActorSystem, + compressionTable: LiteralCompressionTable, + pool: EnvelopeBufferPool) extends GraphStage[FlowShape[Send, EnvelopeBuffer]] { val in: Inlet[Send] = Inlet("Artery.Encoder.in") @@ -108,11 +108,11 @@ class Encoder( } class Decoder( - uniqueLocalAddress: UniqueAddress, - system: ExtendedActorSystem, + uniqueLocalAddress: UniqueAddress, + system: ExtendedActorSystem, resolveActorRefWithLocalAddress: String ⇒ InternalActorRef, - compressionTable: LiteralCompressionTable, - pool: EnvelopeBufferPool) extends GraphStage[FlowShape[EnvelopeBuffer, InboundEnvelope]] { + compressionTable: LiteralCompressionTable, + pool: EnvelopeBufferPool) extends GraphStage[FlowShape[EnvelopeBuffer, InboundEnvelope]] { val in: Inlet[EnvelopeBuffer] = Inlet("Artery.Decoder.in") val out: Outlet[InboundEnvelope] = Outlet("Artery.Decoder.out") val shape: FlowShape[EnvelopeBuffer, InboundEnvelope] = FlowShape(in, out) @@ -172,7 +172,8 @@ class Decoder( push(out, decoded) } catch { case NonFatal(e) ⇒ - log.warning("Failed to deserialize message with serializer id [{}] and manifest [{}]. {}", + log.warning( + "Failed to deserialize message with serializer id [{}] and manifest [{}]. {}", headerBuilder.serializer, headerBuilder.manifest, e.getMessage) pull(in) } finally { diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala index 8be3029670..d1f96ce91b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -198,14 +198,16 @@ private[akka] class InboundHandshake(inboundContext: InboundContext, inControlSt push(out, env) else { // FIXME remove, only debug - log.warning(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, inboundContext.localAddress.uid, env.originUid) - if (log.isDebugEnabled) - log.debug(s"Dropping message [{}] from unknown system with UID [{}]. " + + log.warning( + 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, inboundContext.localAddress.uid, env.originUid) + 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, inboundContext.localAddress.uid, env.originUid) pull(in) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala b/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala index 2cdb2e3542..531e9c4aff 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala @@ -33,7 +33,8 @@ private[akka] class InboundQuarantineCheck(inboundContext: InboundContext) exten push(out, env) case association ⇒ if (association.associationState.isQuarantined(env.originUid)) { - inboundContext.sendControl(association.remoteAddress, + inboundContext.sendControl( + association.remoteAddress, Quarantined(inboundContext.localAddress, UniqueAddress(association.remoteAddress, env.originUid))) pull(in) } else diff --git a/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala b/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala index a76a4b478b..8e871c3f33 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala @@ -21,16 +21,17 @@ import akka.remote.RemoteRef * INTERNAL API */ private[akka] class MessageDispatcher( - system: ExtendedActorSystem, + system: ExtendedActorSystem, provider: RemoteActorRefProvider) { private val remoteDaemon = provider.remoteDaemon private val log = Logging(system, getClass.getName) - def dispatch(recipient: InternalActorRef, - recipientAddress: Address, - message: AnyRef, - senderOption: Option[ActorRef]): Unit = { + def dispatch( + recipient: InternalActorRef, + recipientAddress: Address, + message: AnyRef, + senderOption: Option[ActorRef]): Unit = { import provider.remoteSettings._ @@ -54,8 +55,9 @@ private[akka] class MessageDispatcher( case sel: ActorSelectionMessage ⇒ if (UntrustedMode && (!TrustedSelectionPaths.contains(sel.elements.mkString("/", "/", "")) || sel.msg.isInstanceOf[PossiblyHarmful] || l != provider.rootGuardian)) - log.debug("operating in UntrustedMode, dropping inbound actor selection to [{}], " + - "allow it by adding the path to 'akka.remote.trusted-selection-paths' configuration", + 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 @@ -72,10 +74,12 @@ private[akka] class MessageDispatcher( // if it was originally addressed to us but is in fact remote from our point of view (i.e. remote-deployed) r.!(message)(sender) else - log.error("dropping message [{}] for non-local recipient [{}] arriving at [{}] inbound addresses are [{}]", + log.error( + "dropping message [{}] for non-local recipient [{}] arriving at [{}] inbound addresses are [{}]", message.getClass, r, recipientAddress, provider.transport.addresses.mkString(", ")) - case r ⇒ log.error("dropping message [{}] for unknown recipient [{}] arriving at [{}] inbound addresses are [{}]", + case r ⇒ log.error( + "dropping message [{}] for unknown recipient [{}] arriving at [{}] inbound addresses are [{}]", message.getClass, r, recipientAddress, provider.transport.addresses.mkString(", ")) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala index 472ce89fe7..4f1766821e 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -44,8 +44,8 @@ private[akka] object SystemMessageDelivery { */ private[akka] class SystemMessageDelivery( outboundContext: OutboundContext, - resendInterval: FiniteDuration, - maxBufferSize: Int) + resendInterval: FiniteDuration, + maxBufferSize: Int) extends GraphStage[FlowShape[Send, Send]] { import SystemMessageDelivery._ diff --git a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala index 67b5ce8a3a..0184920d37 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala @@ -9,19 +9,19 @@ class EnvelopeBufferSpec extends AkkaSpec { object TestCompressor extends LiteralCompressionTable { val refToIdx = Map( - "compressable0" -> 0, - "compressable1" -> 1, - "reallylongcompressablestring" -> 2) + "compressable0" → 0, + "compressable1" → 1, + "reallylongcompressablestring" → 2) val idxToRef = refToIdx.map(_.swap) val serializerToIdx = Map( - "serializer0" -> 0, - "serializer1" -> 1) + "serializer0" → 0, + "serializer1" → 1) val idxToSer = serializerToIdx.map(_.swap) val manifestToIdx = Map( - "manifest0" -> 0, - "manifest1" -> 1) + "manifest0" → 0, + "manifest1" → 1) val idxToManifest = manifestToIdx.map(_.swap) override def compressActorRef(ref: String): Int = refToIdx.getOrElse(ref, -1) diff --git a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala index df5f2d8189..27bbd3e13b 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala @@ -33,7 +33,7 @@ class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { private def setupStream( outboundContext: OutboundContext, timeout: FiniteDuration = 5.seconds, - retryInterval: FiniteDuration = 10.seconds, + retryInterval: FiniteDuration = 10.seconds, injectHandshakeInterval: FiniteDuration = 10.seconds): (TestPublisher.Probe[String], TestSubscriber.Probe[Any]) = { val destination = null.asInstanceOf[RemoteActorRef] // not used diff --git a/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala index 6b104257fe..5c3364da39 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala @@ -75,8 +75,8 @@ class SerializationErrorSpec extends AkkaSpec(SerializationErrorSpec.config) wit EventFilter.warning( start = "Failed to deserialize message with serializer id [4]", occurrences = 1).intercept { - remoteRef ! "boom".getBytes("utf-8") - }(systemB) + remoteRef ! "boom".getBytes("utf-8") + }(systemB) remoteRef ! "ping2" expectMsg("ping2") diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala index 561150f8fc..bbc9cefaa2 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -21,9 +21,9 @@ import akka.remote.artery.InboundControlJunction.ControlMessageSubject private[akka] class TestInboundContext( override val localAddress: UniqueAddress, - val controlSubject: TestControlMessageSubject = new TestControlMessageSubject, - val controlProbe: Option[ActorRef] = None, - val replyDropRate: Double = 0.0) extends InboundContext { + 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]() @@ -58,10 +58,10 @@ private[akka] class TestInboundContext( } private[akka] class TestOutboundContext( - override val localAddress: UniqueAddress, - override val remoteAddress: Address, + override val localAddress: UniqueAddress, + override val remoteAddress: Address, override val controlSubject: TestControlMessageSubject, - val controlProbe: Option[ActorRef] = None) extends OutboundContext { + val controlProbe: Option[ActorRef] = None) extends OutboundContext { // access to this is synchronized (it's a test utility) private var _associationState = AssociationState() @@ -117,8 +117,8 @@ private[akka] class TestControlMessageSubject extends ControlMessageSubject { } private[akka] class ManualReplyInboundContext( - replyProbe: ActorRef, - localAddress: UniqueAddress, + replyProbe: ActorRef, + localAddress: UniqueAddress, controlSubject: TestControlMessageSubject) extends TestInboundContext(localAddress, controlSubject) { private var lastReply: Option[(Address, ControlMessage)] = None diff --git a/project/MiMa.scala b/project/MiMa.scala index 0d2cd9941b..0a96b84a24 100644 --- a/project/MiMa.scala +++ b/project/MiMa.scala @@ -747,9 +747,6 @@ object MiMa extends AutoPlugin { ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.http.scaladsl.DefaultSSLContextCreation.validateAndWarnAboutLooseSettings") ), "2.4.4" -> Seq( - // 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"), // #20080, #20081 remove race condition on HTTP client ProblemFilters.exclude[DirectMissingMethodProblem]("akka.http.scaladsl.Http#HostConnectionPool.gatewayFuture"), @@ -851,6 +848,11 @@ object MiMa extends AutoPlugin { ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.cluster.client.ClusterClient.initialContactsSel") ), "2.4.6" -> Seq( + + // 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"), + // internal api FilterAnyProblemStartingWith("akka.stream.impl"), From 38acf442e6584c32660060e919f58a0db05834f5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Endre=20S=C3=A1ndor=20Varga?= Date: Thu, 26 May 2016 13:46:50 +0200 Subject: [PATCH 054/186] Flight recorder, initial data structure sketch --- .../akka/artery/FlightRecorderBench.scala | 62 +++++ .../akka/remote/artery/FlightRecorder.scala | 262 ++++++++++++++++++ .../RollingEventLogSimulationSpec.scala | 215 ++++++++++++++ 3 files changed, 539 insertions(+) create mode 100644 akka-bench-jmh/src/main/scala/akka/artery/FlightRecorderBench.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/RollingEventLogSimulationSpec.scala diff --git a/akka-bench-jmh/src/main/scala/akka/artery/FlightRecorderBench.scala b/akka-bench-jmh/src/main/scala/akka/artery/FlightRecorderBench.scala new file mode 100644 index 0000000000..c09df54030 --- /dev/null +++ b/akka-bench-jmh/src/main/scala/akka/artery/FlightRecorderBench.scala @@ -0,0 +1,62 @@ +package akka.artery + +import java.io.File +import java.nio.channels.FileChannel +import java.nio.file.StandardOpenOption +import java.util.concurrent.{CountDownLatch, TimeUnit} + +import akka.remote.artery.FlightRecorder +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 + + var file: File = _ + var fileChannel: FileChannel = _ + 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_wrtites(): 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-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..f060d1417c --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala @@ -0,0 +1,262 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import java.nio.{ ByteBuffer, ByteOrder } +import java.nio.channels.FileChannel +import java.util.concurrent.atomic.AtomicBoolean + +import akka.util.ByteString +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 +} + +/** + * INTERNAL API + * + * Update clock at various resolutions and aquire the resulting timestamp. + */ +private[remote] trait EventClock { + + def updateWallClock(): Unit + def updateHighSpeedClock(): Unit + + def getWallClockPart: Long + def getHighSpeedPart: Long + +} + +/** + * INTERNAL API + * + * This class is not thread-safe + */ +private[remote] class EventClockImpl extends EventClock { + + private[this] var wallClock: Long = System.currentTimeMillis() + private[this] var highSpeedClock: Long = System.nanoTime() + + override def updateWallClock(): Unit = { + wallClock = System.currentTimeMillis() + highSpeedClock = System.nanoTime() + } + + override def updateHighSpeedClock(): Unit = { + // TODO: Update wall clock once in a while + highSpeedClock = System.nanoTime() + } + + override def getWallClockPart: Long = wallClock + override def getHighSpeedPart: Long = highSpeedClock +} + +/** + * INTERNAL API + */ +private[remote] object SnapshottableRollingEventLog { + val HeadPointerOffset = 0L + val EntriesOffset = 8L + val LogOffset = 0L + + val Committed = 0 + val Dirty = 1 + val CommitEntrySize = 4 +} + +/** + * INTERNAL API + */ +private[remote] class SnapshottableRollingEventLog( + fileChannel: FileChannel, + offset: Long, + entryCount: Long, + logBufferSize: Long, + recordSize: Int) extends AtomicBoolean { + import SnapshottableRollingEventLog._ + + // FIXME: check if power of two + private[this] val LogMask: Long = entryCount - 1L + + private[this] val buffers: Array[MappedResizeableBuffer] = Array.tabulate(FlightRecorder.SnapshotCount) { snapshotId ⇒ + val buffer = new MappedResizeableBuffer(fileChannel, offset + snapshotId * logBufferSize, logBufferSize) + buffer.setMemory(0, logBufferSize.toInt, 0.toByte) + buffer + } + + /* + * 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(log: Int, recordBuffer: ByteBuffer): Unit = { + val logBuffer = buffers(log) + + @tailrec def writeRecord(): Unit = { + // Advance the head + val slotOffset = EntriesOffset + ((logBuffer.getAndAddLong(HeadPointerOffset, 1L) & LogMask) * recordSize) + val payloadOffset = slotOffset + 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(slotOffset, Committed, Dirty)) { + logBuffer.putBytes(payloadOffset, recordBuffer, recordSize) + //println(logBuffer.getLong(recordOffset + 4)) + + // Now this is free to be overwritten + logBuffer.putIntVolatile(slotOffset, Committed) + } else writeRecord() // Try to claim a new slot + } + + writeRecord() + } + + def close(): Unit = buffers.foreach(_.close()) +} + +/** + * INTERNAL API + */ +private[remote] object FlightRecorder { + val LogHeaderSize = 8 + val SnapshotCount = 4 + val SnapshotMask = SnapshotCount - 1 + + // TODO: Dummy values right now, format is under construction + val AlertRecordSize = 128 + val LoFreqRecordSize = 128 + val HiFreqBatchSize = 63 + val HiFreqRecordSize = 16 * (HiFreqBatchSize + 1) // (batched events + header) + + val AlertWindow = 256 + val LoFreqWindow = 256 + val HiFreqWindow = 256 // This is counted in batches ! + + val Alignment = 64 * 1024 // Windows is picky about mapped section alignments + + 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 = HiFreqLogSize * SnapshotCount + val HiFreqSectionSize = LoFreqLogSize * SnapshotCount +} + +/** + * INTERNAL API + */ +private[akka] class FlightRecorder(fileChannel: FileChannel) extends AtomicBoolean { + import FlightRecorder._ + + // FIXME: check if power of two + private[this] val SnapshotMask = SnapshotCount - 1 + private[this] val alertLogs = + new SnapshottableRollingEventLog( + fileChannel = fileChannel, + offset = AlertSectionSize + HiFreqSectionSize, + entryCount = AlertWindow, + logBufferSize = AlertLogSize, + recordSize = AlertRecordSize) + private[this] val loFreqLogs = + new SnapshottableRollingEventLog( + fileChannel = fileChannel, + offset = AlertSectionSize, + entryCount = LoFreqWindow, + logBufferSize = LoFreqLogSize, + recordSize = LoFreqRecordSize) + private[this] val hiFreqLogs = + new SnapshottableRollingEventLog( + fileChannel = fileChannel, + offset = 0, + entryCount = HiFreqWindow, + logBufferSize = HiFreqLogSize, + recordSize = HiFreqRecordSize) + // No need for volatile, guarded by atomic CAS and set + private var currentLog = 0 + + def takeSnapshot(): 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. ¯\_(ツ)_/¯ ") + if (!get && compareAndSet(false, true)) { + // Roll over to the next one + currentLog = (currentLog + 1) & SnapshotMask + set(false) + // At this point it is NOT GUARANTEED that all writers have finished writing to the currently snapshotted + // buffer! + } + } + + def close(): Unit = { + alertLogs.close() + hiFreqLogs.close() + loFreqLogs.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) + + override def alert(code: Int, metadata: Array[Byte]): Unit = { + clock.updateWallClock() + prepareRichRecord(alertRecordBuffer, code, metadata) + alertLogs.write(currentLog, alertRecordBuffer) + // TODO: Flush HiFreq batch + + // TODO: collect a few more events and *then* take a snapshot + takeSnapshot() + } + + override def loFreq(code: Int, metadata: Array[Byte]): Unit = { + clock.updateHighSpeedClock() + prepareRichRecord(loFreqRecordBuffer, code, metadata) + loFreqLogs.write(currentLog, loFreqRecordBuffer) + } + + private def prepareRichRecord(recordBuffer: ByteBuffer, code: Int, metadata: Array[Byte]): Unit = { + recordBuffer.clear() + // FIXME: This is a bit overkill, needs some smarter scheme later, no need to always store the wallclock + recordBuffer.putLong(clock.getWallClockPart) + recordBuffer.putLong(clock.getHighSpeedPart) + recordBuffer.putInt(code) + recordBuffer.put(metadata.length.toByte) + recordBuffer.put(metadata) + // Don't flip here! We always write fixed size records + recordBuffer.position(0) + } + + // FIXME: Try to save as many bytes here as possible! We will see crazy throughput here + override def hiFreq(code: Long, param: Long): Unit = { + hiFreqBatchBuffer.putLong(code) + hiFreqBatchBuffer.putLong(param) + + // If batch is full, time to flush + if (!hiFreqBatchBuffer.hasRemaining) { + hiFreqBatchBuffer.position(0) + hiFreqLogs.write(currentLog, hiFreqBatchBuffer) + 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.getWallClockPart) + hiFreqBatchBuffer.putLong(clock.getHighSpeedPart) + // Mow ready to write some more events... + } + } + + } +} 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() + } + + } + +} From 18a15f33f31ac8e0d8c09d7baf30fc94494513c3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Endre=20S=C3=A1ndor=20Varga?= Date: Wed, 1 Jun 2016 15:02:49 +0200 Subject: [PATCH 055/186] Added reading support for testing --- .../akka/remote/artery/FlightRecorder.scala | 168 +++++--- .../remote/artery/FlightRecorderReader.scala | 211 ++++++++++ .../remote/artery/FlightRecorderSpec.scala | 388 ++++++++++++++++++ 3 files changed, 720 insertions(+), 47 deletions(-) create mode 100644 akka-remote/src/test/scala/akka/remote/artery/FlightRecorderReader.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala diff --git a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala index f060d1417c..890824452d 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala @@ -3,6 +3,7 @@ */ package akka.remote.artery +import java.io.RandomAccessFile import java.nio.{ ByteBuffer, ByteOrder } import java.nio.channels.FileChannel import java.util.concurrent.atomic.AtomicBoolean @@ -20,6 +21,8 @@ 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 } /** @@ -44,17 +47,21 @@ private[remote] trait EventClock { */ private[remote] class EventClockImpl extends EventClock { - private[this] var wallClock: Long = System.currentTimeMillis() - private[this] var highSpeedClock: Long = System.nanoTime() + 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() - highSpeedClock = System.nanoTime() + highSpeedClockOffset = System.nanoTime() + highSpeedClock = 0 } override def updateHighSpeedClock(): Unit = { // TODO: Update wall clock once in a while - highSpeedClock = System.nanoTime() + highSpeedClock = System.nanoTime() - highSpeedClockOffset } override def getWallClockPart: Long = wallClock @@ -64,63 +71,78 @@ private[remote] class EventClockImpl extends EventClock { /** * INTERNAL API */ -private[remote] object SnapshottableRollingEventLog { +private[remote] object RollingEventLogSection { val HeadPointerOffset = 0L - val EntriesOffset = 8L + 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 SnapshottableRollingEventLog( +private[remote] class RollingEventLogSection( fileChannel: FileChannel, offset: Long, entryCount: Long, logBufferSize: Long, recordSize: Int) extends AtomicBoolean { - import SnapshottableRollingEventLog._ + import RollingEventLogSection._ // FIXME: check if power of two private[this] val LogMask: Long = entryCount - 1L - private[this] val buffers: Array[MappedResizeableBuffer] = Array.tabulate(FlightRecorder.SnapshotCount) { snapshotId ⇒ - val buffer = new MappedResizeableBuffer(fileChannel, offset + snapshotId * logBufferSize, logBufferSize) + 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(log: Int, recordBuffer: ByteBuffer): Unit = { - val logBuffer = buffers(log) + def write(logId: Int, recordBuffer: ByteBuffer): Unit = { + val logBuffer = buffers(logId) @tailrec def writeRecord(): Unit = { // Advance the head - val slotOffset = EntriesOffset + ((logBuffer.getAndAddLong(HeadPointerOffset, 1L) & LogMask) * recordSize) - val payloadOffset = slotOffset + CommitEntrySize + 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(slotOffset, Committed, Dirty)) { + 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(slotOffset, Committed) + 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()) } @@ -128,70 +150,105 @@ private[remote] class SnapshottableRollingEventLog( * INTERNAL API */ private[remote] object FlightRecorder { - val LogHeaderSize = 8 + 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 = 63 - val HiFreqRecordSize = 16 * (HiFreqBatchSize + 1) // (batched events + header) + 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 Alignment = 64 * 1024 // Windows is picky about mapped section alignments - 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 = HiFreqLogSize * SnapshotCount - val HiFreqSectionSize = LoFreqLogSize * 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[akka] class FlightRecorder(fileChannel: FileChannel) extends AtomicBoolean { +private[akka] class FlightRecorder(val fileChannel: FileChannel) extends AtomicBoolean { import FlightRecorder._ + private[this] val globalSection = new MappedResizeableBuffer(fileChannel, 0, GlobalSectionSize) + // FIXME: check if power of two private[this] val SnapshotMask = SnapshotCount - 1 private[this] val alertLogs = - new SnapshottableRollingEventLog( + new RollingEventLogSection( fileChannel = fileChannel, - offset = AlertSectionSize + HiFreqSectionSize, + offset = AlertSectionOffset, entryCount = AlertWindow, logBufferSize = AlertLogSize, recordSize = AlertRecordSize) private[this] val loFreqLogs = - new SnapshottableRollingEventLog( + new RollingEventLogSection( fileChannel = fileChannel, - offset = AlertSectionSize, + offset = LoFreqSectionOffset, entryCount = LoFreqWindow, logBufferSize = LoFreqLogSize, recordSize = LoFreqRecordSize) private[this] val hiFreqLogs = - new SnapshottableRollingEventLog( + new RollingEventLogSection( fileChannel = fileChannel, - offset = 0, + offset = HiFreqSectionOffset, entryCount = HiFreqWindow, logBufferSize = HiFreqLogSize, recordSize = HiFreqRecordSize) // No need for volatile, guarded by atomic CAS and set - private var currentLog = 0 + @volatile private var currentLog = 0 - def takeSnapshot(): Unit = { + 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. ¯\_(ツ)_/¯ ") if (!get && compareAndSet(false, true)) { - // Roll over to the next one - currentLog = (currentLog + 1) & SnapshotMask + 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) set(false) // At this point it is NOT GUARANTEED that all writers have finished writing to the currently snapshotted // buffer! @@ -202,6 +259,7 @@ private[akka] class FlightRecorder(fileChannel: FileChannel) extends AtomicBoole alertLogs.close() hiFreqLogs.close() loFreqLogs.close() + globalSection.close() } def createEventSink(): EventSink = new EventSink { @@ -209,15 +267,16 @@ private[akka] class FlightRecorder(fileChannel: FileChannel) extends AtomicBoole 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 = { clock.updateWallClock() prepareRichRecord(alertRecordBuffer, code, metadata) alertLogs.write(currentLog, alertRecordBuffer) - // TODO: Flush HiFreq batch - - // TODO: collect a few more events and *then* take a snapshot - takeSnapshot() + flushHiFreqBatch() + snapshot() } override def loFreq(code: Int, metadata: Array[Byte]): Unit = { @@ -240,21 +299,36 @@ private[akka] class FlightRecorder(fileChannel: FileChannel) extends AtomicBoole // FIXME: Try to save as many bytes here as possible! We will see crazy throughput here override def hiFreq(code: Long, param: Long): Unit = { + hiFreqBatchedEntries += 1 hiFreqBatchBuffer.putLong(code) hiFreqBatchBuffer.putLong(param) // If batch is full, time to flush - if (!hiFreqBatchBuffer.hasRemaining) { + 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.getWallClockPart) + hiFreqBatchBuffer.putLong(clock.getHighSpeedPart) + // 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 = { + if (hiFreqBatchedEntries > 0) { + hiFreqBatchBuffer.putLong(HiFreqEntryCountFieldOffset, hiFreqBatchedEntries) + hiFreqBatchedEntries = 0 hiFreqBatchBuffer.position(0) hiFreqLogs.write(currentLog, hiFreqBatchBuffer) - 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.getWallClockPart) - hiFreqBatchBuffer.putLong(clock.getHighSpeedPart) - // Mow ready to write some more events... + startHiFreqBatch() } } diff --git a/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderReader.scala b/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderReader.scala new file mode 100644 index 0000000000..fd5cbb805d --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderReader.scala @@ -0,0 +1,211 @@ +package akka.remote.artery + +import java.io.IOException +import java.nio.channels.FileChannel +import java.time.Instant + +import org.agrona.concurrent.MappedResizeableBuffer + +import scala.collection.immutable + +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) + +} + +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 + } + } + } + } + + case class RichEntry(timeStamp: Instant, dirty: Boolean, code: Long, metadata: Array[Byte]) { + override def toString: String = s"[$timeStamp] ${if (dirty) "#" else ""} \t $code | ${metadata.mkString(",")}" + } + + case class CompactEntry(timeStamp: Instant, dirty: Boolean, code: Long, param: Long) { + override def toString: String = s"[$timeStamp] ${if (dirty) "#" else ""} \t $code | $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/test/scala/akka/remote/artery/FlightRecorderSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala new file mode 100644 index 0000000000..8c4f4def2e --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala @@ -0,0 +1,388 @@ +package akka.remote.artery + +import java.io.{ File, IOException, RandomAccessFile } +import java.nio.channels.FileChannel +import java.nio.file.StandardOpenOption +import java.time.Instant +import java.util.Arrays +import java.util.concurrent.{ CountDownLatch, TimeUnit } + +import akka.testkit.AkkaSpec + +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 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)) + } + } + + } + + 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() } + } + } + +} From d236b8e15234e92607fe5ac244225b331025961c Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Sun, 29 May 2016 22:15:48 +0200 Subject: [PATCH 056/186] new queue Source for remote sends * new SendQueue Source based on agrona ManyToOneConcurrentArrayQueue * jmh benchmark for send queue * JMH benchmark for Source.queue, Source.actorRef and the new SendQueue * inject the queue so that we can start sending to it before materialization * Get rid of computeIfAbsent in the AssociationRegistry by making it possible to send (enque) messages to the Association instance immediatly after construction. --- .../akka/remote/artery/CodecBenchmark.scala | 12 +- .../scala/akka/remote/artery/LatchSink.scala | 27 ++++ .../remote/artery/SendQueueBenchmark.scala | 138 ++++++++++++++++ .../akka/remote/artery/ArteryTransport.scala | 2 +- .../akka/remote/artery/Association.scala | 140 ++++++++++------ .../scala/akka/remote/artery/SendQueue.scala | 131 +++++++++++++++ .../akka/remote/artery/SendQueueSpec.scala | 152 ++++++++++++++++++ 7 files changed, 552 insertions(+), 50 deletions(-) create mode 100644 akka-bench-jmh/src/main/scala/akka/remote/artery/SendQueueBenchmark.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/SendQueueSpec.scala 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 index 8c09e80f39..82f22e1c89 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -113,7 +113,8 @@ class CodecBenchmark { Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) .runWith(new LatchSink(N, latch))(materializer) - latch.await(30, TimeUnit.SECONDS) + if (!latch.await(30, TimeUnit.SECONDS)) + throw new RuntimeException("Latch didn't complete in time") } @Benchmark @@ -131,7 +132,8 @@ class CodecBenchmark { .map(envelope => envelopePool.release(envelope)) .runWith(new LatchSink(N, latch))(materializer) - latch.await(30, TimeUnit.SECONDS) + if (!latch.await(30, TimeUnit.SECONDS)) + throw new RuntimeException("Latch didn't complete in time") } @Benchmark @@ -164,7 +166,8 @@ class CodecBenchmark { .via(decoder) .runWith(new LatchSink(N, latch))(materializer) - latch.await(30, TimeUnit.SECONDS) + if (!latch.await(30, TimeUnit.SECONDS)) + throw new RuntimeException("Latch didn't complete in time") } @Benchmark @@ -195,7 +198,8 @@ class CodecBenchmark { .via(decoder) .runWith(new LatchSink(N, latch))(materializer) - latch.await(30, TimeUnit.SECONDS) + 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/LatchSink.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/LatchSink.scala index c60bead1ce..d66a6814d0 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/LatchSink.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/LatchSink.scala @@ -4,6 +4,7 @@ package akka.remote.artery import java.util.concurrent.CountDownLatch +import java.util.concurrent.CyclicBarrier import akka.stream.Attributes import akka.stream.Inlet @@ -34,3 +35,29 @@ class LatchSink(countDownAfter: Int, latch: CountDownLatch) extends GraphStage[S 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/SendQueueBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/SendQueueBenchmark.scala new file mode 100644 index 0000000000..f54ca42f0e --- /dev/null +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/SendQueueBenchmark.scala @@ -0,0 +1,138 @@ +/** + * 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-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index a5d5338395..db4322573d 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -464,7 +464,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // InboundContext override def sendControl(to: Address, message: ControlMessage) = - association(to).outboundControlIngress.sendControlMessage(message) + association(to).sendControl(message) override def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = { val cached = recipient.cachedAssociation diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 2064c67cdd..38b1980a62 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -3,18 +3,18 @@ */ package akka.remote.artery +import java.util.Queue + import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.CountDownLatch import java.util.concurrent.TimeUnit -import java.util.function.{ Function ⇒ JFunction } - +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 scala.util.Success - import akka.{ Done, NotUsed } import akka.actor.ActorRef import akka.actor.ActorSelectionMessage @@ -31,11 +31,19 @@ import akka.remote.artery.OutboundHandshake.HandshakeTimeoutException import akka.remote.artery.SystemMessageDelivery.ClearSystemMessageDelivery import akka.stream.AbruptTerminationException import akka.stream.Materializer -import akka.stream.OverflowStrategy import akka.stream.scaladsl.Keep import akka.stream.scaladsl.Source -import akka.stream.scaladsl.SourceQueueWithComplete import akka.util.{ Unsafe, WildcardTree } +import org.agrona.concurrent.ManyToOneConcurrentArrayQueue + +/** + * INTERNAL API + */ +private[remote] object Association { + final case class QueueWrapper(queue: Queue[Send]) extends SendQueue.ProducerApi[Send] { + override def offer(message: Send): Boolean = queue.offer(message) + } +} /** * INTERNAL API @@ -43,25 +51,37 @@ import akka.util.{ Unsafe, WildcardTree } * Thread-safe, mutable holder for association state. Main entry point for remote destined message to a specific * remote address. */ -private[akka] class Association( +private[remote] class Association( val transport: ArteryTransport, val materializer: Materializer, override val remoteAddress: Address, override val controlSubject: ControlMessageSubject, largeMessageDestinations: WildcardTree[NotUsed]) extends AbstractAssociation with OutboundContext { + import Association._ private val log = Logging(transport.system, getClass.getName) private val controlQueueSize = transport.provider.remoteSettings.SysMsgBufferSize + // FIXME config queue size, and it should perhaps also be possible to use some kind of LinkedQueue + // such as agrona.ManyToOneConcurrentLinkedQueue or AbstractNodeQueue for less memory consumption + private val queueSize = 3072 + private val largeQueueSize = 256 private val restartTimeout: FiniteDuration = 5.seconds // FIXME config private val maxRestarts = 5 // FIXME config private val restartCounter = new RestartCounter(maxRestarts, restartTimeout) private val largeMessageChannelEnabled = largeMessageDestinations.children.nonEmpty - @volatile private[this] var queue: SourceQueueWithComplete[Send] = _ - @volatile private[this] var largeQueue: SourceQueueWithComplete[Send] = _ - @volatile private[this] var controlQueue: SourceQueueWithComplete[Send] = _ + // 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[Send] = + new ManyToOneConcurrentArrayQueue[Send](capacity) + + @volatile private[this] var queue: SendQueue.ProducerApi[Send] = QueueWrapper(createQueue(queueSize)) + @volatile private[this] var largeQueue: SendQueue.ProducerApi[Send] = QueueWrapper(createQueue(largeQueueSize)) + @volatile private[this] var controlQueue: SendQueue.ProducerApi[Send] = QueueWrapper(createQueue(controlQueueSize)) @volatile private[this] var _outboundControlIngress: OutboundControlIngress = _ @volatile private[this] var materializing = new CountDownLatch(1) @@ -137,17 +157,20 @@ private[akka] class Association( // FIXME: Use a different envelope than the old Send, but make sure the new is handled by deadLetters properly message match { case _: SystemMessage | ClearSystemMessageDelivery ⇒ - implicit val ec = materializer.executionContext - controlQueue.offer(Send(message, senderOption, recipient, None)).onFailure { - case e ⇒ - quarantine(reason = s"Due to overflow of control queue, size [$controlQueueSize]") + val send = Send(message, senderOption, recipient, None) + if (!controlQueue.offer(send)) { + quarantine(reason = s"Due to overflow of control queue, size [$controlQueueSize]") + transport.system.deadLetters ! send } case _ ⇒ val send = Send(message, senderOption, recipient, None) - if (largeMessageChannelEnabled && isLargeMessageDestination(recipient)) - largeQueue.offer(send) - else - queue.offer(send) + val offerOk = + if (largeMessageChannelEnabled && isLargeMessageDestination(recipient)) + largeQueue.offer(send) + else + queue.offer(send) + if (!offerOk) + transport.system.deadLetters ! send } } else if (log.isDebugEnabled) log.debug("Dropping message to quarantined system {}", remoteAddress) @@ -218,17 +241,23 @@ private[akka] class Association( } - // Idempotent + /** + * 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. + */ def associate(): Unit = { - if (controlQueue eq null) { - // it's important to materialize the outboundControl stream first, - // so that outboundControlIngress is ready when stages for all streams start - runOutboundControlStream() - runOutboundOrdinaryMessagesStream() + if (!controlQueue.isInstanceOf[QueueWrapper]) + throw new IllegalStateException("associate() must only be called once") - if (largeMessageChannelEnabled) { - runOutboundLargeMessagesStream() - } + // it's important to materialize the outboundControl stream first, + // so that outboundControlIngress is ready when stages for all streams start + runOutboundControlStream() + runOutboundOrdinaryMessagesStream() + + if (largeMessageChannelEnabled) { + runOutboundLargeMessagesStream() } } @@ -236,10 +265,15 @@ private[akka] class Association( // stage in the control stream may access the outboundControlIngress before returned here // using CountDownLatch to make sure that materialization is completed before accessing outboundControlIngress materializing = new CountDownLatch(1) - val (q, (control, completed)) = Source.queue(controlQueueSize, OverflowStrategy.backpressure) + + val wrapper = getOrCreateQueueWrapper(controlQueue, queueSize) + controlQueue = wrapper // use new underlying queue immediately for restarts + val (queueValue, (control, completed)) = Source.fromGraph(new SendQueue[Send]) .toMat(transport.outboundControl(this))(Keep.both) .run()(materializer) - controlQueue = q + queueValue.inject(wrapper.queue) + // replace with the materialized value, still same underlying queue + controlQueue = queueValue _outboundControlIngress = control materializing.countDown() attachStreamRestart("Outbound control stream", completed, cause ⇒ { @@ -251,19 +285,35 @@ private[akka] class Association( }) } + private def getOrCreateQueueWrapper(q: SendQueue.ProducerApi[Send], capacity: Int): QueueWrapper = + q match { + case existing: QueueWrapper ⇒ existing + case _ ⇒ + // use new queue for restarts + QueueWrapper(createQueue(capacity)) + } + private def runOutboundOrdinaryMessagesStream(): Unit = { - val (q, completed) = Source.queue(256, OverflowStrategy.dropBuffer) + val wrapper = getOrCreateQueueWrapper(queue, queueSize) + queue = wrapper // use new underlying queue immediately for restarts + val (queueValue, completed) = Source.fromGraph(new SendQueue[Send]) .toMat(transport.outbound(this))(Keep.both) .run()(materializer) - queue = q + queueValue.inject(wrapper.queue) + // replace with the materialized value, still same underlying queue + queue = queueValue attachStreamRestart("Outbound message stream", completed, _ ⇒ runOutboundOrdinaryMessagesStream()) } private def runOutboundLargeMessagesStream(): Unit = { - val (q, completed) = Source.queue(256, OverflowStrategy.dropBuffer) + val wrapper = getOrCreateQueueWrapper(queue, largeQueueSize) + largeQueue = wrapper // use new underlying queue immediately for restarts + val (queueValue, completed) = Source.fromGraph(new SendQueue[Send]) .toMat(transport.outboundLarge(this))(Keep.both) .run()(materializer) - largeQueue = q + queueValue.inject(wrapper.queue) + // replace with the materialized value, still same underlying queue + largeQueue = queueValue attachStreamRestart("Outbound large message stream", completed, _ ⇒ runOutboundLargeMessagesStream()) } @@ -297,21 +347,21 @@ private[akka] class Association( * INTERNAL API */ private[remote] class AssociationRegistry(createAssociation: Address ⇒ Association) { - // FIXME: This does locking on putIfAbsent, we need something smarter - private[this] val associationsByAddress = new ConcurrentHashMap[Address, Association]() - private[this] val associationsByUid = new ConcurrentHashMap[Long, Association]() + private[this] val associationsByAddress = new AtomicReference[Map[Address, Association]](Map.empty) + private[this] val associationsByUid = new ConcurrentHashMap[Long, Association]() // FIXME replace with specialized Long Map - def association(remoteAddress: Address): Association = { - val current = associationsByAddress.get(remoteAddress) - if (current ne null) current - else { - associationsByAddress.computeIfAbsent(remoteAddress, new JFunction[Address, Association] { - override def apply(remoteAddress: Address): Association = { - val newAssociation = createAssociation(remoteAddress) - newAssociation.associate() // This is a bit costly for this blocking method :( + @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 } } 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..ea10fba5b5 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala @@ -0,0 +1,131 @@ +/** + * 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 + } + + 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 = { + 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) + logic.wakeup() + result + } + } + + (logic, queueValue) + + } +} 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() + } + + } + +} From a97290841fdb0c4d462e494ac5276234eb0274b3 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 3 Jun 2016 17:05:07 +0200 Subject: [PATCH 057/186] port some remote tests to Artery * Some are failing and are marked as ignored, will open separate issues for those. * All interesting tests apart from the big RemotingSpec.scala are ported. Relevant parts of it should be ported but into smaller more focused tests. --- .../remote/artery/RemoteDeathWatchSpec.scala | 95 ++++++ .../remote/artery/RemoteDeployerSpec.scala | 60 ++++ .../remote/artery/RemoteDeploymentSpec.scala | 82 +++++ .../akka/remote/artery/RemoteRouterSpec.scala | 258 ++++++++++++++ .../artery/RemoteSendConsistencySpec.scala | 2 +- .../remote/artery/RemoteWatcherSpec.scala | 321 ++++++++++++++++++ .../akka/remote/artery/UntrustedSpec.scala | 187 ++++++++++ 7 files changed, 1004 insertions(+), 1 deletion(-) create mode 100644 akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/RemoteDeployerSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala 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..7313806127 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala @@ -0,0 +1,95 @@ +/** + * 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 = "akka.remote.RemoteActorRefProvider" + deployment { + /watchers.remote = "artery://other@localhost:$otherPort" + } + } + remote.watch-failure-detector.acceptable-heartbeat-pause = 3s + # FIXME do we need the initial-system-message-delivery-timeout? + remote.initial-system-message-delivery-timeout = 3 s + remote.artery.enabled = on + remote.artery.hostname = localhost + remote.artery.port = 0 + } + """) +} + +class RemoteDeathWatchSpec extends AkkaSpec(RemoteDeathWatchSpec.config) with ImplicitSender with DefaultTimeout with DeathWatchSpec { + import RemoteDeathWatchSpec._ + + val other = ActorSystem("other", ConfigFactory.parseString(s"akka.remote.artery.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"artery://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) + } + } + } + + // FIXME this is failing with Artery + "receive Terminated when watched node is unknown host" ignore { + val path = RootActorPath(Address("artery", 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) + } + + // FIXME this is failing with Artery + "receive ActorIdentity(None) when identified node is unknown host" ignore { + val path = RootActorPath(Address("artery", 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..9020f1951a --- /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 = "akka.remote.RemoteActorRefProvider" + 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.hostname = localhost + akka.remote.artery.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..8f89196a68 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala @@ -0,0 +1,82 @@ +/** + * 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.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 = "akka.remote.RemoteActorRefProvider" + akka.remote.artery.enabled = on + akka.remote.artery.hostname = localhost + akka.remote.artery.port = 0 + """) { + + import RemoteDeploymentSpec._ + + val port = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress.port.get + val conf = ConfigFactory.parseString( + s""" + akka.actor.deployment { + /blub.remote = "artery://${system.name}@localhost:$port" + } + """).withFallback(system.settings.config) + + val masterSystem = ActorSystem("Master" + system.name, conf) + val masterPort = masterSystem.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress.port.get + + override def afterTermination(): Unit = { + shutdown(masterSystem) + } + + "Remoting" must { + + // FIXME this test is failing with Artery + "create and supervise children on remote node" ignore { + val senderProbe = TestProbe()(masterSystem) + val r = masterSystem.actorOf(Props[Echo1], "blub") + r.path.toString should ===(s"artery://${system.name}@localhost:${port}/remote/artery/${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/RemoteRouterSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala new file mode 100644 index 0000000000..3700df47c8 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala @@ -0,0 +1,258 @@ +/** + * 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.RemoteScope + +object RemoteRouterSpec { + class Parent extends Actor { + def receive = { + case (p: Props, name: String) ⇒ + sender() ! context.actorOf(p, name) + } + } +} + +class RemoteRouterSpec extends AkkaSpec(""" + akka.loglevel=DEBUG + akka.actor.provider = "akka.remote.RemoteActorRefProvider" + akka.remote.artery.enabled = on + akka.remote.artery.hostname = localhost + akka.remote.artery.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 = system.asInstanceOf[ExtendedActorSystem].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 = ["artery://${sysName}@localhost:${port}"] + } + /elastic-blub { + router = round-robin-pool + resizer { + lower-bound = 2 + upper-bound = 3 + } + target.nodes = ["artery://${sysName}@localhost:${port}"] + } + /remote-blub { + remote = "artery://${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 = ["artery://${sysName}@localhost:${port}"] + } + /local-blub2 { + router = round-robin-pool + nr-of-instances = 4 + target.nodes = ["artery://${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 { + + // FIXME this test is failing with Artery + "deploy its children on remote host driven by configuration" ignore { + val probe = TestProbe()(masterSystem) + val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps), "blub") + val replies = collectRouteePaths(probe, router, 5) + println(s"# replies $replies") // FIXME + val children = replies.toSet + children should have size 2 + children.map(_.parent) should have size 1 + children foreach (_.address.toString should ===(s"artery://${sysName}@localhost:${port}")) + masterSystem.stop(router) + } + + // FIXME this test is failing with Artery + "deploy its children on remote host driven by programatic definition" ignore { + val probe = TestProbe()(masterSystem) + val router = masterSystem.actorOf(new RemoteRouterConfig( + RoundRobinPool(2), + Seq(Address("artery", 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"artery://${sysName}@localhost:${port}")) + masterSystem.stop(router) + } + + // FIXME this test is failing with Artery + "deploy dynamic resizable number of children on remote host driven by configuration" ignore { + 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"artery://${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"artery://${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"artery://${sysName}@localhost:${port}")) + masterSystem.stop(router) + } + + // FIXME this test is failing with Artery + "deploy remote routers based on explicit deployment" ignore { + val probe = TestProbe()(masterSystem) + val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps) + .withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"artery://${sysName}@localhost:${port}")))), "remote-blub2") + router.path.address.toString should ===(s"artery://${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"artery://${sysName}@localhost:${port}")) + masterSystem.stop(router) + } + + // FIXME this test is failing with Artery + "let remote deployment be overridden by local configuration" ignore { + val probe = TestProbe()(masterSystem) + val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps) + .withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"artery://${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("artery", sysName, "localhost", port)) + children foreach (_.address.toString should ===(s"artery://${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"artery://${sysName}@localhost:${port}")))), "local-blub2") + router.path.address.toString should ===(s"artery://${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"artery://${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"artery://${sysName}@localhost:${port}")))), "remote-override") + router.path.address.toString should ===(s"artery://${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"artery://${sysName}@localhost:${port}")) + masterSystem.stop(router) + } + + // FIXME this test is failing with Artery + "set supplied supervisorStrategy" ignore { + 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("artery", 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 index a7256705a8..d14861415f 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala @@ -67,7 +67,7 @@ class RemoteSendConsistencySpec extends AkkaSpec(RemoteSendConsistencySpec.confi } val senderProps = Props(new Actor { - var counter = 100 // FIXME try this test with 1000, why does it take so long? + var counter = 1000 remoteRef ! counter override def receive: Receive = { 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..9a5f11d1f3 --- /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) + final case class Quarantined(address: Address, uid: Option[Int]) + } + + 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[Int]): 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 = "akka.remote.RemoteActorRefProvider" + remote.artery.enabled = on + remote.artery.hostname = localhost + remote.artery.port = 0 + }""") with ImplicitSender { + + import RemoteWatcherSpec._ + import RemoteWatcher._ + + override def expectedTestDuration = 2.minutes + + val remoteSystem = ActorSystem("RemoteSystem", system.settings.config) + val remoteAddress = remoteSystem.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + def remoteAddressUid = AddressUidExtension(remoteSystem).addressUid + + Seq(system, remoteSystem).foreach(muteDeadLetters( + akka.remote.transport.AssociationHandle.Disassociated.getClass, + akka.remote.transport.ActorTransportAdapter.DisassociateUnderlying.getClass)(_)) + + override def afterTermination() { + shutdown(remoteSystem) + } + + val heartbeatRspB = HeartbeatRsp(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(Heartbeat) + expectNoMsg(100 millis) + monitorA ! HeartbeatTick + expectMsg(Heartbeat) + expectNoMsg(100 millis) + monitorA.tell(heartbeatRspB, monitorB) + monitorA ! HeartbeatTick + expectMsg(Heartbeat) + 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(Heartbeat) + 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(Heartbeat) + 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(Heartbeat) + monitorA.tell(heartbeatRspB, monitorB) + expectNoMsg(1 second) + monitorA ! HeartbeatTick + expectMsg(Heartbeat) + monitorA.tell(heartbeatRspB, monitorB) + + within(10 seconds) { + awaitAssert { + monitorA ! HeartbeatTick + expectMsg(Heartbeat) + // 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(Heartbeat) + // no HeartbeatRsp sent + + within(20 seconds) { + awaitAssert { + monitorA ! HeartbeatTick + expectMsg(Heartbeat) + // 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(Heartbeat) + monitorA.tell(heartbeatRspB, monitorB) + expectNoMsg(1 second) + monitorA ! HeartbeatTick + expectMsg(Heartbeat) + monitorA.tell(heartbeatRspB, monitorB) + + within(10 seconds) { + awaitAssert { + monitorA ! HeartbeatTick + expectMsg(Heartbeat) + // 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(Heartbeat) + monitorA.tell(heartbeatRspB, monitorB) + expectNoMsg(1 second) + monitorA ! HeartbeatTick + expectMsg(Heartbeat) + monitorA.tell(heartbeatRspB, monitorB) + monitorA ! HeartbeatTick + expectMsg(Heartbeat) + monitorA ! ReapUnreachableTick + p.expectNoMsg(1 second) + monitorA ! HeartbeatTick + expectMsg(Heartbeat) + monitorA.tell(heartbeatRspB, monitorB) + monitorA ! HeartbeatTick + expectMsg(Heartbeat) + 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(Heartbeat) + // 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/UntrustedSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala new file mode 100644 index 0000000000..65798d6956 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala @@ -0,0 +1,187 @@ +/** + * 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.testkit.EventFilter + +object UntrustedSpec { + final case class IdentifyReq(path: String) + final case class StopChild(name: String) + + 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 = akka.remote.RemoteActorRefProvider + akka.remote.untrusted-mode = on + akka.remote.trusted-selection-paths = ["/user/receptionist", ] + akka.remote.artery.enabled = on + akka.remote.artery.hostname = localhost + akka.remote.artery.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 = akka.remote.RemoteActorRefProvider + akka.remote.artery.enabled = on + akka.remote.artery.hostname = localhost + akka.remote.artery.port = 0 + """)) + val addr = system.asInstanceOf[ExtendedActorSystem].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) + } + + } + +} From 089dd866320678c2379a34b54972cd3dbd43e25f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Endre=20S=C3=A1ndor=20Varga?= Date: Mon, 6 Jun 2016 13:36:05 +0200 Subject: [PATCH 058/186] Initial AFR instrumentation --- .../artery/AeronStreamConcistencySpec.scala | 10 +-- .../artery/AeronStreamLatencySpec.scala | 10 +-- .../artery/AeronStreamMaxThroughputSpec.scala | 4 +- .../scala/akka/remote/artery/AeronSink.scala | 22 +++++- .../akka/remote/artery/AeronSource.scala | 19 ++++- .../akka/remote/artery/ArteryTransport.scala | 69 ++++++++++++++++--- .../akka/remote/artery/FlightRecorder.scala | 54 +++++++++++---- .../remote/artery/FlightRecorderEvents.scala | 38 ++++++++++ .../akka/remote/artery/AeronSinkSpec.scala | 4 +- .../akka/remote/artery/AeronStreamsApp.scala | 16 ++--- .../remote/artery/FlightRecorderSpec.scala | 15 ++++ 11 files changed, 212 insertions(+), 49 deletions(-) create mode 100644 akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala 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 index 880dafd727..9f92fdd8ec 100644 --- 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 @@ -96,8 +96,8 @@ abstract class AeronStreamConsistencySpec "start echo" in { runOn(second) { // just echo back - Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner, pool)) - .runWith(new AeronSink(channel(first), streamId, aeron, taskRunner, pool, giveUpSendAfter)) + Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner, pool, IgnoreEventSink)) + .runWith(new AeronSink(channel(first), streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) } enterBarrier("echo-started") } @@ -110,7 +110,7 @@ abstract class AeronStreamConsistencySpec val killSwitch = KillSwitches.shared("test") val started = TestProbe() val startMsg = "0".getBytes("utf-8") - Source.fromGraph(new AeronSource(channel(first), streamId, aeron, taskRunner, pool)) + Source.fromGraph(new AeronSource(channel(first), streamId, aeron, taskRunner, pool, IgnoreEventSink)) .via(killSwitch.flow) .runForeach { envelope ⇒ val bytes = ByteString.fromByteBuffer(envelope.byteBuffer) @@ -138,7 +138,7 @@ abstract class AeronStreamConsistencySpec envelope } .throttle(1, 200.milliseconds, 1, ThrottleMode.Shaping) - .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) started.expectMsg(Done) } @@ -150,7 +150,7 @@ abstract class AeronStreamConsistencySpec envelope.byteBuffer.flip() envelope } - .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) Await.ready(done, 20.seconds) killSwitch.shutdown() 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 index 8dfd030aef..5f5a77b950 100644 --- 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 @@ -200,7 +200,7 @@ abstract class AeronStreamLatencySpec val killSwitch = KillSwitches.shared(testName) val started = TestProbe() val startMsg = "0".getBytes("utf-8") - Source.fromGraph(new AeronSource(channel(first), streamId, aeron, taskRunner, pool)) + Source.fromGraph(new AeronSource(channel(first), streamId, aeron, taskRunner, pool, IgnoreEventSink)) .via(killSwitch.flow) .runForeach { envelope ⇒ val bytes = ByteString.fromByteBuffer(envelope.byteBuffer) @@ -228,7 +228,7 @@ abstract class AeronStreamLatencySpec envelope } .throttle(1, 200.milliseconds, 1, ThrottleMode.Shaping) - .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) started.expectMsg(Done) } @@ -246,7 +246,7 @@ abstract class AeronStreamLatencySpec sendTimes.set(n - 1, System.nanoTime()) envelope } - .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) barrier.await((totalMessages / messageRate) + 10, SECONDS) } @@ -264,8 +264,8 @@ abstract class AeronStreamLatencySpec "start echo" in { runOn(second) { // just echo back - Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner, pool)) - .runWith(new AeronSink(channel(first), streamId, aeron, taskRunner, pool, giveUpSendAfter)) + Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner, pool, IgnoreEventSink)) + .runWith(new AeronSink(channel(first), streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) } enterBarrier("echo-started") } 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 index e8460d8b4e..49e8e6abb5 100644 --- 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 @@ -174,7 +174,7 @@ abstract class AeronStreamMaxThroughputSpec var count = 0L val done = TestLatch(1) val killSwitch = KillSwitches.shared(testName) - Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner, pool)) + Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner, pool, IgnoreEventSink)) .via(killSwitch.flow) .runForeach { envelope ⇒ val bytes = ByteString.fromByteBuffer(envelope.byteBuffer) @@ -212,7 +212,7 @@ abstract class AeronStreamMaxThroughputSpec envelope.byteBuffer.flip() envelope } - .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) printStats("sender") enterBarrier(testName + "-done") diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala index a9a490579d..ee7adaa1e2 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala @@ -70,10 +70,19 @@ object AeronSink { /** * @param channel eg. "aeron:udp?endpoint=localhost:40123" */ -class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRunner, pool: EnvelopeBufferPool, giveUpSendAfter: Duration) +class AeronSink( + channel: String, + streamId: Int, + aeron: Aeron, + taskRunner: TaskRunner, + pool: EnvelopeBufferPool, + giveUpSendAfter: 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) @@ -96,15 +105,22 @@ class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRu private var offerTaskInProgress = false + 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 = { taskRunner.command(Remove(addOfferTask.task)) + flightRecorder.loFreq(AeronSink_TaskRunnerRemoved, channelMetadata) pub.close() + flightRecorder.loFreq(AeronSink_PublicationClosed, channelMetadata) completed.complete(completedValue) + flightRecorder.loFreq(AeronSink_Stopped, channelMetadata) } // InHandler @@ -112,6 +128,7 @@ class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRu envelopeInFlight = grab(in) backoffCount = spinning lastMsgSize = envelopeInFlight.byteBuffer.limit + flightRecorder.hiFreq(AeronSink_EnvelopeGrabbed, lastMsgSize) publish() } @@ -130,6 +147,7 @@ class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRu offerTask.buffer = envelopeInFlight.aeronBuffer offerTask.msgSize = lastMsgSize taskRunner.command(addOfferTask) + flightRecorder.hiFreq(AeronSink_DelegateToTaskRunner, lastMsgSize) } } else { onOfferSuccess() @@ -137,6 +155,7 @@ class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRu } private def onOfferSuccess(): Unit = { + flightRecorder.hiFreq(AeronSink_EnvelopeOffered, lastMsgSize) offerTaskInProgress = false pool.release(envelopeInFlight) offerTask.buffer = null @@ -151,6 +170,7 @@ class AeronSink(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRu private def onGiveUp(): Unit = { offerTaskInProgress = false val cause = new GaveUpSendingException(s"Gave up sending message to $channel after $giveUpSendAfter.") + flightRecorder.alert(AeronSink_GaveUpEnvelope, cause.getMessage.getBytes("US-ASCII")) completedValue = Failure(cause) failStage(cause) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala index 48af59f14d..4f2947bc5e 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala @@ -61,10 +61,18 @@ object AeronSource { /** * @param channel eg. "aeron:udp?endpoint=localhost:40123" */ -class AeronSource(channel: String, streamId: Int, aeron: Aeron, taskRunner: TaskRunner, pool: EnvelopeBufferPool) +class AeronSource( + channel: String, + streamId: Int, + aeron: Aeron, + taskRunner: TaskRunner, + pool: EnvelopeBufferPool, + flightRecorder: EventSink +) extends GraphStage[SourceShape[EnvelopeBuffer]] { import AeronSource._ import TaskRunner._ + import FlightRecorderEvents._ val out: Outlet[EnvelopeBuffer] = Outlet("AeronSource") override val shape: SourceShape[EnvelopeBuffer] = SourceShape(out) @@ -85,9 +93,16 @@ class AeronSource(channel: String, streamId: Int, aeron: Aeron, taskRunner: Task private val messageHandler = new MessageHandler(pool) private val addPollTask: Add = Add(pollTask(sub, messageHandler, getAsyncCallback(onMessage))) + private val channelMetadata = channel.getBytes("US-ASCII") + + 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 @@ -115,12 +130,14 @@ class AeronSource(channel: String, streamId: Int, aeron: Aeron, taskRunner: Task subscriberLoop() // recursive } else { // delegate backoff to shared TaskRunner + flightRecorder.hiFreq(AeronSource_DelegateToTaskRunner, 0) taskRunner.command(addPollTask) } } } private def onMessage(data: EnvelopeBuffer): Unit = { + flightRecorder.hiFreq(AeronSource_Received, data.byteBuffer.limit) push(out, data) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index a5d5338395..91c122303c 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -5,6 +5,7 @@ package akka.remote.artery import java.io.File import java.nio.ByteOrder + import scala.concurrent.Future import scala.concurrent.Promise import scala.concurrent.duration._ @@ -60,11 +61,14 @@ import org.agrona.ErrorHandler import org.agrona.IoUtil import java.io.File import java.net.InetSocketAddress -import java.nio.channels.DatagramChannel +import java.nio.channels.{ DatagramChannel, FileChannel } + import akka.remote.artery.OutboundControlJunction.OutboundControlIngress import io.aeron.CncFileDescriptor import java.util.concurrent.atomic.AtomicLong + import akka.actor.Cancellable + import scala.collection.JavaConverters._ import akka.stream.ActorMaterializerSettings /** @@ -213,6 +217,7 @@ private[akka] trait OutboundContext { private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: RemoteActorRefProvider) extends RemoteTransport(_system, _provider) with InboundContext { import provider.remoteSettings + import FlightRecorderEvents._ // these vars are initialized once in the start method @volatile private[this] var _localAddress: UniqueAddress = _ @@ -266,6 +271,11 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R val envelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) val largeEnvelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumLargeFrameSize, ArteryTransport.MaximumPooledBuffers) + val (afrFileChannel, afrFlie, flightRecorder) = initializeFlightRecorder() + + // !!! WARNING !!! This is *NOT* thread safe, + private val topLevelFREvents = flightRecorder.createEventSink() + // FIXME: Compression table must be owned by each channel instead // of having a global one val compression = new Compression(system) @@ -276,8 +286,11 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R override def start(): Unit = { startMediaDriver() startAeron() + topLevelFREvents.loFreq(Transport_AeronStarted, NoMetaData) startAeronErrorLog() + topLevelFREvents.loFreq(Transport_AeronErrorLogStarted, NoMetaData) taskRunner.start() + topLevelFREvents.loFreq(Transport_TaskRunnerStarted, NoMetaData) val port = if (remoteSettings.ArteryPort == 0) ArteryTransport.autoSelectPort(remoteSettings.ArteryHostname) @@ -290,14 +303,18 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R AddressUidExtension(system).longAddressUid) _addresses = Set(_localAddress.address) + // 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")) + val materializerSettings = ActorMaterializerSettings( remoteSettings.config.getConfig("akka.remote.artery.advanced.materializer")) materializer = ActorMaterializer(materializerSettings)(system) - materializer = ActorMaterializer()(system) messageDispatcher = new MessageDispatcher(system, provider) + topLevelFREvents.loFreq(Transport_MaterializerStarted, NoMetaData) runInboundStreams() + topLevelFREvents.loFreq(Transport_StartupFinished, NoMetaData) log.info("Remoting started; listening on address: {}", defaultAddress) } @@ -313,6 +330,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R driverContext.driverTimeoutMs(SECONDS.toNanos(20)) val driver = MediaDriver.launchEmbedded(driverContext) log.debug("Started embedded media driver in directory [{}]", driver.aeronDirectoryName) + topLevelFREvents.loFreq(Transport_MediaDriverStarted, driver.aeronDirectoryName().getBytes("US-ASCII")) mediaDriver = Some(driver) } } @@ -322,6 +340,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R case None ⇒ remoteSettings.AeronDirectoryName } + // TODO: Add FR events private def startAeron(): Unit = { val ctx = new Aeron.Context @@ -355,6 +374,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R aeron = Aeron.connect(ctx) } + // TODO Add FR Events private def startAeronErrorLog(): Unit = { val errorLog = new AeronErrorLog(new File(aeronDir, CncFileDescriptor.CNC_FILE)) val lastTimestamp = new AtomicLong(0L) @@ -376,7 +396,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundControlStream(): Unit = { - val (c, completed) = Source.fromGraph(new AeronSource(inboundChannel, controlStreamId, aeron, taskRunner, envelopePool)) + val (c, completed) = Source.fromGraph( + new AeronSource(inboundChannel, controlStreamId, aeron, taskRunner, envelopePool, flightRecorder.createEventSink()) + ) .viaMat(inboundControlFlow)(Keep.right) .toMat(Sink.ignore)(Keep.both) .run()(materializer) @@ -413,7 +435,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundOrdinaryMessagesStream(): Unit = { - val completed = Source.fromGraph(new AeronSource(inboundChannel, ordinaryStreamId, aeron, taskRunner, envelopePool)) + val completed = Source.fromGraph( + new AeronSource(inboundChannel, ordinaryStreamId, aeron, taskRunner, envelopePool, flightRecorder.createEventSink()) + ) .via(inboundFlow) .runWith(Sink.ignore)(materializer) @@ -421,7 +445,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundLargeMessagesStream(): Unit = { - val completed = Source.fromGraph(new AeronSource(inboundChannel, largeStreamId, aeron, taskRunner, largeEnvelopePool)) + val completed = Source.fromGraph( + new AeronSource(inboundChannel, largeStreamId, aeron, taskRunner, largeEnvelopePool, flightRecorder.createEventSink() + )) .via(inboundLargeFlow) .runWith(Sink.ignore)(materializer) @@ -448,15 +474,29 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R override def shutdown(): Future[Done] = { _shutdown = true killSwitch.shutdown() - if (taskRunner != null) taskRunner.stop() - if (aeronErrorLogTask != null) aeronErrorLogTask.cancel() + topLevelFREvents.loFreq(Transport_KillSwitchPulled, NoMetaData) + if (taskRunner != null) { + taskRunner.stop() + topLevelFREvents.loFreq(Transport_Stopped, NoMetaData) + } + if (aeronErrorLogTask != null) { + aeronErrorLogTask.cancel() + topLevelFREvents.loFreq(Transport_AeronErrorLogTaskStopped, NoMetaData) + } if (aeron != null) aeron.close() mediaDriver.foreach { driver ⇒ // this is only for embedded media driver driver.close() // FIXME it should also be configurable to not delete dir IoUtil.delete(new File(driver.aeronDirectoryName), true) + topLevelFREvents.loFreq(Transport_MediaFileDeleted, NoMetaData) } + topLevelFREvents.loFreq(Transport_FlightRecorderClose, NoMetaData) + flightRecorder.close() + afrFileChannel.force(true) + afrFileChannel.close() + // TODO: Be smarter about this in tests and make it always-on-for prod + afrFlie.delete() Future.successful(Done) } @@ -504,7 +544,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) .via(encoder) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), ordinaryStreamId, aeron, taskRunner, - envelopePool, giveUpSendAfter))(Keep.right) + envelopePool, giveUpSendAfter, flightRecorder.createEventSink()))(Keep.right) } def outboundLarge(outboundContext: OutboundContext): Sink[Send, Future[Done]] = { @@ -512,7 +552,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) .via(createEncoder(largeEnvelopePool)) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), largeStreamId, aeron, taskRunner, - envelopePool, giveUpSendAfter))(Keep.right) + envelopePool, giveUpSendAfter, flightRecorder.createEventSink()))(Keep.right) } def outboundControl(outboundContext: OutboundContext): Sink[Send, (OutboundControlIngress, Future[Done])] = { @@ -522,7 +562,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .viaMat(new OutboundControlJunction(outboundContext))(Keep.right) .via(encoder) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), controlStreamId, aeron, taskRunner, - envelopePool, Duration.Inf))(Keep.both) + envelopePool, Duration.Inf, flightRecorder.createEventSink()))(Keep.both) // FIXME we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages } @@ -573,6 +613,15 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R Source.maybe[ByteString].via(killSwitch.flow))((a, b) ⇒ a) } + private def initializeFlightRecorder(): (FileChannel, File, FlightRecorder) = { + // TODO: Figure out where to put it, currently using temporary files + val afrFile = File.createTempFile("artery", ".afr") + afrFile.deleteOnExit() + + val fileChannel = FlightRecorder.prepareFileForFlightRecorder(afrFile) + (fileChannel, afrFile, new FlightRecorder(fileChannel)) + } + } /** diff --git a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala index 890824452d..820125998b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala @@ -3,9 +3,10 @@ */ package akka.remote.artery -import java.io.RandomAccessFile +import java.io.{ File, RandomAccessFile } import java.nio.{ ByteBuffer, ByteOrder } import java.nio.channels.FileChannel +import java.nio.file.StandardOpenOption import java.util.concurrent.atomic.AtomicBoolean import akka.util.ByteString @@ -25,6 +26,16 @@ private[remote] trait EventSink { 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 * @@ -35,8 +46,8 @@ private[remote] trait EventClock { def updateWallClock(): Unit def updateHighSpeedClock(): Unit - def getWallClockPart: Long - def getHighSpeedPart: Long + def wallClockPart: Long + def highSpeedPart: Long } @@ -64,8 +75,8 @@ private[remote] class EventClockImpl extends EventClock { highSpeedClock = System.nanoTime() - highSpeedClockOffset } - override def getWallClockPart: Long = wallClock - override def getHighSpeedPart: Long = highSpeedClock + override def wallClockPart: Long = wallClock + override def highSpeedPart: Long = highSpeedClock } /** @@ -93,11 +104,11 @@ private[remote] object RollingEventLogSection { * INTERNAL API */ private[remote] class RollingEventLogSection( - fileChannel: FileChannel, - offset: Long, - entryCount: Long, + fileChannel: FileChannel, + offset: Long, + entryCount: Long, logBufferSize: Long, - recordSize: Int) extends AtomicBoolean { + recordSize: Int) { import RollingEventLogSection._ // FIXME: check if power of two @@ -150,6 +161,16 @@ private[remote] class RollingEventLogSection( * INTERNAL API */ private[remote] object FlightRecorder { + + def prepareFileForFlightRecorder(file: File): FileChannel = { + // Force the size, otherwise memory mapping will fail on *nixes + val randomAccessFile = new RandomAccessFile(file, "rwd") + randomAccessFile.setLength(FlightRecorder.TotalSize) + randomAccessFile.close() + + FileChannel.open(file.toPath, StandardOpenOption.CREATE, StandardOpenOption.WRITE, StandardOpenOption.READ) + } + val Alignment = 64 * 1024 // Windows is picky about mapped section alignments val MagicString = 0x31524641 // "AFR1", little-endian @@ -288,11 +309,14 @@ private[akka] class FlightRecorder(val fileChannel: FileChannel) extends AtomicB private def prepareRichRecord(recordBuffer: ByteBuffer, code: Int, metadata: Array[Byte]): Unit = { recordBuffer.clear() // FIXME: This is a bit overkill, needs some smarter scheme later, no need to always store the wallclock - recordBuffer.putLong(clock.getWallClockPart) - recordBuffer.putLong(clock.getHighSpeedPart) + recordBuffer.putLong(clock.wallClockPart) + recordBuffer.putLong(clock.highSpeedPart) recordBuffer.putInt(code) - recordBuffer.put(metadata.length.toByte) - recordBuffer.put(metadata) + // 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) } @@ -313,8 +337,8 @@ private[akka] class FlightRecorder(val fileChannel: FileChannel) extends AtomicB 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.getWallClockPart) - hiFreqBatchBuffer.putLong(clock.getHighSpeedPart) + hiFreqBatchBuffer.putLong(clock.wallClockPart) + hiFreqBatchBuffer.putLong(clock.highSpeedPart) // Leave space for the size field hiFreqBatchBuffer.putLong(0L) // Reserved for now 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..1f61bb2063 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala @@ -0,0 +1,38 @@ +package akka.remote.artery + +object FlightRecorderEvents { + + 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 + + // Aeron Sink events + val AeronSink_Started = 13 + val AeronSink_TaskRunnerRemoved = 14 + val AeronSink_PublicationClosed = 15 + val AeronSink_Stopped = 16 + val AeronSink_EnvelopeGrabbed = 17 + val AeronSink_EnvelopeOffered = 18 + val AeronSink_GaveUpEnvelope = 19 + val AeronSink_DelegateToTaskRunner = 20 + + // Aeron Source events + val AeronSource_Started = 21 + val AeronSource_Stopped = 22 + val AeronSource_Received = 23 + val AeronSource_DelegateToTaskRunner = 24 + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala index 313ac9e979..0500299727 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala @@ -57,7 +57,7 @@ class AeronSinkSpec extends AkkaSpec with ImplicitSender { 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)) + 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) @@ -71,7 +71,7 @@ class AeronSinkSpec extends AkkaSpec with ImplicitSender { envelope.byteBuffer.flip() envelope } - .runWith(new AeronSink(channel, 1, aeron, taskRunner, pool, 500.millis)) + .runWith(new AeronSink(channel, 1, aeron, taskRunner, pool, 500.millis, IgnoreEventSink)) // without the give up timeout the stream would not complete/fail intercept[GaveUpSendingException] { diff --git a/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala b/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala index 12e664d732..c9f94b99ce 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala @@ -160,7 +160,7 @@ object AeronStreamsApp { var t0 = System.nanoTime() var count = 0L var payloadSize = 0L - Source.fromGraph(new AeronSource(channel1, streamId, aeron, taskRunner, pool)) + Source.fromGraph(new AeronSource(channel1, streamId, aeron, taskRunner, pool, IgnoreEventSink)) .map { envelope ⇒ r.onMessage(1, envelope.byteBuffer.limit) envelope @@ -202,19 +202,19 @@ object AeronStreamsApp { envelope.byteBuffer.flip() envelope } - .runWith(new AeronSink(channel1, streamId, aeron, taskRunner, pool, giveUpSendAfter)) + .runWith(new AeronSink(channel1, streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) } def runEchoReceiver(): Unit = { // just echo back on channel2 reporterExecutor.execute(reporter) val r = reporter - Source.fromGraph(new AeronSource(channel1, streamId, aeron, taskRunner, pool)) + Source.fromGraph(new AeronSource(channel1, streamId, aeron, taskRunner, pool, IgnoreEventSink)) .map { envelope ⇒ r.onMessage(1, envelope.byteBuffer.limit) envelope } - .runWith(new AeronSink(channel2, streamId, aeron, taskRunner, pool, giveUpSendAfter)) + .runWith(new AeronSink(channel2, streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) } def runEchoSender(): Unit = { @@ -226,7 +226,7 @@ object AeronStreamsApp { var repeat = 3 val count = new AtomicInteger var t0 = System.nanoTime() - Source.fromGraph(new AeronSource(channel2, streamId, aeron, taskRunner, pool)) + Source.fromGraph(new AeronSource(channel2, streamId, aeron, taskRunner, pool, IgnoreEventSink)) .map { envelope ⇒ r.onMessage(1, envelope.byteBuffer.limit) envelope @@ -265,7 +265,7 @@ object AeronStreamsApp { envelope.byteBuffer.flip() envelope } - .runWith(new AeronSink(channel1, streamId, aeron, taskRunner, pool, giveUpSendAfter)) + .runWith(new AeronSink(channel1, streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) barrier.await() } @@ -275,7 +275,7 @@ object AeronStreamsApp { def runDebugReceiver(): Unit = { import system.dispatcher - Source.fromGraph(new AeronSource(channel1, streamId, aeron, taskRunner, pool)) + Source.fromGraph(new AeronSource(channel1, streamId, aeron, taskRunner, pool, IgnoreEventSink)) .map { envelope ⇒ val bytes = Array.ofDim[Byte](envelope.byteBuffer.limit) envelope.byteBuffer.get(bytes) @@ -304,7 +304,7 @@ object AeronStreamsApp { envelope.byteBuffer.flip() envelope } - .runWith(new AeronSink(channel1, streamId, aeron, taskRunner, pool, giveUpSendAfter)) + .runWith(new AeronSink(channel1, streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) } def runStats(): Unit = { diff --git a/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala index 8c4f4def2e..c6249a383e 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala @@ -144,6 +144,21 @@ class FlightRecorderSpec extends AkkaSpec { 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() From ea231b1cbcfe191cc21450a9249e27544ecee10e Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 2 Jun 2016 07:21:32 +0200 Subject: [PATCH 059/186] test support for blackhole in Artery, #20589 --- .../akka/remote/testconductor/Conductor.scala | 15 +- .../akka/remote/testkit/MultiNodeSpec.scala | 1 + akka-remote/src/main/resources/reference.conf | 3 + .../scala/akka/remote/RemoteSettings.scala | 1 + .../akka/remote/artery/ArteryTransport.scala | 150 ++++++++++---- .../akka/remote/artery/Association.scala | 69 ++++++- .../scala/akka/remote/artery/TestStage.scala | 187 ++++++++++++++++++ 7 files changed, 369 insertions(+), 57 deletions(-) create mode 100644 akka-remote/src/main/scala/akka/remote/artery/TestStage.scala 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..cb1114fb12 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.EnableArtery) { + if (!transport.provider.remoteSettings.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/MultiNodeSpec.scala b/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala index bd0446bce2..cad8949dfa 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 @@ -99,6 +99,7 @@ abstract class MultiNodeConfig { if (_testTransport) ConfigFactory.parseString( """ akka.remote.netty.tcp.applied-adapters = [trttl, gremlin] + akka.remote.artery.advanced.test-mode = on """) else ConfigFactory.empty diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 59da3085ef..77463125b4 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -107,6 +107,9 @@ akka { large-message-destinations = [] advanced { + # 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} materializer { diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index dcd033b248..3eb9efdb80 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -29,6 +29,7 @@ final class RemoteSettings(val config: Config) { val EmbeddedMediaDriver = getBoolean("akka.remote.artery.advanced.embedded-media-driver") val AeronDirectoryName = getString("akka.remote.artery.advanced.aeron-dir") requiring (dir ⇒ EmbeddedMediaDriver || dir.nonEmpty, "aeron-dir must be defined when using external media driver") + val TestMode: Boolean = getBoolean("akka.remote.artery.advanced.test-mode") val LogReceive: Boolean = getBoolean("akka.remote.log-received-messages") diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index ef33bd4cbb..cda46826a3 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -4,7 +4,12 @@ package akka.remote.artery import java.io.File -import java.nio.ByteOrder +import java.net.InetSocketAddress +import java.nio.channels.DatagramChannel +import java.util.concurrent.CopyOnWriteArrayList +import java.util.concurrent.atomic.AtomicLong + +import scala.collection.JavaConverters._ import scala.concurrent.Future import scala.concurrent.Promise @@ -12,24 +17,24 @@ import scala.concurrent.duration._ import scala.util.Failure import scala.util.Success import scala.util.Try + import akka.Done import akka.NotUsed import akka.actor.ActorRef import akka.actor.Address +import akka.actor.Cancellable import akka.actor.ExtendedActorSystem import akka.actor.InternalActorRef -import akka.actor.Props import akka.event.Logging import akka.event.LoggingAdapter import akka.remote.AddressUidExtension import akka.remote.EndpointManager.Send import akka.remote.EventPublisher -import akka.remote.MessageSerializer import akka.remote.RemoteActorRef import akka.remote.RemoteActorRefProvider +import akka.remote.RemoteSettings import akka.remote.RemoteTransport import akka.remote.RemotingLifecycleEvent -import akka.remote.SeqNo import akka.remote.ThisActorSystemQuarantinedEvent import akka.remote.UniqueAddress import akka.remote.artery.InboundControlJunction.ControlMessageObserver @@ -37,22 +42,22 @@ import akka.remote.artery.InboundControlJunction.ControlMessageSubject import akka.remote.artery.OutboundControlJunction.OutboundControlIngress import akka.remote.transport.AkkaPduCodec import akka.remote.transport.AkkaPduProtobufCodec -import akka.serialization.Serialization import akka.stream.AbruptTerminationException import akka.stream.ActorMaterializer +import akka.stream.ActorMaterializerSettings import akka.stream.KillSwitches import akka.stream.Materializer import akka.stream.SharedKillSwitch import akka.stream.scaladsl.Flow -import akka.stream.scaladsl.Framing import akka.stream.scaladsl.Keep import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Source -import akka.util.{ ByteString, ByteStringBuilder, WildcardTree } import akka.util.Helpers.ConfigOps import akka.util.Helpers.Requiring +import akka.util.WildcardTree import io.aeron.Aeron import io.aeron.AvailableImageHandler +import io.aeron.CncFileDescriptor import io.aeron.Image import io.aeron.UnavailableImageHandler import io.aeron.driver.MediaDriver @@ -71,6 +76,7 @@ import akka.actor.Cancellable import scala.collection.JavaConverters._ import akka.stream.ActorMaterializerSettings + /** * INTERNAL API */ @@ -216,7 +222,6 @@ private[akka] trait OutboundContext { */ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: RemoteActorRefProvider) extends RemoteTransport(_system, _provider) with InboundContext { - import provider.remoteSettings import FlightRecorderEvents._ // these vars are initialized once in the start method @@ -240,6 +245,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val killSwitch: SharedKillSwitch = KillSwitches.shared("transportKillSwitch") @volatile private[this] var _shutdown = false + private val testStages: CopyOnWriteArrayList[TestManagementApi] = new CopyOnWriteArrayList + // FIXME config private val systemMessageResendInterval: FiniteDuration = 1.second private val handshakeRetryInterval: FiniteDuration = 1.second @@ -283,6 +290,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val associationRegistry = new AssociationRegistry( remoteAddress ⇒ new Association(this, materializer, remoteAddress, controlSubject, largeMessageDestinations)) + def remoteSettings: RemoteSettings = provider.remoteSettings + override def start(): Unit = { startMediaDriver() startAeron() @@ -396,13 +405,24 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundControlStream(): Unit = { - val (c, completed) = Source.fromGraph( - new AeronSource(inboundChannel, controlStreamId, aeron, taskRunner, envelopePool, flightRecorder.createEventSink()) - ) - .viaMat(inboundControlFlow)(Keep.right) - .toMat(Sink.ignore)(Keep.both) - .run()(materializer) - controlSubject = c + val (ctrl, completed) = + if (remoteSettings.TestMode) { + val (mgmt, (ctrl, completed)) = + aeronSource(controlStreamId, envelopePool) + .via(inboundFlow) + .viaMat(inboundTestFlow)(Keep.right) + .toMat(inboundControlSink)(Keep.both) + .run()(materializer) + testStages.add(mgmt) + (ctrl, completed) + } else { + aeronSource(controlStreamId, envelopePool) + .via(inboundFlow) + .toMat(inboundControlSink)(Keep.right) + .run()(materializer) + } + + controlSubject = ctrl controlSubject.attach(new ControlMessageObserver { override def notify(inboundEnvelope: InboundEnvelope): Unit = { @@ -435,21 +455,46 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundOrdinaryMessagesStream(): Unit = { - val completed = Source.fromGraph( - new AeronSource(inboundChannel, ordinaryStreamId, aeron, taskRunner, envelopePool, flightRecorder.createEventSink()) - ) - .via(inboundFlow) - .runWith(Sink.ignore)(materializer) + val completed = + if (remoteSettings.TestMode) { + val (mgmt, c) = aeronSource(ordinaryStreamId, envelopePool) + .via(inboundFlow) + .viaMat(inboundTestFlow)(Keep.right) + .toMat(inboundSink)(Keep.both) + .run()(materializer) + testStages.add(mgmt) + c + } else { + aeronSource(ordinaryStreamId, envelopePool) + .via(inboundFlow) + .toMat(inboundSink)(Keep.right) + .run()(materializer) + } attachStreamRestart("Inbound message stream", completed, () ⇒ runInboundOrdinaryMessagesStream()) } private def runInboundLargeMessagesStream(): Unit = { - val completed = Source.fromGraph( - new AeronSource(inboundChannel, largeStreamId, aeron, taskRunner, largeEnvelopePool, flightRecorder.createEventSink() - )) + val completed = + if (remoteSettings.TestMode) { + val (mgmt, c) = aeronSource(largeStreamId, largeEnvelopePool) + .via(inboundLargeFlow) + .viaMat(inboundTestFlow)(Keep.right) + .toMat(inboundSink)(Keep.both) + .run()(materializer) + testStages.add(mgmt) + c + } else { + aeronSource(largeStreamId, largeEnvelopePool) + .via(inboundLargeFlow) + .toMat(inboundSink)(Keep.right) + .run()(materializer) + } + + aeronSource(largeStreamId, largeEnvelopePool) .via(inboundLargeFlow) - .runWith(Sink.ignore)(materializer) + .toMat(inboundSink)(Keep.right) + .run()(materializer) attachStreamRestart("Inbound large message stream", completed, () ⇒ runInboundLargeMessagesStream()) } @@ -502,6 +547,17 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private[remote] def isShutdown(): Boolean = _shutdown + override def managementCommand(cmd: Any): Future[Boolean] = { + if (testStages.isEmpty) + Future.successful(false) + else { + import scala.collection.JavaConverters._ + import system.dispatcher + val allTestStages = testStages.asScala.toVector ++ associationRegistry.allAssociations.flatMap(_.testStages) + Future.sequence(allTestStages.map(_.send(cmd))).map(_ ⇒ true) + } + } + // InboundContext override def sendControl(to: Address, message: ControlMessage) = association(to).sendControl(message) @@ -572,6 +628,10 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R def encoder: Flow[Send, EnvelopeBuffer, NotUsed] = createEncoder(envelopePool) + def aeronSource(streamId: Int, pool: EnvelopeBufferPool): Source[EnvelopeBuffer, NotUsed] = + Source.fromGraph(new AeronSource(inboundChannel, streamId, aeron, taskRunner, pool, + flightRecorder.createEventSink())) + val messageDispatcherSink: Sink[InboundEnvelope, Future[Done]] = Sink.foreach[InboundEnvelope] { m ⇒ messageDispatcher.dispatch(m.recipient, m.recipientAddress, m.message, m.senderOption) } @@ -584,33 +644,31 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R def decoder: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = createDecoder(envelopePool) - def inboundSink: Sink[InboundEnvelope, NotUsed] = + def inboundSink: Sink[InboundEnvelope, Future[Done]] = Flow[InboundEnvelope] .via(new InboundHandshake(this, inControlStream = false)) .via(new InboundQuarantineCheck(this)) - .to(messageDispatcherSink) + .toMat(messageDispatcherSink)(Keep.right) - def inboundFlow: Flow[EnvelopeBuffer, ByteString, NotUsed] = { - Flow.fromSinkAndSource( - decoder.to(inboundSink), - Source.maybe[ByteString].via(killSwitch.flow)) + def inboundFlow: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { + Flow[EnvelopeBuffer] + .via(killSwitch.flow) + .via(decoder) } - def inboundLargeFlow: Flow[EnvelopeBuffer, ByteString, NotUsed] = { - Flow.fromSinkAndSource( - createDecoder(largeEnvelopePool).to(inboundSink), - Source.maybe[ByteString].via(killSwitch.flow)) + def inboundLargeFlow: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { + Flow[EnvelopeBuffer] + .via(killSwitch.flow) + .via(createDecoder(largeEnvelopePool)) } - def inboundControlFlow: Flow[EnvelopeBuffer, ByteString, ControlMessageSubject] = { - Flow.fromSinkAndSourceMat( - decoder - .via(new InboundHandshake(this, inControlStream = true)) - .via(new InboundQuarantineCheck(this)) - .viaMat(new InboundControlJunction)(Keep.right) - .via(new SystemMessageAcker(this)) - .to(messageDispatcherSink), - Source.maybe[ByteString].via(killSwitch.flow))((a, b) ⇒ a) + def inboundControlSink: Sink[InboundEnvelope, (ControlMessageSubject, Future[Done])] = { + Flow[InboundEnvelope] + .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(): (FileChannel, File, FlightRecorder) = { @@ -622,6 +680,12 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R (fileChannel, afrFile, new FlightRecorder(fileChannel)) } + def inboundTestFlow: Flow[InboundEnvelope, InboundEnvelope, TestManagementApi] = + Flow.fromGraph(new InboundTestStage(this)) + + def outboundTestFlow(association: Association): Flow[Send, Send, TestManagementApi] = + Flow.fromGraph(new OutboundTestStage(association)) + } /** diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 38b1980a62..fefce1f329 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -6,6 +6,7 @@ package akka.remote.artery import java.util.Queue import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.CountDownLatch import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicReference @@ -61,7 +62,7 @@ private[remote] class Association( import Association._ private val log = Logging(transport.system, getClass.getName) - private val controlQueueSize = transport.provider.remoteSettings.SysMsgBufferSize + private val controlQueueSize = transport.remoteSettings.SysMsgBufferSize // FIXME config queue size, and it should perhaps also be possible to use some kind of LinkedQueue // such as agrona.ManyToOneConcurrentLinkedQueue or AbstractNodeQueue for less memory consumption private val queueSize = 3072 @@ -85,6 +86,13 @@ private[remote] class Association( @volatile private[this] var _outboundControlIngress: OutboundControlIngress = _ @volatile private[this] var materializing = new CountDownLatch(1) + private val _testStages: CopyOnWriteArrayList[TestManagementApi] = new CopyOnWriteArrayList + + def testStages(): List[TestManagementApi] = { + import scala.collection.JavaConverters._ + _testStages.asScala.toList + } + def outboundControlIngress: OutboundControlIngress = { if (_outboundControlIngress ne null) _outboundControlIngress @@ -268,9 +276,22 @@ private[remote] class Association( val wrapper = getOrCreateQueueWrapper(controlQueue, queueSize) controlQueue = wrapper // use new underlying queue immediately for restarts - val (queueValue, (control, completed)) = Source.fromGraph(new SendQueue[Send]) - .toMat(transport.outboundControl(this))(Keep.both) - .run()(materializer) + + val (queueValue, (control, completed)) = + if (transport.remoteSettings.TestMode) { + val ((queueValue, mgmt), (control, completed)) = + Source.fromGraph(new SendQueue[Send]) + .viaMat(transport.outboundTestFlow(this))(Keep.both) + .toMat(transport.outboundControl(this))(Keep.both) + .run()(materializer) + _testStages.add(mgmt) + (queueValue, (control, completed)) + } else { + Source.fromGraph(new SendQueue[Send]) + .toMat(transport.outboundControl(this))(Keep.both) + .run()(materializer) + } + queueValue.inject(wrapper.queue) // replace with the materialized value, still same underlying queue controlQueue = queueValue @@ -296,21 +317,46 @@ private[remote] class Association( private def runOutboundOrdinaryMessagesStream(): Unit = { val wrapper = getOrCreateQueueWrapper(queue, queueSize) queue = wrapper // use new underlying queue immediately for restarts - val (queueValue, completed) = Source.fromGraph(new SendQueue[Send]) - .toMat(transport.outbound(this))(Keep.both) - .run()(materializer) + + val (queueValue, completed) = + if (transport.remoteSettings.TestMode) { + val ((queueValue, mgmt), completed) = Source.fromGraph(new SendQueue[Send]) + .viaMat(transport.outboundTestFlow(this))(Keep.both) + .toMat(transport.outbound(this))(Keep.both) + .run()(materializer) + _testStages.add(mgmt) + (queueValue, completed) + } else { + Source.fromGraph(new SendQueue[Send]) + .toMat(transport.outbound(this))(Keep.both) + .run()(materializer) + } + queueValue.inject(wrapper.queue) // replace with the materialized value, still same underlying queue queue = queueValue + attachStreamRestart("Outbound message stream", completed, _ ⇒ runOutboundOrdinaryMessagesStream()) } private def runOutboundLargeMessagesStream(): Unit = { val wrapper = getOrCreateQueueWrapper(queue, largeQueueSize) largeQueue = wrapper // use new underlying queue immediately for restarts - val (queueValue, completed) = Source.fromGraph(new SendQueue[Send]) - .toMat(transport.outboundLarge(this))(Keep.both) - .run()(materializer) + + val (queueValue, completed) = + if (transport.remoteSettings.TestMode) { + val ((queueValue, mgmt), completed) = Source.fromGraph(new SendQueue[Send]) + .viaMat(transport.outboundTestFlow(this))(Keep.both) + .toMat(transport.outboundLarge(this))(Keep.both) + .run()(materializer) + _testStages.add(mgmt) + (queueValue, completed) + } else { + Source.fromGraph(new SendQueue[Send]) + .toMat(transport.outboundLarge(this))(Keep.both) + .run()(materializer) + } + queueValue.inject(wrapper.queue) // replace with the materialized value, still same underlying queue largeQueue = queueValue @@ -375,4 +421,7 @@ private[remote] class AssociationRegistry(createAssociation: Address ⇒ Associa throw new IllegalArgumentException(s"UID collision old [$previous] new [$a]") a } + + def allAssociations: Set[Association] = + associationsByAddress.get.values.toSet } 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..ec70bb69cf --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala @@ -0,0 +1,187 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import scala.concurrent.ExecutionContext +import scala.concurrent.Future +import scala.concurrent.Promise +import scala.concurrent.duration._ + +import akka.Done +import akka.actor.Address +import akka.remote.EndpointManager.Send +import akka.remote.transport.ThrottlerTransportAdapter.Blackhole +import akka.remote.transport.ThrottlerTransportAdapter.Direction +import akka.remote.transport.ThrottlerTransportAdapter.SetThrottle +import akka.remote.transport.ThrottlerTransportAdapter.Unthrottled +import akka.stream.Attributes +import akka.stream.FlowShape +import akka.stream.Inlet +import akka.stream.Outlet +import akka.stream.stage.AsyncCallback +import akka.stream.stage.CallbackWrapper +import akka.stream.stage.GraphStageWithMaterializedValue +import akka.stream.stage.InHandler +import akka.stream.stage.OutHandler +import akka.stream.stage.TimerGraphStageLogic + +/** + * INTERNAL API + */ +private[remote] trait TestManagementApi { + def send(command: Any)(implicit ec: ExecutionContext): Future[Done] +} + +/** + * INTERNAL API + */ +private[remote] class TestManagementApiImpl(stopped: Future[Done], callback: AsyncCallback[TestManagementMessage]) + extends TestManagementApi { + + override def send(command: Any)(implicit ec: ExecutionContext): Future[Done] = { + if (stopped.isCompleted) + Future.successful(Done) + else { + val done = Promise[Done]() + callback.invoke(TestManagementMessage(command, done)) + Future.firstCompletedOf(List(done.future, stopped)) + } + } +} + +/** + * INTERNAL API + */ +private[remote] final case class TestManagementMessage(command: Any, done: Promise[Done]) + +/** + * INTERNAL API + */ +private[remote] class OutboundTestStage(outboundContext: OutboundContext) + extends GraphStageWithMaterializedValue[FlowShape[Send, Send], TestManagementApi] { + val in: Inlet[Send] = Inlet("OutboundTestStage.in") + val out: Outlet[Send] = Outlet("OutboundTestStage.out") + override val shape: FlowShape[Send, Send] = 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 TimerGraphStageLogic(shape) with CallbackWrapper[TestManagementMessage] with InHandler with OutHandler with StageLogging { + + private var blackhole = Set.empty[Address] + + private val callback = getAsyncCallback[TestManagementMessage] { + case TestManagementMessage(command, done) ⇒ + command match { + case SetThrottle(address, Direction.Send | Direction.Both, Blackhole) ⇒ + log.info("blackhole outbound messages to {}", address) + blackhole += address + case SetThrottle(address, Direction.Send | Direction.Both, Unthrottled) ⇒ + log.info("accept outbound messages to {}", address) + blackhole -= address + case _ ⇒ // not interested + } + done.success(Done) + } + + override def preStart(): Unit = { + initCallback(callback.invoke) + } + + override def postStop(): Unit = stoppedPromise.success(Done) + + // InHandler + override def onPush(): Unit = { + val env = grab(in) + if (blackhole(outboundContext.remoteAddress)) { + log.debug( + "dropping outbound message [{}] to [{}] because of blackhole", + env.message.getClass.getName, outboundContext.remoteAddress) + pull(in) // drop message + } else + push(out, env) + } + + // OutHandler + override def onPull(): Unit = pull(in) + + setHandlers(in, out, this) + } + + val managementApi: TestManagementApi = new TestManagementApiImpl(stoppedPromise.future, logic) + + (logic, managementApi) + } + +} + +/** + * INTERNAL API + */ +private[remote] class InboundTestStage(inboundContext: InboundContext) + extends GraphStageWithMaterializedValue[FlowShape[InboundEnvelope, InboundEnvelope], TestManagementApi] { + 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 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 TimerGraphStageLogic(shape) with CallbackWrapper[TestManagementMessage] with InHandler with OutHandler with StageLogging { + + private var blackhole = Set.empty[Address] + + private val callback = getAsyncCallback[TestManagementMessage] { + case TestManagementMessage(command, done) ⇒ + command match { + case SetThrottle(address, Direction.Receive | Direction.Both, Blackhole) ⇒ + log.info("blackhole inbound messages from {}", address) + blackhole += address + case SetThrottle(address, Direction.Receive | Direction.Both, Unthrottled) ⇒ + log.info("accept inbound messages from {}", address) + blackhole -= address + case _ ⇒ // not interested + } + done.success(Done) + } + + override def preStart(): Unit = { + initCallback(callback.invoke) + } + + override def postStop(): Unit = stoppedPromise.success(Done) + + // InHandler + override def onPush(): Unit = { + val env = grab(in) + inboundContext.association(env.originUid) match { + case null ⇒ + // unknown, handshake not completed + push(out, env) + case association ⇒ + if (blackhole(association.remoteAddress)) { + log.debug( + "dropping inbound message [{}] from [{}] with UID [{}] because of blackhole", + env.message.getClass.getName, association.remoteAddress, env.originUid) + pull(in) // drop message + } else + push(out, env) + } + } + + // OutHandler + override def onPull(): Unit = pull(in) + + setHandlers(in, out, this) + } + + val managementApi: TestManagementApi = new TestManagementApiImpl(stoppedPromise.future, logic) + + (logic, managementApi) + } + +} + From 08703d6191ffaa54657fd4807eb80117525a61a6 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Sat, 4 Jun 2016 21:43:48 +0200 Subject: [PATCH 060/186] avoid unessary wakeup in SendQueue --- akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala b/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala index ea10fba5b5..6710bb1ff7 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala @@ -119,8 +119,10 @@ private[remote] final class SendQueue[T] extends GraphStageWithMaterializedValue 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) + if (result && needWakeup) { + needWakeup = false logic.wakeup() + } result } } From 6240acf25e4292e708f6563bd324f256ea1e0a5f Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Sat, 4 Jun 2016 21:44:13 +0200 Subject: [PATCH 061/186] improve performance of serializer lookup by id * Map.apply allocates Some --- .../akka/serialization/Serialization.scala | 32 +++++++++++++++++-- 1 file changed, 29 insertions(+), 3 deletions(-) diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index 673fee3706..9889759455 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -17,6 +17,7 @@ import scala.util.Success import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicReference import scala.annotation.tailrec +import java.util.NoSuchElementException object Serialization { @@ -101,7 +102,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.") @@ -116,7 +117,7 @@ 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.") @@ -161,7 +162,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { * Returns either the resulting object or throws an exception if deserialization fails. */ def deserializeByteBuffer(buf: ByteBuffer, serializerId: Int, manifest: String): AnyRef = { - 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.") @@ -288,6 +289,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 def shouldWarnAboutJavaSerializer(serializedClass: Class[_], serializer: Serializer) = { From 0b953a54b281f3133745f327a99b5efb90377028 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Sat, 4 Jun 2016 21:53:27 +0200 Subject: [PATCH 062/186] optimize quarantine uid check * specialized uid Set --- .../akka/remote/artery/ArteryTransport.scala | 45 +++++++++++++++---- 1 file changed, 37 insertions(+), 8 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index cda46826a3..796e414d64 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -76,7 +76,7 @@ import akka.actor.Cancellable import scala.collection.JavaConverters._ import akka.stream.ActorMaterializerSettings - +import scala.annotation.tailrec /** * INTERNAL API */ @@ -126,7 +126,39 @@ private[akka] trait InboundContext { */ private[akka] object AssociationState { def apply(): AssociationState = - new AssociationState(incarnation = 1, uniqueRemoteAddressPromise = Promise(), quarantined = Set.empty) + new AssociationState(incarnation = 1, uniqueRemoteAddressPromise = Promise(), quarantined = QuarantinedUidSet.empty) + + object QuarantinedUidSet { + val maxEntries = 10 // ok to not keep all old uids + def empty: QuarantinedUidSet = new QuarantinedUidSet(Array.emptyLongArray) + } + + class QuarantinedUidSet private (uids: Array[Long]) { + import QuarantinedUidSet._ + + def add(uid: Long): QuarantinedUidSet = { + if (apply(uid)) + this + else { + val newUids = Array.ofDim[Long](math.min(uids.length + 1, maxEntries)) + newUids(0) = uid + if (uids.length > 0) + System.arraycopy(uids, 0, newUids, 1, newUids.length - 1) + new QuarantinedUidSet(newUids) + } + } + + def apply(uid: Long): Boolean = { + @tailrec def find(i: Int): Boolean = + if (i == uids.length) false + else if (uids(i) == uid) true + else find(i + 1) + find(0) + } + + override def toString(): String = + uids.mkString("QuarantinedUidSet(", ",", ")") + } } /** @@ -135,7 +167,7 @@ private[akka] object AssociationState { private[akka] final class AssociationState( val incarnation: Int, val uniqueRemoteAddressPromise: Promise[UniqueAddress], - val quarantined: Set[Long]) { + val quarantined: AssociationState.QuarantinedUidSet) { /** * Full outbound address with UID for this association. @@ -154,7 +186,7 @@ private[akka] final class AssociationState( def newQuarantined(): AssociationState = uniqueRemoteAddressPromise.future.value match { case Some(Success(a)) ⇒ - new AssociationState(incarnation, uniqueRemoteAddressPromise, quarantined = quarantined + a.uid) + new AssociationState(incarnation, uniqueRemoteAddressPromise, quarantined = quarantined.add(a.uid)) case _ ⇒ this } @@ -165,10 +197,7 @@ private[akka] final class AssociationState( } } - def isQuarantined(uid: Long): Boolean = { - // FIXME does this mean boxing (allocation) because of Set[Long]? Use specialized Set. org.agrona.collections.LongHashSet? - quarantined(uid) - } + def isQuarantined(uid: Long): Boolean = quarantined(uid) override def toString(): String = { val a = uniqueRemoteAddressPromise.future.value match { From 78797e537a3a157a9aec3a211ff6c0ac9385d0c5 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Sat, 4 Jun 2016 21:57:34 +0200 Subject: [PATCH 063/186] adjust default-remote-dispatcher settings * and fix materializer creation --- akka-remote/src/main/resources/reference.conf | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 77463125b4..3c88823a5e 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -618,10 +618,11 @@ 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 { From c808522f6d7b03426e0efa828f68a886126cecc7 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Sat, 4 Jun 2016 22:14:28 +0200 Subject: [PATCH 064/186] optimize access to association UniqueAddress --- .../akka/remote/artery/ArteryTransport.scala | 22 ++++++++++++++----- .../akka/remote/artery/Association.scala | 15 +++++-------- 2 files changed, 23 insertions(+), 14 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 796e414d64..a48d91e28a 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -159,6 +159,7 @@ private[akka] object AssociationState { override def toString(): String = uids.mkString("QuarantinedUidSet(", ",", ")") } + } /** @@ -169,15 +170,26 @@ private[akka] final class AssociationState( val uniqueRemoteAddressPromise: Promise[UniqueAddress], val quarantined: AssociationState.QuarantinedUidSet) { + // 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[Try[UniqueAddress]] = { - // FIXME we should cache access to uniqueRemoteAddress.value (avoid allocations), used in many places - uniqueRemoteAddress.value + 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 = @@ -192,8 +204,8 @@ private[akka] final class AssociationState( def isQuarantined(): Boolean = { uniqueRemoteAddressValue match { - case Some(Success(a)) ⇒ isQuarantined(a.uid) - case _ ⇒ false // handshake not completed yet + case Some(a) ⇒ isQuarantined(a.uid) + case _ ⇒ false // handshake not completed yet } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index fefce1f329..2edd74cdbd 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -137,16 +137,16 @@ private[remote] class Association( val current = associationState current.uniqueRemoteAddressPromise.trySuccess(peer) current.uniqueRemoteAddressValue() match { - case Some(Success(`peer`)) ⇒ // our value + case Some(`peer`) ⇒ // our value case _ ⇒ val newState = current.newIncarnation(Promise.successful(peer)) if (swapState(current, newState)) { current.uniqueRemoteAddressValue() match { - case Some(Success(old)) ⇒ + case Some(old) ⇒ log.debug( "Incarnation {} of association to [{}] with new UID [{}] (old UID [{}])", newState.incarnation, peer.address, peer.uid, old.uid) - case _ ⇒ + case None ⇒ // Failed, nothing to do } // if swap failed someone else completed before us, and that is fine @@ -206,10 +206,7 @@ private[remote] class Association( // OutboundContext override def quarantine(reason: String): Unit = { - val uid = associationState.uniqueRemoteAddressValue() match { - case Some(Success(a)) ⇒ Some(a.uid) - case _ ⇒ None - } + val uid = associationState.uniqueRemoteAddressValue().map(_.uid) quarantine(reason, uid) } @@ -218,7 +215,7 @@ private[remote] class Association( case Some(u) ⇒ val current = associationState current.uniqueRemoteAddressValue() match { - case Some(Success(peer)) if peer.uid == u ⇒ + case Some(peer) if peer.uid == u ⇒ if (!current.isQuarantined(u)) { val newState = current.newQuarantined() if (swapState(current, newState)) { @@ -233,7 +230,7 @@ private[remote] class Association( } else quarantine(reason, uid) // recursive } - case Some(Success(peer)) ⇒ + case Some(peer) ⇒ log.debug( "Quarantine of [{}] ignored due to non-matching UID, quarantine requested for [{}] but current is [{}]. {}", remoteAddress, u, peer.uid, reason) From a814034342fca2970979a040bd8b6ea48620ef5b Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Sun, 5 Jun 2016 15:40:06 +0200 Subject: [PATCH 065/186] Option value class, to avoid allocations for optional sender --- .../src/main/scala/akka/util/OptionVal.scala | 69 +++++++++++++++++++ .../akka/remote/artery/CodecBenchmark.scala | 8 +-- .../scala/akka/remote/artery/LatchSink.scala | 2 +- .../remote/artery/SendQueueBenchmark.scala | 3 +- .../src/main/scala/akka/remote/Endpoint.scala | 6 +- .../akka/remote/RemoteActorRefProvider.scala | 7 +- .../scala/akka/remote/RemoteTransport.scala | 3 +- .../src/main/scala/akka/remote/Remoting.scala | 5 +- .../akka/remote/artery/ArteryTransport.scala | 14 ++-- .../akka/remote/artery/Association.scala | 10 +-- .../scala/akka/remote/artery/BufferPool.scala | 4 ++ .../scala/akka/remote/artery/Codecs.scala | 53 +++++++------- .../scala/akka/remote/artery/Control.scala | 3 +- .../scala/akka/remote/artery/Handshake.scala | 6 +- .../artery/InboundQuarantineCheck.scala | 5 +- .../remote/artery/MessageDispatcher.scala | 3 +- .../akka/remote/transport/AkkaPduCodec.scala | 45 ++++++------ .../artery/InboundControlJunctionSpec.scala | 4 +- .../remote/artery/InboundHandshakeSpec.scala | 4 +- .../artery/OutboundControlJunctionSpec.scala | 4 +- .../remote/artery/OutboundHandshakeSpec.scala | 4 +- .../artery/SystemMessageDeliverySpec.scala | 7 +- .../akka/remote/artery/TestContext.scala | 9 ++- .../remote/transport/AkkaProtocolSpec.scala | 3 +- project/MiMa.scala | 10 ++- 25 files changed, 191 insertions(+), 100 deletions(-) create mode 100644 akka-actor/src/main/scala/akka/util/OptionVal.scala diff --git a/akka-actor/src/main/scala/akka/util/OptionVal.scala b/akka-actor/src/main/scala/akka/util/OptionVal.scala new file mode 100644 index 0000000000..7238c54233 --- /dev/null +++ b/akka-actor/src/main/scala/akka/util/OptionVal.scala @@ -0,0 +1,69 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.util + +/** + * INTERNAL API + */ +private[akka] object OptionVal { + + def apply[A >: Null](x: A): OptionVal[A] = new OptionVal(x) + + object Some { + def apply[A >: Null](x: A): OptionVal[A] = new OptionVal(x) + def unapply[A >: Null](x: OptionVal[A]): OptionVal[A] = x + } + + /** + * Represents non-existent values, `null` values. + */ + val None = new OptionVal[Null](null) +} + +/** + * INTERNAL API + * Represents optional values similar to `scala.Option`, but + * as a value class to avoid allocations. + * + * Note that it can be used in pattern matching without allocations + * because it has name based extractor using methods `isEmpty` and `get`. + * See http://hseeberger.github.io/blog/2013/10/04/name-based-extractors-in-scala-2-dot-11/ + */ +private[akka] final class OptionVal[+A >: Null](val x: A) extends AnyVal { + + /** + * Returns true if the option is `OptionVal.None`, false otherwise. + */ + def isEmpty: Boolean = + x == null + + /** + * Returns true if the option is `OptionVal.None`, false otherwise. + */ + def isDefined: Boolean = !isEmpty + + /** + * Returns the option's value if the option is nonempty, otherwise + * return `default`. + */ + def getOrElse[B >: A](default: B): B = + if (x == null) default else x + + /** + * Returns the option's value if it is nonempty, or `null` if it is empty. + */ + def orNull[A1 >: A](implicit ev: Null <:< A1): A1 = this getOrElse ev(null) + + /** + * Returns the option's value. + * @note The option must be nonEmpty. + * @throws java.util.NoSuchElementException if the option is empty. + */ + def get: A = + if (x == null) throw new NoSuchElementException("OptionVal.None.get") + else x + + override def toString: String = + if (x == null) "None" else s"Some($x)" +} 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 index 82f22e1c89..6e96859ecf 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -5,13 +5,10 @@ package akka.remote.artery import java.nio.ByteBuffer import java.nio.ByteOrder - import java.util.concurrent.CountDownLatch import java.util.concurrent.TimeUnit - import scala.concurrent.Await import scala.concurrent.duration._ - import akka.NotUsed import akka.actor.ActorSystem import akka.actor.ExtendedActorSystem @@ -28,6 +25,7 @@ import akka.stream.ActorMaterializerSettings import akka.stream.scaladsl._ import com.typesafe.config.ConfigFactory import org.openjdk.jmh.annotations._ +import akka.util.OptionVal @State(Scope.Benchmark) @OutputTimeUnit(TimeUnit.MILLISECONDS) @@ -127,7 +125,7 @@ class CodecBenchmark { Flow.fromGraph(new Encoder(uniqueLocalAddress, system, compression, envelopePool)) Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) - .map(_ ⇒ Send(payload, None, remoteRefB, None)) + .map(_ ⇒ Send(payload, OptionVal.None, remoteRefB, None)) .via(encoder) .map(envelope => envelopePool.release(envelope)) .runWith(new LatchSink(N, latch))(materializer) @@ -193,7 +191,7 @@ class CodecBenchmark { resolveActorRefWithLocalAddress, compression, envelopePool)) Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) - .map(_ ⇒ Send(payload, None, remoteRefB, None)) + .map(_ ⇒ Send(payload, OptionVal.None, remoteRefB, None)) .via(encoder) .via(decoder) .runWith(new LatchSink(N, latch))(materializer) 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 index d66a6814d0..6f1d2d539b 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/LatchSink.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/LatchSink.scala @@ -37,7 +37,7 @@ class LatchSink(countDownAfter: Int, latch: CountDownLatch) extends GraphStage[S } class BarrierSink(countDownAfter: Int, latch: CountDownLatch, barrierAfter: Int, barrier: CyclicBarrier) - extends GraphStage[SinkShape[Any]] { + extends GraphStage[SinkShape[Any]] { val in: Inlet[Any] = Inlet("BarrierSink") override val shape: SinkShape[Any] = SinkShape(in) 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 index f54ca42f0e..2b131923f3 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/SendQueueBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/SendQueueBenchmark.scala @@ -34,7 +34,8 @@ class SendQueueBenchmark { val config = ConfigFactory.parseString( """ - """) + """ + ) implicit val system = ActorSystem("SendQueueBenchmark", config) diff --git a/akka-remote/src/main/scala/akka/remote/Endpoint.scala b/akka-remote/src/main/scala/akka/remote/Endpoint.scala index c92f1b1032..1f65987741 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/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index f8ca7a5e34..119b406d8f 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -11,15 +11,14 @@ 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.{ 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 /** * INTERNAL API @@ -528,13 +527,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 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 } override def provider: RemoteActorRefProvider = remote.provider diff --git a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala index 0797e2d2cf..c865aa508d 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 diff --git a/akka-remote/src/main/scala/akka/remote/Remoting.scala b/akka-remote/src/main/scala/akka/remote/Remoting.scala index c742b58aa6..8a35a63126 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 @@ -209,7 +210,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) } @@ -249,7 +250,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/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index a48d91e28a..1e61bc2468 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -77,6 +77,7 @@ import akka.actor.Cancellable import scala.collection.JavaConverters._ import akka.stream.ActorMaterializerSettings import scala.annotation.tailrec +import akka.util.OptionVal /** * INTERNAL API */ @@ -84,7 +85,7 @@ private[akka] final case class InboundEnvelope( recipient: InternalActorRef, recipientAddress: Address, message: AnyRef, - senderOption: Option[ActorRef], + senderOption: OptionVal[ActorRef], originUid: Long) /** @@ -111,11 +112,10 @@ private[akka] trait InboundContext { /** * Lookup the outbound association for a given UID. - * Will return `null` if the UID is unknown, i.e. - * handshake not completed. `null` is used instead of `Optional` - * to avoid allocations. + * Will return `OptionVal.None` if the UID is unknown, i.e. + * handshake not completed. */ - def association(uid: Long): OutboundContext + def association(uid: Long): OptionVal[OutboundContext] def completeHandshake(peer: UniqueAddress): Unit @@ -603,7 +603,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R override def sendControl(to: Address, message: ControlMessage) = association(to).sendControl(message) - override def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = { + override def send(message: Any, senderOption: OptionVal[ActorRef], recipient: RemoteActorRef): Unit = { val cached = recipient.cachedAssociation val a = @@ -620,7 +620,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R override def association(remoteAddress: Address): Association = associationRegistry.association(remoteAddress) - override def association(uid: Long): Association = + override def association(uid: Long): OptionVal[Association] = associationRegistry.association(uid) override def completeHandshake(peer: UniqueAddress): Unit = { diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 2edd74cdbd..b46a479b76 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -4,7 +4,6 @@ package akka.remote.artery import java.util.Queue - import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.CountDownLatch @@ -36,6 +35,7 @@ import akka.stream.scaladsl.Keep import akka.stream.scaladsl.Source import akka.util.{ Unsafe, WildcardTree } import org.agrona.concurrent.ManyToOneConcurrentArrayQueue +import akka.util.OptionVal /** * INTERNAL API @@ -158,7 +158,7 @@ private[remote] class Association( override def sendControl(message: ControlMessage): Unit = outboundControlIngress.sendControlMessage(message) - def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit = { + def send(message: Any, senderOption: OptionVal[ActorRef], recipient: RemoteActorRef): Unit = { // allow ActorSelectionMessage to pass through quarantine, to be able to establish interaction with new system // FIXME where is that ActorSelectionMessage check in old remoting? if (message.isInstanceOf[ActorSelectionMessage] || !associationState.isQuarantined() || message == ClearSystemMessageDelivery) { @@ -224,7 +224,7 @@ private[remote] class Association( "Association to [{}] with UID [{}] is irrecoverably failed. Quarantining address. {}", remoteAddress, u, reason) // end delivery of system messages to that incarnation after this point - send(ClearSystemMessageDelivery, None, dummyRecipient) + send(ClearSystemMessageDelivery, OptionVal.None, dummyRecipient) // try to tell the other system that we have quarantined it sendControl(Quarantined(localAddress, peer)) } else @@ -408,8 +408,8 @@ private[remote] class AssociationRegistry(createAssociation: Address ⇒ Associa } } - def association(uid: Long): Association = - associationsByUid.get(uid) + def association(uid: Long): OptionVal[Association] = + OptionVal(associationsByUid.get(uid)) def setUID(peer: UniqueAddress): Association = { val a = association(peer.address) diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index 450099c99f..8842d548b6 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -97,6 +97,7 @@ sealed trait HeaderBuilder { def senderActorRef: String def setNoSender(): Unit + def isNoSender: Boolean def recipientActorRef_=(ref: String): Unit def recipientActorRef: String @@ -135,6 +136,9 @@ private[remote] final class HeaderBuilderImpl(val compressionTable: LiteralCompr _senderActorRefIdx = EnvelopeBuffer.DeadLettersCode } + def isNoSender: Boolean = + (_senderActorRef eq null) && _senderActorRefIdx == EnvelopeBuffer.DeadLettersCode + def senderActorRef: String = { if (_senderActorRef ne null) _senderActorRef else { diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index eb6470e73b..299cd7e5fd 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -1,7 +1,6 @@ package akka.remote.artery import scala.util.control.NonFatal - import akka.actor.{ ActorRef, InternalActorRef } import akka.actor.ActorSystem import akka.actor.ExtendedActorSystem @@ -11,13 +10,14 @@ 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.OptionVal // TODO: Long UID class Encoder( uniqueLocalAddress: UniqueAddress, - system: ActorSystem, - compressionTable: LiteralCompressionTable, - pool: EnvelopeBufferPool) + system: ActorSystem, + compressionTable: LiteralCompressionTable, + pool: EnvelopeBufferPool) extends GraphStage[FlowShape[Send, EnvelopeBuffer]] { val in: Inlet[Send] = Inlet("Artery.Encoder.in") @@ -34,7 +34,6 @@ class Encoder( private val serialization = SerializationExtension(system) private val serializationInfo = Serialization.Information(localAddress, system) - private val noSender = system.deadLetters.path.toSerializationFormatWithAddress(localAddress) private val senderCache = new java.util.HashMap[ActorRef, String] private var recipientCache = new java.util.HashMap[ActorRef, String] @@ -57,7 +56,8 @@ class Encoder( headerBuilder.recipientActorRef = recipientStr send.senderOption match { - case Some(sender) ⇒ + case OptionVal.None => headerBuilder.setNoSender() + case OptionVal.Some(sender) => val senderStr = senderCache.get(sender) match { case null ⇒ val s = sender.path.toSerializationFormatWithAddress(localAddress) @@ -69,9 +69,6 @@ class Encoder( case s ⇒ s } headerBuilder.senderActorRef = senderStr - case None ⇒ - //headerBuilder.setNoSender() - headerBuilder.senderActorRef = noSender } try { @@ -108,11 +105,11 @@ class Encoder( } class Decoder( - uniqueLocalAddress: UniqueAddress, - system: ExtendedActorSystem, + uniqueLocalAddress: UniqueAddress, + system: ExtendedActorSystem, resolveActorRefWithLocalAddress: String ⇒ InternalActorRef, - compressionTable: LiteralCompressionTable, - pool: EnvelopeBufferPool) extends GraphStage[FlowShape[EnvelopeBuffer, InboundEnvelope]] { + compressionTable: LiteralCompressionTable, + pool: EnvelopeBufferPool) extends GraphStage[FlowShape[EnvelopeBuffer, InboundEnvelope]] { val in: Inlet[EnvelopeBuffer] = Inlet("Artery.Decoder.in") val out: Outlet[InboundEnvelope] = Outlet("Artery.Decoder.out") val shape: FlowShape[EnvelopeBuffer, InboundEnvelope] = FlowShape(in, out) @@ -124,7 +121,7 @@ class Decoder( private val serialization = SerializationExtension(system) private val recipientCache = new java.util.HashMap[String, InternalActorRef] - private val senderCache = new java.util.HashMap[String, Option[ActorRef]] + private val senderCache = new java.util.HashMap[String, ActorRef] override protected def logSource = classOf[Decoder] @@ -146,17 +143,21 @@ class Decoder( case ref ⇒ ref } - val senderOption: Option[ActorRef] = senderCache.get(headerBuilder.senderActorRef) match { - case null ⇒ - val ref = resolveActorRefWithLocalAddress(headerBuilder.senderActorRef) - // FIXME this cache will be replaced by compression table - if (senderCache.size() >= 1000) - senderCache.clear() - val refOpt = Some(ref) - senderCache.put(headerBuilder.senderActorRef, refOpt) - refOpt - case refOpt ⇒ refOpt - } + val senderOption = + if (headerBuilder.isNoSender) + OptionVal.None + else { + senderCache.get(headerBuilder.senderActorRef) match { + case null ⇒ + val ref = resolveActorRefWithLocalAddress(headerBuilder.senderActorRef) + // FIXME this cache will be replaced by compression table + if (senderCache.size() >= 1000) + senderCache.clear() + senderCache.put(headerBuilder.senderActorRef, ref) + OptionVal(ref) + case ref ⇒ OptionVal(ref) + } + } try { val deserializedMessage = MessageSerializer.deserializeForArtery( @@ -166,7 +167,7 @@ class Decoder( recipient, localAddress, // FIXME: Is this needed anymore? What should we do here? deserializedMessage, - senderOption, // FIXME: No need for an option, decode simply to deadLetters instead + senderOption, headerBuilder.uid) push(out, decoded) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Control.scala b/akka-remote/src/main/scala/akka/remote/artery/Control.scala index 7d32d0889b..6d68ee0825 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Control.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Control.scala @@ -18,6 +18,7 @@ import akka.stream.stage.GraphStageWithMaterializedValue import akka.stream.stage.InHandler import akka.stream.stage.OutHandler import akka.remote.UniqueAddress +import akka.util.OptionVal /** * INTERNAL API: Marker trait for reply messages @@ -197,7 +198,7 @@ private[akka] class OutboundControlJunction(outboundContext: OutboundContext) } private def wrap(message: ControlMessage): Send = - Send(message, None, outboundContext.dummyRecipient, None) + Send(message, OptionVal.None, outboundContext.dummyRecipient, None) setHandlers(in, out, this) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala index d1f96ce91b..84dd9871f1 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -5,7 +5,6 @@ package akka.remote.artery import scala.concurrent.duration._ import scala.util.control.NoStackTrace - import akka.remote.EndpointManager.Send import akka.remote.UniqueAddress import akka.stream.Attributes @@ -17,6 +16,7 @@ import akka.stream.stage.GraphStageLogic import akka.stream.stage.InHandler import akka.stream.stage.OutHandler import akka.stream.stage.TimerGraphStageLogic +import akka.util.OptionVal /** * INTERNAL API @@ -123,7 +123,7 @@ private[akka] class OutboundHandshake(outboundContext: OutboundContext, timeout: private def pushHandshakeReq(): Unit = { injectHandshakeTickScheduled = true scheduleOnce(InjectHandshakeTick, injectHandshakeInterval) - push(out, Send(HandshakeReq(outboundContext.localAddress), None, outboundContext.dummyRecipient, None)) + push(out, Send(HandshakeReq(outboundContext.localAddress), OptionVal.None, outboundContext.dummyRecipient, None)) } private def handshakeCompleted(): Unit = { @@ -215,7 +215,7 @@ private[akka] class InboundHandshake(inboundContext: InboundContext, inControlSt private def isKnownOrigin(originUid: Long): Boolean = { // FIXME these association lookups are probably too costly for each message, need local cache or something - (inboundContext.association(originUid) ne null) + inboundContext.association(originUid).isDefined } // OutHandler diff --git a/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala b/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala index 531e9c4aff..1a4454c58b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala @@ -12,6 +12,7 @@ import akka.stream.stage.GraphStageLogic import akka.stream.stage.InHandler import akka.stream.stage.OutHandler import akka.remote.UniqueAddress +import akka.util.OptionVal /** * INTERNAL API @@ -28,10 +29,10 @@ private[akka] class InboundQuarantineCheck(inboundContext: InboundContext) exten override def onPush(): Unit = { val env = grab(in) inboundContext.association(env.originUid) match { - case null ⇒ + case OptionVal.None => // unknown, handshake not completed push(out, env) - case association ⇒ + case OptionVal.Some(association) => if (association.associationState.isQuarantined(env.originUid)) { inboundContext.sendControl( association.remoteAddress, diff --git a/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala b/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala index 8e871c3f33..7525473479 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala @@ -16,6 +16,7 @@ import akka.dispatch.sysmsg.SystemMessage import akka.event.Logging import akka.remote.RemoteActorRefProvider import akka.remote.RemoteRef +import akka.util.OptionVal /** * INTERNAL API @@ -31,7 +32,7 @@ private[akka] class MessageDispatcher( recipient: InternalActorRef, recipientAddress: Address, message: AnyRef, - senderOption: Option[ActorRef]): Unit = { + senderOption: OptionVal[ActorRef]): Unit = { import provider.remoteSettings._ 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..b0c35bc1f4 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 @@ -35,11 +36,11 @@ private[remote] object AkkaPduCodec { final case class Payload(bytes: ByteString) extends AkkaPdu final case class Message( - recipient: InternalActorRef, - recipientAddress: Address, + recipient: InternalActorRef, + recipientAddress: Address, serializedMessage: SerializedMessage, - senderOption: Option[ActorRef], - seqOption: Option[SeqNo]) extends HasSequenceNumber { + senderOption: OptionVal[ActorRef], + seqOption: Option[SeqNo]) extends HasSequenceNumber { def reliableDeliveryEnabled = seqOption.isDefined @@ -94,12 +95,12 @@ private[remote] trait AkkaPduCodec { def decodeMessage(raw: ByteString, provider: RemoteActorRefProvider, localAddress: Address): (Option[Ack], Option[Message]) def constructMessage( - localAddress: Address, - recipient: ActorRef, + 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 } @@ -118,19 +119,23 @@ private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec { } override def constructMessage( - localAddress: Address, - recipient: ActorRef, + 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) @@ -176,8 +181,8 @@ private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec { } override def decodeMessage( - raw: ByteString, - provider: RemoteActorRefProvider, + raw: ByteString, + provider: RemoteActorRefProvider, localAddress: Address): (Option[Ack], Option[Message]) = { val ackAndEnvelope = AckAndEnvelopeContainer.parseFrom(raw.toArray) @@ -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 @@ -226,7 +231,7 @@ private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec { Address(encodedAddress.getProtocol, encodedAddress.getSystem, encodedAddress.getHostname, encodedAddress.getPort) private def constructControlMessagePdu( - code: WireFormats.CommandType, + code: WireFormats.CommandType, handshakeInfo: Option[AkkaHandshakeInfo.Builder]): ByteString = { val controlMessageBuilder = AkkaControlMessage.newBuilder() diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala index 0981ade341..efa0a10b54 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala @@ -4,7 +4,6 @@ package akka.remote.artery import scala.concurrent.duration._ - import akka.actor.Address import akka.actor.InternalActorRef import akka.remote.UniqueAddress @@ -18,6 +17,7 @@ import akka.stream.testkit.scaladsl.TestSource import akka.testkit.AkkaSpec import akka.testkit.ImplicitSender import akka.testkit.TestProbe +import akka.util.OptionVal object InboundControlJunctionSpec { case object Control1 extends ControlMessage @@ -42,7 +42,7 @@ class InboundControlJunctionSpec extends AkkaSpec with ImplicitSender { val recipient = null.asInstanceOf[InternalActorRef] // not used val ((upstream, controlSubject), downstream) = TestSource.probe[AnyRef] - .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, None, addressA.uid)) + .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, OptionVal.None, addressA.uid)) .viaMat(new InboundControlJunction)(Keep.both) .map { case InboundEnvelope(_, _, msg, _, _) ⇒ msg } .toMat(TestSink.probe[Any])(Keep.both) diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala index 4a1774a7bc..5aa42f93ce 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala @@ -5,7 +5,6 @@ package akka.remote.artery import scala.concurrent.Await import scala.concurrent.duration._ - import akka.actor.Address import akka.actor.InternalActorRef import akka.remote.UniqueAddress @@ -22,6 +21,7 @@ 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 @@ -41,7 +41,7 @@ class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { private def setupStream(inboundContext: InboundContext, timeout: FiniteDuration = 5.seconds): (TestPublisher.Probe[AnyRef], TestSubscriber.Probe[Any]) = { val recipient = null.asInstanceOf[InternalActorRef] // not used TestSource.probe[AnyRef] - .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, None, addressA.uid)) + .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, OptionVal.None, addressA.uid)) .via(new InboundHandshake(inboundContext, inControlStream = true)) .map { case InboundEnvelope(_, _, msg, _, _) ⇒ msg } .toMat(TestSink.probe[Any])(Keep.both) diff --git a/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala index 206a355ece..7b79fdaced 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala @@ -4,7 +4,6 @@ package akka.remote.artery import scala.concurrent.duration._ - import akka.actor.Address import akka.remote.EndpointManager.Send import akka.remote.RemoteActorRef @@ -17,6 +16,7 @@ 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 @@ -41,7 +41,7 @@ class OutboundControlJunctionSpec extends AkkaSpec with ImplicitSender { val destination = null.asInstanceOf[RemoteActorRef] // not used val ((upstream, controlIngress), downstream) = TestSource.probe[String] - .map(msg ⇒ Send(msg, None, destination, None)) + .map(msg ⇒ Send(msg, OptionVal.None, destination, None)) .viaMat(new OutboundControlJunction(outboundContext))(Keep.both) .map { case Send(msg, _, _, _) ⇒ msg } .toMat(TestSink.probe[Any])(Keep.both) diff --git a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala index 27bbd3e13b..1b3adef27d 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala @@ -4,7 +4,6 @@ package akka.remote.artery import scala.concurrent.duration._ - import akka.actor.Address import akka.remote.EndpointManager.Send import akka.remote.RemoteActorRef @@ -22,6 +21,7 @@ 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 { @@ -38,7 +38,7 @@ class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { val destination = null.asInstanceOf[RemoteActorRef] // not used TestSource.probe[String] - .map(msg ⇒ Send(msg, None, destination, None)) + .map(msg ⇒ Send(msg, OptionVal.None, destination, None)) .via(new OutboundHandshake(outboundContext, timeout, retryInterval, injectHandshakeInterval)) .map { case Send(msg, _, _, _) ⇒ msg } .toMat(TestSink.probe[Any])(Keep.both) diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index f9173f04e3..a6d309ebf2 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -4,10 +4,8 @@ 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 @@ -33,6 +31,7 @@ import akka.testkit.ImplicitSender import akka.testkit.TestActors import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory +import akka.util.OptionVal object SystemMessageDeliverySpec { @@ -68,7 +67,7 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi private def send(sendCount: Int, resendInterval: FiniteDuration, outboundContext: OutboundContext): Source[Send, NotUsed] = { val remoteRef = null.asInstanceOf[RemoteActorRef] // not used Source(1 to sendCount) - .map(n ⇒ Send("msg-" + n, None, remoteRef, None)) + .map(n ⇒ Send("msg-" + n, OptionVal.None, remoteRef, None)) .via(new SystemMessageDelivery(outboundContext, resendInterval, maxBufferSize = 1000)) } @@ -77,7 +76,7 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi Flow[Send] .map { case Send(sysEnv: SystemMessageEnvelope, _, _, _) ⇒ - InboundEnvelope(recipient, addressB.address, sysEnv, None, addressA.uid) + InboundEnvelope(recipient, addressB.address, sysEnv, OptionVal.None, addressA.uid) } .async .via(new SystemMessageAcker(inboundContext)) diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala index bbc9cefaa2..0ec5454ab8 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -6,11 +6,9 @@ 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 @@ -18,6 +16,7 @@ import akka.remote.RemoteActorRef import akka.remote.UniqueAddress import akka.remote.artery.InboundControlJunction.ControlMessageObserver import akka.remote.artery.InboundControlJunction.ControlMessageSubject +import akka.util.OptionVal private[akka] class TestInboundContext( override val localAddress: UniqueAddress, @@ -44,8 +43,8 @@ private[akka] class TestInboundContext( case existing ⇒ existing } - override def association(uid: Long): OutboundContext = - associationsByUid.get(uid) + override def association(uid: Long): OptionVal[OutboundContext] = + OptionVal(associationsByUid.get(uid)) override def completeHandshake(peer: UniqueAddress): Unit = { val a = association(peer.address).asInstanceOf[TestOutboundContext] @@ -85,7 +84,7 @@ private[akka] class TestOutboundContext( override def sendControl(message: ControlMessage) = { controlProbe.foreach(_ ! message) - controlSubject.sendControl(InboundEnvelope(null, remoteAddress, message, None, localAddress.uid)) + controlSubject.sendControl(InboundEnvelope(null, remoteAddress, message, OptionVal.None, localAddress.uid)) } // FIXME we should be able to Send without a recipient ActorRef 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..3825bba924 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 { @@ -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/project/MiMa.scala b/project/MiMa.scala index 0a96b84a24..a375eec6ba 100644 --- a/project/MiMa.scala +++ b/project/MiMa.scala @@ -851,7 +851,15 @@ object MiMa extends AutoPlugin { // 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"), + 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"), // internal api FilterAnyProblemStartingWith("akka.stream.impl"), From 2e0986254c3cd25e32c1dbb08a8d8dc79a8f0168 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Sun, 5 Jun 2016 19:11:52 +0200 Subject: [PATCH 066/186] improve the test somewhat --- .../akka/remote/artery/LatencySpec.scala | 2 + .../remote/artery/MaxThroughputSpec.scala | 70 +++++++++++++++---- 2 files changed, 59 insertions(+), 13 deletions(-) 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 index fa0ad212ad..602b4b5edc 100644 --- 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 @@ -43,6 +43,8 @@ object LatencySpec extends MultiNodeConfig { akka.test.LatencySpec.repeatCount = 1 akka { loglevel = ERROR + # avoid TestEventListener + loggers = ["akka.event.Logging$$DefaultLogger"] testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s actor { provider = "akka.remote.RemoteActorRefProvider" 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 index 77e0039614..f7c5dc823d 100644 --- 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 @@ -3,18 +3,22 @@ */ 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.RemoteActorRefProvider import akka.remote.testconductor.RoleName import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.STMultiNodeSpec +import akka.serialization.ByteBufferSerializer +import akka.serialization.SerializerWithStringManifest import akka.testkit._ import com.typesafe.config.ConfigFactory -import java.net.InetAddress object MaxThroughputSpec extends MultiNodeConfig { val first = role("first") @@ -28,11 +32,20 @@ object MaxThroughputSpec extends MultiNodeConfig { akka.test.MaxThroughputSpec.totalMessagesFactor = 1.0 akka { loglevel = ERROR + # avoid TestEventListener + loggers = ["akka.event.Logging$$DefaultLogger"] testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s actor { provider = "akka.remote.RemoteActorRefProvider" serialize-creators = false serialize-messages = false + + serializers { + test = "akka.remote.artery.MaxThroughputSpec$$TestSerializer" + } + serialization-bindings { + "akka.remote.artery.MaxThroughputSpec$$FlowControl" = test + } } remote.artery { enabled = on @@ -57,9 +70,13 @@ object MaxThroughputSpec extends MultiNodeConfig { Props(new Receiver(reporter, payloadSize)).withDispatcher("akka.remote.default-remote-dispatcher") class Receiver(reporter: RateReporter, payloadSize: Int) extends Actor { - var c = 0L + private var c = 0L def receive = { + case msg: Array[Byte] ⇒ + if (msg.length != payloadSize) throw new IllegalArgumentException("Invalid message") + reporter.onMessage(1, payloadSize) + c += 1 case Start ⇒ c = 0 sender() ! Start @@ -68,10 +85,7 @@ object MaxThroughputSpec extends MultiNodeConfig { context.stop(self) case m: Echo ⇒ sender() ! m - case msg: Array[Byte] ⇒ - if (msg.length != payloadSize) throw new IllegalArgumentException("Invalid message") - reporter.onMessage(1, payloadSize) - c += 1 + } } @@ -117,14 +131,14 @@ object MaxThroughputSpec extends MultiNodeConfig { val throughput = (totalReceived * 1000.0 / took) println( s"=== MaxThroughput ${self.path.name}: " + - f"throughput ${throughput}%,.0f msg/s, " + - f"${throughput * payloadSize}%,.0f bytes/s, " + + f"throughput ${throughput * testSettings.senderReceiverPairs}%,.0f msg/s, " + + f"${throughput * payloadSize * testSettings.senderReceiverPairs}%,.0f bytes/s, " + s"dropped ${totalMessages - totalReceived}, " + s"max round-trip $maxRoundTripMillis ms, " + s"burst size $burstSize, " + s"payload size $payloadSize, " + s"$took ms to deliver $totalReceived messages") - plotRef ! PlotResult().add(testName, throughput * payloadSize / 1024 / 1024) + plotRef ! PlotResult().add(testName, throughput * payloadSize * testSettings.senderReceiverPairs / 1024 / 1024) context.stop(self) } @@ -132,7 +146,8 @@ object MaxThroughputSpec extends MultiNodeConfig { val batchSize = math.min(remaining, burstSize) var i = 0 while (i < batchSize) { - target ! payload + // target ! payload + target.tell(payload, ActorRef.noSender) i += 1 } remaining -= batchSize @@ -153,6 +168,31 @@ object MaxThroughputSpec extends MultiNodeConfig { payloadSize: Int, senderReceiverPairs: Int) + 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] = ??? + override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef = ??? + } + } class MaxThroughputSpecMultiJvmNode1 extends MaxThroughputSpec @@ -195,6 +235,12 @@ abstract class MaxThroughputSpec } val scenarios = List( + TestSettings( + testName = "warmup", + totalMessages = adjustedTotalMessages(20000), + burstSize = 1000, + payloadSize = 100, + senderReceiverPairs = 1), TestSettings( testName = "1-to-1", totalMessages = adjustedTotalMessages(20000), @@ -216,7 +262,7 @@ abstract class MaxThroughputSpec TestSettings( testName = "5-to-5", totalMessages = adjustedTotalMessages(20000), - burstSize = 1000, + burstSize = 200, // don't exceed the send queue capacity 200*5*3=3000 payloadSize = 100, senderReceiverPairs = 5)) @@ -269,7 +315,5 @@ abstract class MaxThroughputSpec s"be great for ${s.testName}, burstSize = ${s.burstSize}, payloadSize = ${s.payloadSize}" in test(s) } - // TODO add more tests, such as 5-to-5 sender receiver pairs - } } From 64a61d5a4cbbe3e62688b1e9e91312500ccf4b92 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 6 Jun 2016 08:26:15 +0200 Subject: [PATCH 067/186] object pool for InboundEnvelope --- .../akka/remote/artery/CodecBenchmark.scala | 16 ++- .../akka/remote/artery/ArteryTransport.scala | 102 +++++++++++++++--- .../scala/akka/remote/artery/Codecs.scala | 28 ++--- .../scala/akka/remote/artery/Control.scala | 2 +- .../scala/akka/remote/artery/Handshake.scala | 20 ++-- .../artery/InboundQuarantineCheck.scala | 4 +- .../scala/akka/remote/artery/ObjectPool.scala | 24 +++++ .../remote/artery/SystemMessageDelivery.scala | 10 +- .../scala/akka/remote/artery/TestStage.scala | 6 +- .../akka/remote/transport/AkkaPduCodec.scala | 38 +++---- .../artery/InboundControlJunctionSpec.scala | 2 +- .../remote/artery/InboundHandshakeSpec.scala | 2 +- .../akka/remote/artery/TestContext.scala | 1 + 13 files changed, 182 insertions(+), 73 deletions(-) create mode 100644 akka-remote/src/main/scala/akka/remote/artery/ObjectPool.scala 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 index 6e96859ecf..f573f98bc4 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -51,6 +51,10 @@ class CodecBenchmark { val systemB = ActorSystem("systemB", system.settings.config) val envelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) + val inboundEnvelopePool = new ObjectPool[InboundEnvelope]( + 16, + create = () ⇒ new ReusableInboundEnvelope, clear = inEnvelope ⇒ inEnvelope.asInstanceOf[ReusableInboundEnvelope].clear() + ) val compression = new Compression(system) val headerIn = HeaderBuilder(compression) val envelopeTemplateBuffer = ByteBuffer.allocate(ArteryTransport.MaximumFrameSize).order(ByteOrder.LITTLE_ENDIAN) @@ -151,7 +155,7 @@ class CodecBenchmark { val decoder: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = Flow.fromGraph(new Decoder(uniqueLocalAddress, system.asInstanceOf[ExtendedActorSystem], - resolveActorRefWithLocalAddress, compression, envelopePool)) + resolveActorRefWithLocalAddress, compression, envelopePool, inboundEnvelopePool)) Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) .map { _ => @@ -162,6 +166,10 @@ class CodecBenchmark { envelope } .via(decoder) + .map { env => + inboundEnvelopePool.release(env) + () + } .runWith(new LatchSink(N, latch))(materializer) if (!latch.await(30, TimeUnit.SECONDS)) @@ -188,12 +196,16 @@ class CodecBenchmark { val decoder: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = Flow.fromGraph(new Decoder(uniqueLocalAddress, system.asInstanceOf[ExtendedActorSystem], - resolveActorRefWithLocalAddress, compression, envelopePool)) + resolveActorRefWithLocalAddress, compression, envelopePool, inboundEnvelopePool)) Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) .map(_ ⇒ Send(payload, OptionVal.None, remoteRefB, None)) .via(encoder) .via(decoder) + .map { env => + inboundEnvelopePool.release(env) + () + } .runWith(new LatchSink(N, latch))(materializer) if (!latch.await(30, TimeUnit.SECONDS)) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 1e61bc2468..68d05736bb 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -78,15 +78,82 @@ import scala.collection.JavaConverters._ import akka.stream.ActorMaterializerSettings import scala.annotation.tailrec import akka.util.OptionVal + /** * INTERNAL API */ -private[akka] final case class InboundEnvelope( - recipient: InternalActorRef, - recipientAddress: Address, - message: AnyRef, - senderOption: OptionVal[ActorRef], - originUid: Long) +private[akka] object InboundEnvelope { + def apply( + recipient: InternalActorRef, + recipientAddress: Address, + message: AnyRef, + senderOption: OptionVal[ActorRef], + originUid: Long): InboundEnvelope = { + val env = new ReusableInboundEnvelope + env.init(recipient, recipientAddress, message, senderOption, originUid) + env + } + +} + +/** + * INTERNAL API + */ +private[akka] trait InboundEnvelope { + def recipient: InternalActorRef + def recipientAddress: Address + def message: AnyRef + def senderOption: OptionVal[ActorRef] + def originUid: Long + + def withMessage(message: AnyRef): InboundEnvelope +} + +/** + * INTERNAL API + */ +private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { + private var _recipient: InternalActorRef = null + private var _recipientAddress: Address = null + private var _message: AnyRef = null + private var _senderOption: OptionVal[ActorRef] = OptionVal.None + private var _originUid: Long = 0L + + override def recipient: InternalActorRef = _recipient + override def recipientAddress: Address = _recipientAddress + override def message: AnyRef = _message + override def senderOption: OptionVal[ActorRef] = _senderOption + override def originUid: Long = _originUid + + override def withMessage(message: AnyRef): InboundEnvelope = { + _message = message + this + } + + def clear(): Unit = { + _recipient = null + _recipientAddress = null + _message = null + _senderOption = OptionVal.None + _originUid = 0L + } + + def init( + recipient: InternalActorRef, + recipientAddress: Address, + message: AnyRef, + senderOption: OptionVal[ActorRef], + originUid: Long): Unit = { + _recipient = recipient + _recipientAddress = recipientAddress + _message = message + _senderOption = senderOption + _originUid = originUid + } + + override def toString: String = + s"InboundEnvelope($recipient, $recipientAddress, $message, $senderOption, $originUid)" +} /** * INTERNAL API @@ -316,8 +383,12 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val maxRestarts = 5 // FIXME config private val restartCounter = new RestartCounter(maxRestarts, restartTimeout) - val envelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) - val largeEnvelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumLargeFrameSize, ArteryTransport.MaximumPooledBuffers) + private val envelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) + private val largeEnvelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumLargeFrameSize, ArteryTransport.MaximumPooledBuffers) + + private val inboundEnvelopePool = new ObjectPool[InboundEnvelope]( + 16, + create = () ⇒ new ReusableInboundEnvelope, clear = inEnvelope ⇒ inEnvelope.asInstanceOf[ReusableInboundEnvelope].clear()) val (afrFileChannel, afrFlie, flightRecorder) = initializeFlightRecorder() @@ -532,11 +603,6 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .run()(materializer) } - aeronSource(largeStreamId, largeEnvelopePool) - .via(inboundLargeFlow) - .toMat(inboundSink)(Keep.right) - .run()(materializer) - attachStreamRestart("Inbound large message stream", completed, () ⇒ runInboundLargeMessagesStream()) } @@ -664,8 +730,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // FIXME we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages } - def createEncoder(pool: EnvelopeBufferPool): Flow[Send, EnvelopeBuffer, NotUsed] = - Flow.fromGraph(new Encoder(localAddress, system, compression, pool)) + def createEncoder(bufferPool: EnvelopeBufferPool): Flow[Send, EnvelopeBuffer, NotUsed] = + Flow.fromGraph(new Encoder(localAddress, system, compression, bufferPool)) def encoder: Flow[Send, EnvelopeBuffer, NotUsed] = createEncoder(envelopePool) @@ -675,12 +741,14 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R val messageDispatcherSink: Sink[InboundEnvelope, Future[Done]] = Sink.foreach[InboundEnvelope] { m ⇒ messageDispatcher.dispatch(m.recipient, m.recipientAddress, m.message, m.senderOption) + inboundEnvelopePool.release(m) } - def createDecoder(pool: EnvelopeBufferPool): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { + def createDecoder(bufferPool: EnvelopeBufferPool): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { val resolveActorRefWithLocalAddress: String ⇒ InternalActorRef = recipient ⇒ provider.resolveActorRefWithLocalAddress(recipient, localAddress.address) - Flow.fromGraph(new Decoder(localAddress, system, resolveActorRefWithLocalAddress, compression, pool)) + Flow.fromGraph(new Decoder(localAddress, system, resolveActorRefWithLocalAddress, compression, bufferPool, + inboundEnvelopePool)) } def decoder: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = createDecoder(envelopePool) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 299cd7e5fd..47df0a7efc 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -15,9 +15,9 @@ import akka.util.OptionVal // TODO: Long UID class Encoder( uniqueLocalAddress: UniqueAddress, - system: ActorSystem, - compressionTable: LiteralCompressionTable, - pool: EnvelopeBufferPool) + system: ActorSystem, + compressionTable: LiteralCompressionTable, + bufferPool: EnvelopeBufferPool) extends GraphStage[FlowShape[Send, EnvelopeBuffer]] { val in: Inlet[Send] = Inlet("Artery.Encoder.in") @@ -41,7 +41,7 @@ class Encoder( override def onPush(): Unit = { val send = grab(in) - val envelope = pool.acquire() + val envelope = bufferPool.acquire() val recipientStr = recipientCache.get(send.recipient) match { case null ⇒ @@ -56,8 +56,8 @@ class Encoder( headerBuilder.recipientActorRef = recipientStr send.senderOption match { - case OptionVal.None => headerBuilder.setNoSender() - case OptionVal.Some(sender) => + case OptionVal.None ⇒ headerBuilder.setNoSender() + case OptionVal.Some(sender) ⇒ val senderStr = senderCache.get(sender) match { case null ⇒ val s = sender.path.toSerializationFormatWithAddress(localAddress) @@ -85,7 +85,7 @@ class Encoder( } catch { case NonFatal(e) ⇒ - pool.release(envelope) + bufferPool.release(envelope) send.message match { case _: SystemMessageEnvelope ⇒ log.error(e, "Failed to serialize system message [{}].", send.message.getClass.getName) @@ -105,11 +105,12 @@ class Encoder( } class Decoder( - uniqueLocalAddress: UniqueAddress, - system: ExtendedActorSystem, + uniqueLocalAddress: UniqueAddress, + system: ExtendedActorSystem, resolveActorRefWithLocalAddress: String ⇒ InternalActorRef, - compressionTable: LiteralCompressionTable, - pool: EnvelopeBufferPool) extends GraphStage[FlowShape[EnvelopeBuffer, InboundEnvelope]] { + compressionTable: LiteralCompressionTable, + bufferPool: EnvelopeBufferPool, + inEnvelopePool: ObjectPool[InboundEnvelope]) extends GraphStage[FlowShape[EnvelopeBuffer, InboundEnvelope]] { val in: Inlet[EnvelopeBuffer] = Inlet("Artery.Decoder.in") val out: Outlet[InboundEnvelope] = Outlet("Artery.Decoder.out") val shape: FlowShape[EnvelopeBuffer, InboundEnvelope] = FlowShape(in, out) @@ -163,7 +164,8 @@ class Decoder( val deserializedMessage = MessageSerializer.deserializeForArtery( system, serialization, headerBuilder, envelope) - val decoded = InboundEnvelope( + val decoded = inEnvelopePool.acquire() + decoded.asInstanceOf[ReusableInboundEnvelope].init( recipient, localAddress, // FIXME: Is this needed anymore? What should we do here? deserializedMessage, @@ -178,7 +180,7 @@ class Decoder( headerBuilder.serializer, headerBuilder.manifest, e.getMessage) pull(in) } finally { - pool.release(envelope) + bufferPool.release(envelope) } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Control.scala b/akka-remote/src/main/scala/akka/remote/artery/Control.scala index 6d68ee0825..c88d63a43a 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Control.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Control.scala @@ -105,7 +105,7 @@ private[akka] class InboundControlJunction // InHandler override def onPush(): Unit = { grab(in) match { - case env @ InboundEnvelope(_, _, _: ControlMessage, _, _) ⇒ + case env: InboundEnvelope if env.message.isInstanceOf[ControlMessage] ⇒ observers.foreach(_.notify(env)) pull(in) case env ⇒ diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala index 84dd9871f1..93d8fad3ec 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -166,23 +166,25 @@ private[akka] class InboundHandshake(inboundContext: InboundContext, inControlSt if (inControlStream) setHandler(in, new InHandler { override def onPush(): Unit = { - grab(in) match { - case InboundEnvelope(_, _, HandshakeReq(from), _, _) ⇒ - onHandshakeReq(from) - case InboundEnvelope(_, _, HandshakeRsp(from), _, _) ⇒ + val env = grab(in) + env.message match { + case HandshakeReq(from) ⇒ onHandshakeReq(from) + case HandshakeRsp(from) ⇒ inboundContext.completeHandshake(from) pull(in) - case other ⇒ onMessage(other) + case _ ⇒ + onMessage(env) } } }) else setHandler(in, new InHandler { override def onPush(): Unit = { - grab(in) match { - case InboundEnvelope(_, _, HandshakeReq(from), _, _) ⇒ - onHandshakeReq(from) - case other ⇒ onMessage(other) + val env = grab(in) + env.message match { + case HandshakeReq(from) ⇒ onHandshakeReq(from) + case _ ⇒ + onMessage(env) } } }) diff --git a/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala b/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala index 1a4454c58b..9727ecd3f3 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala @@ -29,10 +29,10 @@ private[akka] class InboundQuarantineCheck(inboundContext: InboundContext) exten override def onPush(): Unit = { val env = grab(in) inboundContext.association(env.originUid) match { - case OptionVal.None => + case OptionVal.None ⇒ // unknown, handshake not completed push(out, env) - case OptionVal.Some(association) => + case OptionVal.Some(association) ⇒ if (association.associationState.isQuarantined(env.originUid)) { inboundContext.sendControl( association.remoteAddress, 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..0020d52481 --- /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) = { + clear(obj) + (!pool.offer(obj)) + } +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala index 4f1766821e..a6ba8b25fe 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -226,12 +226,13 @@ private[akka] class SystemMessageAcker(inboundContext: InboundContext) extends G // InHandler override def onPush(): Unit = { - grab(in) match { - case env @ InboundEnvelope(_, _, sysEnv @ SystemMessageEnvelope(_, n, ackReplyTo), _, _) ⇒ + val env = grab(in) + env.message match { + case sysEnv @ SystemMessageEnvelope(_, n, ackReplyTo) ⇒ if (n == seqNo) { inboundContext.sendControl(ackReplyTo.address, Ack(n, localAddress)) seqNo += 1 - val unwrapped = env.copy(message = sysEnv.message) + val unwrapped = env.withMessage(sysEnv.message) push(out, unwrapped) } else if (n < seqNo) { inboundContext.sendControl(ackReplyTo.address, Ack(n, localAddress)) @@ -240,11 +241,10 @@ private[akka] class SystemMessageAcker(inboundContext: InboundContext) extends G inboundContext.sendControl(ackReplyTo.address, Nack(seqNo - 1, localAddress)) pull(in) } - case env ⇒ + case _ ⇒ // messages that don't need acking push(out, env) } - } // OutHandler diff --git a/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala b/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala index ec70bb69cf..07b369950f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala @@ -7,7 +7,6 @@ import scala.concurrent.ExecutionContext import scala.concurrent.Future import scala.concurrent.Promise import scala.concurrent.duration._ - import akka.Done import akka.actor.Address import akka.remote.EndpointManager.Send @@ -25,6 +24,7 @@ import akka.stream.stage.GraphStageWithMaterializedValue import akka.stream.stage.InHandler import akka.stream.stage.OutHandler import akka.stream.stage.TimerGraphStageLogic +import akka.util.OptionVal /** * INTERNAL API @@ -158,10 +158,10 @@ private[remote] class InboundTestStage(inboundContext: InboundContext) override def onPush(): Unit = { val env = grab(in) inboundContext.association(env.originUid) match { - case null ⇒ + case OptionVal.None ⇒ // unknown, handshake not completed push(out, env) - case association ⇒ + case OptionVal.Some(association) ⇒ if (blackhole(association.remoteAddress)) { log.debug( "dropping inbound message [{}] from [{}] with UID [{}] because of blackhole", 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 b0c35bc1f4..4777778a97 100644 --- a/akka-remote/src/main/scala/akka/remote/transport/AkkaPduCodec.scala +++ b/akka-remote/src/main/scala/akka/remote/transport/AkkaPduCodec.scala @@ -36,11 +36,11 @@ private[remote] object AkkaPduCodec { final case class Payload(bytes: ByteString) extends AkkaPdu final case class Message( - recipient: InternalActorRef, - recipientAddress: Address, + recipient: InternalActorRef, + recipientAddress: Address, serializedMessage: SerializedMessage, - senderOption: OptionVal[ActorRef], - seqOption: Option[SeqNo]) extends HasSequenceNumber { + senderOption: OptionVal[ActorRef], + seqOption: Option[SeqNo]) extends HasSequenceNumber { def reliableDeliveryEnabled = seqOption.isDefined @@ -95,12 +95,12 @@ private[remote] trait AkkaPduCodec { def decodeMessage(raw: ByteString, provider: RemoteActorRefProvider, localAddress: Address): (Option[Ack], Option[Message]) def constructMessage( - localAddress: Address, - recipient: ActorRef, + localAddress: Address, + recipient: ActorRef, serializedMessage: SerializedMessage, - senderOption: OptionVal[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 } @@ -119,12 +119,12 @@ private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec { } override def constructMessage( - localAddress: Address, - recipient: ActorRef, + localAddress: Address, + recipient: ActorRef, serializedMessage: SerializedMessage, - senderOption: OptionVal[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 @@ -132,8 +132,8 @@ private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec { envelopeBuilder.setRecipient(serializeActorRef(recipient.path.address, recipient)) senderOption match { - case OptionVal.Some(sender) => envelopeBuilder.setSender(serializeActorRef(localAddress, sender)) - case OptionVal.None => + case OptionVal.Some(sender) ⇒ envelopeBuilder.setSender(serializeActorRef(localAddress, sender)) + case OptionVal.None ⇒ } seqOption foreach { seq ⇒ envelopeBuilder.setSeq(seq.rawValue) } @@ -181,8 +181,8 @@ private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec { } override def decodeMessage( - raw: ByteString, - provider: RemoteActorRefProvider, + raw: ByteString, + provider: RemoteActorRefProvider, localAddress: Address): (Option[Ack], Option[Message]) = { val ackAndEnvelope = AckAndEnvelopeContainer.parseFrom(raw.toArray) @@ -231,7 +231,7 @@ private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec { Address(encodedAddress.getProtocol, encodedAddress.getSystem, encodedAddress.getHostname, encodedAddress.getPort) private def constructControlMessagePdu( - code: WireFormats.CommandType, + code: WireFormats.CommandType, handshakeInfo: Option[AkkaHandshakeInfo.Builder]): ByteString = { val controlMessageBuilder = AkkaControlMessage.newBuilder() diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala index efa0a10b54..2542833025 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala @@ -44,7 +44,7 @@ class InboundControlJunctionSpec extends AkkaSpec with ImplicitSender { val ((upstream, controlSubject), downstream) = TestSource.probe[AnyRef] .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, OptionVal.None, addressA.uid)) .viaMat(new InboundControlJunction)(Keep.both) - .map { case InboundEnvelope(_, _, msg, _, _) ⇒ msg } + .map { case env: InboundEnvelope ⇒ env.message } .toMat(TestSink.probe[Any])(Keep.both) .run() diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala index 5aa42f93ce..009ed1b34a 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala @@ -43,7 +43,7 @@ class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { TestSource.probe[AnyRef] .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, OptionVal.None, addressA.uid)) .via(new InboundHandshake(inboundContext, inControlStream = true)) - .map { case InboundEnvelope(_, _, msg, _, _) ⇒ msg } + .map { case env: InboundEnvelope ⇒ env.message } .toMat(TestSink.probe[Any])(Keep.both) .run() } diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala index 0ec5454ab8..9d7586c07d 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -17,6 +17,7 @@ import akka.remote.UniqueAddress import akka.remote.artery.InboundControlJunction.ControlMessageObserver import akka.remote.artery.InboundControlJunction.ControlMessageSubject import akka.util.OptionVal +import akka.actor.InternalActorRef private[akka] class TestInboundContext( override val localAddress: UniqueAddress, From 7a1a316e8a3370e7596cb1d5aaab7bade31be5b8 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 10 Jun 2016 13:04:23 +0200 Subject: [PATCH 068/186] reduce allocations with specialized ImmutableLongMap (#20750) * reduce allocations with specialized ImmutableLongMap * backed by arrays, allocation free lookups with binary search * use it for UID -> Association Map * pass Association in InboundEnvelope to reduce to only one lookup per incoming message * use ImmutableLongMap instead of the QuarantinedUIDSet --- .../akka/remote/artery/CodecBenchmark.scala | 14 +- .../artery/FlightRecorderBench.scala | 22 +-- .../akka/remote/artery/ArteryTransport.scala | 64 +++----- .../akka/remote/artery/Association.scala | 22 ++- .../scala/akka/remote/artery/Codecs.scala | 10 +- .../scala/akka/remote/artery/Handshake.scala | 8 +- .../akka/remote/artery/ImmutableLongMap.scala | 136 ++++++++++++++++ .../artery/InboundQuarantineCheck.scala | 2 +- .../scala/akka/remote/artery/TestStage.scala | 2 +- .../remote/artery/ImmutableLongMapSpec.scala | 148 ++++++++++++++++++ .../artery/InboundControlJunctionSpec.scala | 2 +- .../remote/artery/InboundHandshakeSpec.scala | 3 +- .../artery/SystemMessageDeliverySpec.scala | 3 +- .../akka/remote/artery/TestContext.scala | 3 +- 14 files changed, 366 insertions(+), 73 deletions(-) rename akka-bench-jmh/src/main/scala/akka/{ => remote}/artery/FlightRecorderBench.scala (73%) create mode 100644 akka-remote/src/main/scala/akka/remote/artery/ImmutableLongMap.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/ImmutableLongMapSpec.scala 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 index f573f98bc4..95969a10b4 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -26,6 +26,7 @@ import akka.stream.scaladsl._ import com.typesafe.config.ConfigFactory import org.openjdk.jmh.annotations._ import akka.util.OptionVal +import akka.actor.Address @State(Scope.Benchmark) @OutputTimeUnit(TimeUnit.MILLISECONDS) @@ -65,6 +66,15 @@ class CodecBenchmark { ) 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): Unit = ??? + } + private var materializer: ActorMaterializer = _ private var remoteRefB: RemoteActorRef = _ private var resolvedRef: InternalActorRef = _ @@ -154,7 +164,7 @@ class CodecBenchmark { } val decoder: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = - Flow.fromGraph(new Decoder(uniqueLocalAddress, system.asInstanceOf[ExtendedActorSystem], + Flow.fromGraph(new Decoder(inboundContext, system.asInstanceOf[ExtendedActorSystem], resolveActorRefWithLocalAddress, compression, envelopePool, inboundEnvelopePool)) Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) @@ -195,7 +205,7 @@ class CodecBenchmark { } val decoder: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = - Flow.fromGraph(new Decoder(uniqueLocalAddress, system.asInstanceOf[ExtendedActorSystem], + Flow.fromGraph(new Decoder(inboundContext, system.asInstanceOf[ExtendedActorSystem], resolveActorRefWithLocalAddress, compression, envelopePool, inboundEnvelopePool)) Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) diff --git a/akka-bench-jmh/src/main/scala/akka/artery/FlightRecorderBench.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/FlightRecorderBench.scala similarity index 73% rename from akka-bench-jmh/src/main/scala/akka/artery/FlightRecorderBench.scala rename to akka-bench-jmh/src/main/scala/akka/remote/artery/FlightRecorderBench.scala index c09df54030..9b286f34c0 100644 --- a/akka-bench-jmh/src/main/scala/akka/artery/FlightRecorderBench.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/FlightRecorderBench.scala @@ -1,12 +1,14 @@ -package akka.artery +/** + * 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 akka.remote.artery.FlightRecorder -import org.openjdk.jmh.annotations.{OperationsPerInvocation, _} +import java.util.concurrent.CountDownLatch +import java.util.concurrent.TimeUnit +import org.openjdk.jmh.annotations.{ OperationsPerInvocation, _ } @State(Scope.Benchmark) @OutputTimeUnit(TimeUnit.MICROSECONDS) @@ -18,12 +20,12 @@ class FlightRecorderBench { val Writes = 10000000 - var file: File = _ - var fileChannel: FileChannel = _ - var recorder: FlightRecorder = _ + private var file: File = _ + private var fileChannel: FileChannel = _ + private var recorder: FlightRecorder = _ @Setup - def setup():Unit = { + def setup(): Unit = { file = File.createTempFile("akka-flightrecorder", "dat") file.deleteOnExit() fileChannel = FileChannel.open(file.toPath, StandardOpenOption.CREATE, StandardOpenOption.WRITE, StandardOpenOption.READ) @@ -31,7 +33,7 @@ class FlightRecorderBench { } @TearDown - def shutdown():Unit = { + def shutdown(): Unit = { fileChannel.force(false) recorder.close() fileChannel.close() diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 68d05736bb..5acdf104be 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -7,6 +7,7 @@ import java.io.File import java.net.InetSocketAddress import java.nio.channels.DatagramChannel import java.util.concurrent.CopyOnWriteArrayList +import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicLong import scala.collection.JavaConverters._ @@ -88,9 +89,10 @@ private[akka] object InboundEnvelope { recipientAddress: Address, message: AnyRef, senderOption: OptionVal[ActorRef], - originUid: Long): InboundEnvelope = { + originUid: Long, + association: OptionVal[OutboundContext]): InboundEnvelope = { val env = new ReusableInboundEnvelope - env.init(recipient, recipientAddress, message, senderOption, originUid) + env.init(recipient, recipientAddress, message, senderOption, originUid, association) env } @@ -105,6 +107,7 @@ private[akka] trait InboundEnvelope { def message: AnyRef def senderOption: OptionVal[ActorRef] def originUid: Long + def association: OptionVal[OutboundContext] def withMessage(message: AnyRef): InboundEnvelope } @@ -118,12 +121,14 @@ private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { private var _message: AnyRef = null private var _senderOption: OptionVal[ActorRef] = OptionVal.None private var _originUid: Long = 0L + private var _association: OptionVal[OutboundContext] = OptionVal.None override def recipient: InternalActorRef = _recipient override def recipientAddress: Address = _recipientAddress override def message: AnyRef = _message override def senderOption: OptionVal[ActorRef] = _senderOption override def originUid: Long = _originUid + override def association: OptionVal[OutboundContext] = _association override def withMessage(message: AnyRef): InboundEnvelope = { _message = message @@ -136,6 +141,7 @@ private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { _message = null _senderOption = OptionVal.None _originUid = 0L + _association = OptionVal.None } def init( @@ -143,16 +149,18 @@ private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { recipientAddress: Address, message: AnyRef, senderOption: OptionVal[ActorRef], - originUid: Long): Unit = { + originUid: Long, + association: OptionVal[OutboundContext]): Unit = { _recipient = recipient _recipientAddress = recipientAddress _message = message _senderOption = senderOption _originUid = originUid + _association = association } override def toString: String = - s"InboundEnvelope($recipient, $recipientAddress, $message, $senderOption, $originUid)" + s"InboundEnvelope($recipient, $recipientAddress, $message, $senderOption, $originUid, $association)" } /** @@ -193,40 +201,13 @@ private[akka] trait InboundContext { */ private[akka] object AssociationState { def apply(): AssociationState = - new AssociationState(incarnation = 1, uniqueRemoteAddressPromise = Promise(), quarantined = QuarantinedUidSet.empty) + new AssociationState(incarnation = 1, uniqueRemoteAddressPromise = Promise(), + quarantined = ImmutableLongMap.empty[QuarantinedTimestamp]) - object QuarantinedUidSet { - val maxEntries = 10 // ok to not keep all old uids - def empty: QuarantinedUidSet = new QuarantinedUidSet(Array.emptyLongArray) + final case class QuarantinedTimestamp(nanoTime: Long) { + override def toString: String = + s"Quarantined ${TimeUnit.NANOSECONDS.toSeconds(System.nanoTime() - nanoTime)} seconds ago" } - - class QuarantinedUidSet private (uids: Array[Long]) { - import QuarantinedUidSet._ - - def add(uid: Long): QuarantinedUidSet = { - if (apply(uid)) - this - else { - val newUids = Array.ofDim[Long](math.min(uids.length + 1, maxEntries)) - newUids(0) = uid - if (uids.length > 0) - System.arraycopy(uids, 0, newUids, 1, newUids.length - 1) - new QuarantinedUidSet(newUids) - } - } - - def apply(uid: Long): Boolean = { - @tailrec def find(i: Int): Boolean = - if (i == uids.length) false - else if (uids(i) == uid) true - else find(i + 1) - find(0) - } - - override def toString(): String = - uids.mkString("QuarantinedUidSet(", ",", ")") - } - } /** @@ -235,7 +216,9 @@ private[akka] object AssociationState { private[akka] final class AssociationState( val incarnation: Int, val uniqueRemoteAddressPromise: Promise[UniqueAddress], - val quarantined: AssociationState.QuarantinedUidSet) { + 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 @@ -265,7 +248,8 @@ private[akka] final class AssociationState( def newQuarantined(): AssociationState = uniqueRemoteAddressPromise.future.value match { case Some(Success(a)) ⇒ - new AssociationState(incarnation, uniqueRemoteAddressPromise, quarantined = quarantined.add(a.uid)) + new AssociationState(incarnation, uniqueRemoteAddressPromise, + quarantined = quarantined.updated(a.uid, QuarantinedTimestamp(System.nanoTime()))) case _ ⇒ this } @@ -276,7 +260,7 @@ private[akka] final class AssociationState( } } - def isQuarantined(uid: Long): Boolean = quarantined(uid) + def isQuarantined(uid: Long): Boolean = quarantined.contains(uid) override def toString(): String = { val a = uniqueRemoteAddressPromise.future.value match { @@ -747,7 +731,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R def createDecoder(bufferPool: EnvelopeBufferPool): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { val resolveActorRefWithLocalAddress: String ⇒ InternalActorRef = recipient ⇒ provider.resolveActorRefWithLocalAddress(recipient, localAddress.address) - Flow.fromGraph(new Decoder(localAddress, system, resolveActorRefWithLocalAddress, compression, bufferPool, + Flow.fromGraph(new Decoder(this, system, resolveActorRefWithLocalAddress, compression, bufferPool, inboundEnvelopePool)) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index b46a479b76..8a4e85c5ee 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -391,7 +391,7 @@ private[remote] class Association( */ private[remote] class AssociationRegistry(createAssociation: Address ⇒ Association) { private[this] val associationsByAddress = new AtomicReference[Map[Address, Association]](Map.empty) - private[this] val associationsByUid = new ConcurrentHashMap[Long, Association]() // FIXME replace with specialized Long Map + private[this] val associationsByUid = new AtomicReference[ImmutableLongMap[Association]](ImmutableLongMap.empty) @tailrec final def association(remoteAddress: Address): Association = { val currentMap = associationsByAddress.get @@ -409,14 +409,22 @@ private[remote] class AssociationRegistry(createAssociation: Address ⇒ Associa } def association(uid: Long): OptionVal[Association] = - OptionVal(associationsByUid.get(uid)) + associationsByUid.get.get(uid) - def setUID(peer: UniqueAddress): Association = { + @tailrec final def setUID(peer: UniqueAddress): Association = { + val currentMap = associationsByUid.get val a = association(peer.address) - val previous = associationsByUid.put(peer.uid, a) - if ((previous ne null) && (previous ne a)) - throw new IllegalArgumentException(s"UID collision old [$previous] new [$a]") - a + // make sure we don't overwrite same UID with different association + currentMap.get(peer.uid) match { + case OptionVal.Some(previous) if (previous ne a) ⇒ + throw new IllegalArgumentException(s"UID collision old [$previous] new [$a]") + case _ ⇒ // ok + } + val newMap = currentMap.updated(peer.uid, a) + if (associationsByUid.compareAndSet(currentMap, newMap)) + a + else + setUID(peer) // lost CAS, retry } def allAssociations: Set[Association] = diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 47df0a7efc..db77bc75ae 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -105,7 +105,7 @@ class Encoder( } class Decoder( - uniqueLocalAddress: UniqueAddress, + inboundContext: InboundContext, system: ExtendedActorSystem, resolveActorRefWithLocalAddress: String ⇒ InternalActorRef, compressionTable: LiteralCompressionTable, @@ -117,7 +117,7 @@ class Decoder( override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) with InHandler with OutHandler with StageLogging { - private val localAddress = uniqueLocalAddress.address + private val localAddress = inboundContext.localAddress.address private val headerBuilder = HeaderBuilder(compressionTable) private val serialization = SerializationExtension(system) @@ -160,6 +160,9 @@ class Decoder( } } + val originUid = headerBuilder.uid + val association = inboundContext.association(originUid) + try { val deserializedMessage = MessageSerializer.deserializeForArtery( system, serialization, headerBuilder, envelope) @@ -170,7 +173,8 @@ class Decoder( localAddress, // FIXME: Is this needed anymore? What should we do here? deserializedMessage, senderOption, - headerBuilder.uid) + originUid, + association) push(out, decoded) } catch { diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala index 93d8fad3ec..22b633208f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -196,7 +196,7 @@ private[akka] class InboundHandshake(inboundContext: InboundContext, inControlSt } private def onMessage(env: InboundEnvelope): Unit = { - if (isKnownOrigin(env.originUid)) + if (isKnownOrigin(env)) push(out, env) else { // FIXME remove, only debug @@ -215,10 +215,8 @@ private[akka] class InboundHandshake(inboundContext: InboundContext, inControlSt } } - private def isKnownOrigin(originUid: Long): Boolean = { - // FIXME these association lookups are probably too costly for each message, need local cache or something - inboundContext.association(originUid).isDefined - } + private def isKnownOrigin(env: InboundEnvelope): Boolean = + env.association.isDefined // OutHandler override def onPull(): Unit = pull(in) 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/InboundQuarantineCheck.scala b/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala index 9727ecd3f3..a2b58991f5 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala @@ -28,7 +28,7 @@ private[akka] class InboundQuarantineCheck(inboundContext: InboundContext) exten // InHandler override def onPush(): Unit = { val env = grab(in) - inboundContext.association(env.originUid) match { + env.association match { case OptionVal.None ⇒ // unknown, handshake not completed push(out, env) diff --git a/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala b/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala index 07b369950f..e9fba1ffdd 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala @@ -157,7 +157,7 @@ private[remote] class InboundTestStage(inboundContext: InboundContext) // InHandler override def onPush(): Unit = { val env = grab(in) - inboundContext.association(env.originUid) match { + env.association match { case OptionVal.None ⇒ // unknown, handshake not completed push(out, env) 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 index 2542833025..5afbdcec07 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala @@ -42,7 +42,7 @@ class InboundControlJunctionSpec extends AkkaSpec with ImplicitSender { val recipient = null.asInstanceOf[InternalActorRef] // not used val ((upstream, controlSubject), downstream) = TestSource.probe[AnyRef] - .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, OptionVal.None, addressA.uid)) + .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, OptionVal.None, addressA.uid, OptionVal.None)) .viaMat(new InboundControlJunction)(Keep.both) .map { case env: InboundEnvelope ⇒ env.message } .toMat(TestSink.probe[Any])(Keep.both) diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala index 009ed1b34a..fb369d4bd4 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala @@ -41,7 +41,8 @@ class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { private def setupStream(inboundContext: InboundContext, timeout: FiniteDuration = 5.seconds): (TestPublisher.Probe[AnyRef], TestSubscriber.Probe[Any]) = { val recipient = null.asInstanceOf[InternalActorRef] // not used TestSource.probe[AnyRef] - .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, OptionVal.None, addressA.uid)) + .map(msg ⇒ InboundEnvelope(recipient, addressB.address, 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) diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index a6d309ebf2..a84c964338 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -76,7 +76,8 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi Flow[Send] .map { case Send(sysEnv: SystemMessageEnvelope, _, _, _) ⇒ - InboundEnvelope(recipient, addressB.address, sysEnv, OptionVal.None, addressA.uid) + InboundEnvelope(recipient, addressB.address, sysEnv, OptionVal.None, addressA.uid, + inboundContext.association(addressA.uid)) } .async .via(new SystemMessageAcker(inboundContext)) diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala index 9d7586c07d..94763cda5c 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -85,7 +85,8 @@ private[akka] class TestOutboundContext( override def sendControl(message: ControlMessage) = { controlProbe.foreach(_ ! message) - controlSubject.sendControl(InboundEnvelope(null, remoteAddress, message, OptionVal.None, localAddress.uid)) + controlSubject.sendControl(InboundEnvelope(null, remoteAddress, message, OptionVal.None, localAddress.uid, + OptionVal.None)) } // FIXME we should be able to Send without a recipient ActorRef From 7ce6dffabfcc480ae22cac7819607d94a3193b93 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 8 Jun 2016 12:40:40 +0200 Subject: [PATCH 069/186] send dropped system messages to deadLetters * publish remote lifecycle event for quarantined --- .../scala/akka/remote/RemoteActorRefProvider.scala | 8 +++++--- .../scala/akka/remote/artery/ArteryTransport.scala | 11 +++-------- .../main/scala/akka/remote/artery/Association.scala | 3 +++ .../akka/remote/artery/SystemMessageDelivery.scala | 13 +++++++++++++ .../akka/remote/artery/RemoteDeathWatchSpec.scala | 6 ++---- .../remote/artery/SystemMessageDeliverySpec.scala | 3 ++- 6 files changed, 28 insertions(+), 16 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 119b406d8f..70bf13e572 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -497,12 +497,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) } /** @@ -529,11 +531,11 @@ private[akka] class RemoteActorRef private[akka] ( provider.remoteWatcher ! RemoteWatcher.UnwatchRemote(watchee, watcher) 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, OptionVal(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/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 5acdf104be..9edaa01bf2 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -9,16 +9,13 @@ import java.nio.channels.DatagramChannel import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicLong - import scala.collection.JavaConverters._ - 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 akka.Done import akka.NotUsed import akka.actor.ActorRef @@ -68,13 +65,10 @@ import org.agrona.IoUtil import java.io.File import java.net.InetSocketAddress import java.nio.channels.{ DatagramChannel, FileChannel } - import akka.remote.artery.OutboundControlJunction.OutboundControlIngress import io.aeron.CncFileDescriptor import java.util.concurrent.atomic.AtomicLong - import akka.actor.Cancellable - import scala.collection.JavaConverters._ import akka.stream.ActorMaterializerSettings import scala.annotation.tailrec @@ -331,7 +325,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R override def addresses: Set[Address] = _addresses override def localAddressForRemote(remote: Address): Address = defaultAddress override val log: LoggingAdapter = Logging(system, getClass.getName) - private val eventPublisher = new EventPublisher(system, log, remoteSettings.RemoteLifecycleEventsLogLevel) + val eventPublisher = new EventPublisher(system, log, remoteSettings.RemoteLifecycleEventsLogLevel) private val codec: AkkaPduCodec = AkkaPduProtobufCodec private val killSwitch: SharedKillSwitch = KillSwitches.shared("transportKillSwitch") @@ -705,7 +699,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R def outboundControl(outboundContext: OutboundContext): Sink[Send, (OutboundControlIngress, Future[Done])] = { Flow.fromGraph(killSwitch.flow[Send]) .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) - .via(new SystemMessageDelivery(outboundContext, systemMessageResendInterval, remoteSettings.SysMsgBufferSize)) + .via(new SystemMessageDelivery(outboundContext, system.deadLetters, systemMessageResendInterval, + remoteSettings.SysMsgBufferSize)) .viaMat(new OutboundControlJunction(outboundContext))(Keep.right) .via(encoder) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), controlStreamId, aeron, taskRunner, diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 8a4e85c5ee..ca90c9a366 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -36,6 +36,7 @@ import akka.stream.scaladsl.Source import akka.util.{ Unsafe, WildcardTree } import org.agrona.concurrent.ManyToOneConcurrentArrayQueue import akka.util.OptionVal +import akka.remote.QuarantinedEvent /** * INTERNAL API @@ -223,6 +224,8 @@ private[remote] class Association( log.warning( "Association to [{}] with UID [{}] is irrecoverably failed. Quarantining address. {}", remoteAddress, u, reason) + // FIXME when we complete the switch to Long UID we must use Long here also, issue #20644 + transport.eventPublisher.notifyListeners(QuarantinedEvent(remoteAddress, u.toInt)) // end delivery of system messages to that incarnation after this point send(ClearSystemMessageDelivery, OptionVal.None, dummyRecipient) // try to tell the other system that we have quarantined it diff --git a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala index a6ba8b25fe..de2ee9da60 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -23,6 +23,7 @@ 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 /** * INTERNAL API @@ -44,6 +45,7 @@ private[akka] object SystemMessageDelivery { */ private[akka] class SystemMessageDelivery( outboundContext: OutboundContext, + deadLetters: ActorRef, resendInterval: FiniteDuration, maxBufferSize: Int) extends GraphStage[FlowShape[Send, Send]] { @@ -87,6 +89,8 @@ private[akka] class SystemMessageDelivery( } override def postStop(): Unit = { + sendUnacknowledgedToDeadLetters() + unacknowledged.clear() outboundContext.controlSubject.detach(this) } @@ -180,12 +184,14 @@ private[akka] class SystemMessageDelivery( } else { // buffer overflow outboundContext.quarantine(reason = s"System message delivery buffer overflow, size [$maxBufferSize]") + deadLetters ! s pull(in) } } } private def clear(): Unit = { + sendUnacknowledgedToDeadLetters() seqNo = 0L // sequence number for the first message will be 1 unacknowledged.clear() resending.clear() @@ -193,6 +199,13 @@ private[akka] class SystemMessageDelivery( 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 diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala index 7313806127..297cb9fcc0 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala @@ -72,8 +72,7 @@ class RemoteDeathWatchSpec extends AkkaSpec(RemoteDeathWatchSpec.config) with Im } } - // FIXME this is failing with Artery - "receive Terminated when watched node is unknown host" ignore { + "receive Terminated when watched node is unknown host" in { val path = RootActorPath(Address("artery", system.name, "unknownhost", 2552)) / "user" / "subject" system.actorOf(Props(new Actor { context.watch(context.actorFor(path)) @@ -85,8 +84,7 @@ class RemoteDeathWatchSpec extends AkkaSpec(RemoteDeathWatchSpec.config) with Im expectMsg(60.seconds, path) } - // FIXME this is failing with Artery - "receive ActorIdentity(None) when identified node is unknown host" ignore { + "receive ActorIdentity(None) when identified node is unknown host" in { val path = RootActorPath(Address("artery", 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/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index a84c964338..5f9a3e0b50 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -66,9 +66,10 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi private def send(sendCount: Int, resendInterval: FiniteDuration, outboundContext: OutboundContext): Source[Send, NotUsed] = { val remoteRef = null.asInstanceOf[RemoteActorRef] // not used + val deadLetters = TestProbe().ref Source(1 to sendCount) .map(n ⇒ Send("msg-" + n, OptionVal.None, remoteRef, None)) - .via(new SystemMessageDelivery(outboundContext, resendInterval, maxBufferSize = 1000)) + .via(new SystemMessageDelivery(outboundContext, deadLetters, resendInterval, maxBufferSize = 1000)) } private def inbound(inboundContext: InboundContext): Flow[Send, InboundEnvelope, NotUsed] = { From d6c048f59a10eb616a97290d7eb3ff722402e70f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Fri, 10 Jun 2016 15:04:13 +0200 Subject: [PATCH 070/186] A simpler ActorRefProvider config #20649 (#20767) * Provide shorter aliases for the ActorRefProviders #20649 * Use the new actorefprovider aliases throughout code and docs * Cleaner alias replacement logic --- akka-actor/src/main/resources/reference.conf | 6 ++++-- .../src/main/scala/akka/actor/ActorSystem.scala | 11 ++++++++++- .../scala/akka/remote/artery/CodecBenchmark.scala | 2 +- .../test/scala/akka/cluster/metrics/TestUtil.scala | 6 +++--- .../metrics/protobuf/MessageSerializerSpec.scala | 2 +- .../ClusterShardingCustomShardAllocationSpec.scala | 2 +- .../cluster/sharding/ClusterShardingFailureSpec.scala | 2 +- .../sharding/ClusterShardingGetStateSpec.scala | 2 +- .../sharding/ClusterShardingGetStatsSpec.scala | 2 +- .../ClusterShardingGracefulShutdownSpec.scala | 2 +- .../cluster/sharding/ClusterShardingLeavingSpec.scala | 2 +- .../akka/cluster/sharding/ClusterShardingSpec.scala | 2 +- .../RemoveInternalClusterShardingDataSpec.scala | 2 +- .../scala/akka/cluster/client/ClusterClientSpec.scala | 2 +- .../akka/cluster/client/ClusterClientStopSpec.scala | 2 +- .../pubsub/DistributedPubSubMediatorSpec.scala | 2 +- .../singleton/ClusterSingletonManagerChaosSpec.scala | 2 +- .../singleton/ClusterSingletonManagerLeaveSpec.scala | 2 +- .../singleton/ClusterSingletonManagerSpec.scala | 2 +- .../ClusterSingletonManagerStartupSpec.scala | 2 +- .../java/akka/cluster/client/ClusterClientTest.java | 2 +- .../cluster/pubsub/DistributedPubSubMediatorTest.java | 2 +- .../pubsub/DistributedPubSubMediatorRouterSpec.scala | 2 +- .../cluster/singleton/ClusterSingletonProxySpec.scala | 2 +- .../scala/akka/cluster/MultiNodeClusterSpec.scala | 2 +- .../src/multi-jvm/scala/akka/cluster/StressSpec.scala | 2 +- .../scala/akka/cluster/SunnyWeatherSpec.scala | 2 +- .../test/scala/akka/cluster/ClusterDeployerSpec.scala | 2 +- .../cluster/ClusterDomainEventPublisherSpec.scala | 2 +- .../src/test/scala/akka/cluster/ClusterSpec.scala | 4 ++-- .../test/scala/akka/cluster/DowningProviderSpec.scala | 2 +- .../scala/akka/cluster/MetricsCollectorSpec.scala | 2 +- .../scala/akka/cluster/StartupWithOneThreadSpec.scala | 2 +- .../protobuf/ClusterMessageSerializerSpec.scala | 2 +- .../cluster/routing/ClusterRouterSupervisorSpec.scala | 2 +- .../akka/cluster/routing/WeightedRouteesSpec.scala | 2 +- .../akka/cluster/ddata/JepsenInspiredInsertSpec.scala | 2 +- .../scala/akka/cluster/ddata/PerformanceSpec.scala | 2 +- .../akka/cluster/ddata/ReplicatorChaosSpec.scala | 2 +- .../akka/cluster/ddata/ReplicatorPruningSpec.scala | 2 +- .../scala/akka/cluster/ddata/ReplicatorSpec.scala | 2 +- .../akka/cluster/ddata/LocalConcurrencySpec.scala | 2 +- .../test/scala/akka/cluster/ddata/LotsOfDataBot.scala | 2 +- .../akka/cluster/ddata/WriteAggregatorSpec.scala | 2 +- .../ddata/protobuf/ReplicatedDataSerializerSpec.scala | 2 +- .../protobuf/ReplicatorMessageSerializerSpec.scala | 2 +- akka-docs/rst/general/configuration.rst | 2 +- akka-docs/rst/java/cluster-client.rst | 4 ++-- akka-docs/rst/java/cluster-usage.rst | 2 +- akka-docs/rst/java/remoting.rst | 4 ++-- akka-docs/rst/scala/cluster-client.rst | 4 ++-- akka-docs/rst/scala/cluster-usage.rst | 2 +- .../rst/scala/code/docs/cluster/ClusterDocSpec.scala | 2 +- .../code/docs/ddata/DistributedDataDocSpec.scala | 2 +- .../code/docs/remoting/RemoteDeploymentDocSpec.scala | 2 +- akka-docs/rst/scala/remoting.rst | 4 ++-- .../scala/akka/remote/testkit/MultiNodeSpec.scala | 2 +- .../journal/leveldb/PersistencePluginProxySpec.scala | 2 +- .../journal/leveldb/SharedLeveldbJournalSpec.scala | 2 +- .../persistence/serialization/SerializerSpec.scala | 2 +- .../remote/artery/AeronStreamConcistencySpec.scala | 2 +- .../akka/remote/artery/AeronStreamLatencySpec.scala | 2 +- .../remote/artery/AeronStreamMaxThroughputSpec.scala | 2 +- .../remote/artery/HandshakeRestartReceiverSpec.scala | 2 +- .../scala/akka/remote/artery/LatencySpec.scala | 2 +- .../scala/akka/remote/artery/MaxThroughputSpec.scala | 2 +- .../scala/akka/remote/testconductor/BarrierSpec.scala | 2 +- .../akka/remote/testconductor/ControllerSpec.scala | 2 +- .../src/test/scala/akka/remote/ActorsLeakSpec.scala | 2 +- .../src/test/scala/akka/remote/DaemonicSpec.scala | 2 +- .../src/test/scala/akka/remote/LogSourceSpec.scala | 2 +- .../scala/akka/remote/RemoteActorMailboxSpec.scala | 2 +- .../src/test/scala/akka/remote/RemoteConfigSpec.scala | 2 +- .../remote/RemoteConsistentHashingRouterSpec.scala | 2 +- .../test/scala/akka/remote/RemoteDeathWatchSpec.scala | 2 +- .../test/scala/akka/remote/RemoteDeployerSpec.scala | 2 +- .../test/scala/akka/remote/RemoteInitErrorSpec.scala | 2 +- .../src/test/scala/akka/remote/RemoteRouterSpec.scala | 2 +- .../test/scala/akka/remote/RemoteWatcherSpec.scala | 2 +- .../src/test/scala/akka/remote/RemotingSpec.scala | 2 +- .../akka/remote/Ticket1978CommunicationSpec.scala | 2 +- .../akka/remote/TypedActorRemoteDeploySpec.scala | 2 +- .../src/test/scala/akka/remote/UntrustedSpec.scala | 4 ++-- .../akka/remote/artery/HandshakeFailureSpec.scala | 2 +- .../scala/akka/remote/artery/HandshakeRetrySpec.scala | 2 +- .../akka/remote/artery/LargeMessagesStreamSpec.scala | 2 +- .../remote/artery/RemoteActorRefProviderSpec.scala | 2 +- .../akka/remote/artery/RemoteDeathWatchSpec.scala | 2 +- .../scala/akka/remote/artery/RemoteDeployerSpec.scala | 2 +- .../akka/remote/artery/RemoteDeploymentSpec.scala | 2 +- .../scala/akka/remote/artery/RemoteRouterSpec.scala | 2 +- .../remote/artery/RemoteSendConsistencySpec.scala | 2 +- .../scala/akka/remote/artery/RemoteWatcherSpec.scala | 2 +- .../akka/remote/artery/SerializationErrorSpec.scala | 2 +- .../remote/artery/SystemMessageDeliverySpec.scala | 2 +- .../test/scala/akka/remote/artery/UntrustedSpec.scala | 4 ++-- .../akka/remote/transport/AkkaProtocolSpec.scala | 2 +- .../remote/transport/AkkaProtocolStressTest.scala | 2 +- .../akka/remote/transport/GenericTransportSpec.scala | 2 +- .../transport/SystemMessageDeliveryStressTest.scala | 2 +- .../transport/ThrottlerTransportAdapterSpec.scala | 2 +- .../remote/transport/netty/NettyTransportSpec.scala | 2 +- .../src/main/resources/application.conf | 2 +- .../cluster/stats/StatsSampleSingleMasterSpec.scala | 2 +- .../scala/sample/cluster/stats/StatsSampleSpec.scala | 2 +- .../transformation/TransformationSampleSpec.scala | 2 +- .../akka-sample-cluster-java/tutorial/index.html | 2 +- .../src/main/resources/application.conf | 2 +- .../cluster/stats/StatsSampleSingleMasterSpec.scala | 2 +- .../scala/sample/cluster/stats/StatsSampleSpec.scala | 2 +- .../transformation/TransformationSampleSpec.scala | 2 +- .../akka-sample-cluster-scala/tutorial/index.html | 2 +- .../src/main/resources/application.conf | 2 +- .../sample/distributeddata/ReplicatedCacheSpec.scala | 2 +- .../distributeddata/ReplicatedMetricsSpec.scala | 2 +- .../sample/distributeddata/ServiceRegistrySpec.scala | 2 +- .../sample/distributeddata/ShoppingCartSpec.scala | 2 +- .../sample/distributeddata/VotingServiceSpec.scala | 2 +- .../src/main/resources/application.conf | 2 +- .../sample/distributeddata/ReplicatedCacheSpec.scala | 2 +- .../distributeddata/ReplicatedMetricsSpec.scala | 2 +- .../sample/distributeddata/ServiceRegistrySpec.scala | 2 +- .../sample/distributeddata/ShoppingCartSpec.scala | 2 +- .../sample/distributeddata/VotingServiceSpec.scala | 2 +- .../core/src/main/resources/application.conf | 2 +- .../src/main/resources/common.conf | 2 +- .../src/main/resources/common.conf | 2 +- 127 files changed, 148 insertions(+), 137 deletions(-) diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index a58c8e0c21..394025c4db 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. diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index c0789e1481..15f190bbfd 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-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala index 95969a10b4..b02ca87672 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -40,7 +40,7 @@ class CodecBenchmark { """ akka { loglevel = WARNING - actor.provider = "akka.remote.RemoteActorRefProvider" + actor.provider = remote remote.artery.enabled = on remote.artery.hostname = localhost remote.artery.port = 0 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..243915a8a8 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 @@ -153,7 +153,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 +163,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 +178,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/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 0788c895cb..cc9e19b88a 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 @@ -125,7 +125,7 @@ abstract class ClusterShardingSpecConfig(val mode: String) extends MultiNodeConf 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..26d549e40c 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 @@ -30,7 +30,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 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 09a761863b..1483e8784c 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/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..0c32cd051a 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 @@ -68,7 +68,7 @@ object ClusterSingletonProxySpec { min-nr-of-members = 2 } - actor.provider = "akka.cluster.ClusterActorRefProvider" + actor.provider = "cluster" remote { log-remote-lifecycle-events = off 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..ed5ac063ff 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala @@ -34,7 +34,7 @@ object MultiNodeClusterSpec { if (failureDetectorPuppet) clusterConfigWithFailureDetectorPuppet else clusterConfig def clusterConfig: Config = ConfigFactory.parseString(""" - akka.actor.provider = akka.cluster.ClusterActorRefProvider + akka.actor.provider = cluster akka.cluster { jmx.enabled = off gossip-interval = 200 ms 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..06ae2bc94e 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala @@ -124,7 +124,7 @@ 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 auto-down-unreachable-after = 1s 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 c9e0bb7104..7ab3a00bfb 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala @@ -24,7 +24,7 @@ class SunnyWeatherMultiJvmSpec(artery: Boolean) extends MultiNodeConfig { val common = """ akka { - actor.provider = akka.cluster.ClusterActorRefProvider + actor.provider = cluster loggers = ["akka.testkit.TestEventListener"] loglevel = INFO remote.log-remote-lifecycle-events = off 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..c3232097df 100644 --- a/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala @@ -21,7 +21,7 @@ import akka.testkit.TestProbe object ClusterDomainEventPublisherSpec { val config = """ - 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/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..49ae6f32fc 100644 --- a/akka-cluster/src/test/scala/akka/cluster/routing/WeightedRouteesSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/routing/WeightedRouteesSpec.scala @@ -12,7 +12,7 @@ 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 """)) { 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 b24b185779..10c57cc006 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 2c72c3cf0c..78efdb3a46 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..9b8cb45a9d 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 @@ -50,7 +50,7 @@ object WriteAggregatorSpec { } class WriteAggregatorSpec extends AkkaSpec(""" - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.actor.provider = "cluster" akka.remote.netty.tcp.port=0 """) with ImplicitSender { 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..e96cca39e1 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 @@ -28,7 +28,7 @@ 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 { 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..314e1f34b9 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 @@ -26,7 +26,7 @@ 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 { diff --git a/akka-docs/rst/general/configuration.rst b/akka-docs/rst/general/configuration.rst index 1fd9624337..c8df1b3a2e 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 a5c8598d02..169917f11d 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 7fb1cbc8e4..02ec42a18e 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 eeca0c151f..058b17a472 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 ea9e449dc9..7cc6437a08 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-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 cad8949dfa..9e637341c1 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 @@ -196,7 +196,7 @@ 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.hostname" → selfName, "akka.remote.netty.tcp.hostname" → selfName, "akka.remote.netty.tcp.port" → selfPort, 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/artery/AeronStreamConcistencySpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/AeronStreamConcistencySpec.scala index 9f92fdd8ec..bdd82c4061 100644 --- 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 @@ -37,7 +37,7 @@ object AeronStreamConsistencySpec extends MultiNodeConfig { akka { loglevel = INFO actor { - provider = "akka.remote.RemoteActorRefProvider" + provider = remote } remote.artery.enabled = off } 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 index 5f5a77b950..8c9c504fe5 100644 --- 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 @@ -48,7 +48,7 @@ object AeronStreamLatencySpec extends MultiNodeConfig { loglevel = ERROR testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s actor { - provider = "akka.remote.RemoteActorRefProvider" + provider = remote serialize-creators = false serialize-messages = false } 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 index 49e8e6abb5..eca9bf7af4 100644 --- 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 @@ -41,7 +41,7 @@ object AeronStreamMaxThroughputSpec extends MultiNodeConfig { loglevel = ERROR testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s actor { - provider = "akka.remote.RemoteActorRefProvider" + provider = remote serialize-creators = false serialize-messages = false } 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 index 7084c138ce..648c4e41ff 100644 --- 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 @@ -24,7 +24,7 @@ object HandshakeRestartReceiverSpec extends MultiNodeConfig { ConfigFactory.parseString(s""" akka { loglevel = INFO - actor.provider = "akka.remote.RemoteActorRefProvider" + actor.provider = remote remote.artery { enabled = on } 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 index 602b4b5edc..1ee34ee518 100644 --- 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 @@ -47,7 +47,7 @@ object LatencySpec extends MultiNodeConfig { loggers = ["akka.event.Logging$$DefaultLogger"] testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s actor { - provider = "akka.remote.RemoteActorRefProvider" + provider = remote serialize-creators = false serialize-messages = false } 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 index f7c5dc823d..65f966091d 100644 --- 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 @@ -36,7 +36,7 @@ object MaxThroughputSpec extends MultiNodeConfig { loggers = ["akka.event.Logging$$DefaultLogger"] testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s actor { - provider = "akka.remote.RemoteActorRefProvider" + provider = remote serialize-creators = false serialize-messages = false 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/test/scala/akka/remote/ActorsLeakSpec.scala b/akka-remote/src/test/scala/akka/remote/ActorsLeakSpec.scala index ba38dd29ab..409891d001 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 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..2528408023 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" } 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..1b51e68455 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 diff --git a/akka-remote/src/test/scala/akka/remote/RemoteWatcherSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteWatcherSpec.scala index 23d1072c51..59f23d4398 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteWatcherSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteWatcherSpec.scala @@ -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 8799e47f6b..67769be238 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 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/HandshakeFailureSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala index 9a368c8404..ce8126368a 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala @@ -18,7 +18,7 @@ object HandshakeFailureSpec { val commonConfig = ConfigFactory.parseString(s""" akka { - actor.provider = "akka.remote.RemoteActorRefProvider" + actor.provider = remote remote.artery.enabled = on remote.artery.hostname = localhost remote.artery.port = 0 diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala index 1b90d60611..44c79967b3 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala @@ -18,7 +18,7 @@ object HandshakeRetrySpec { val commonConfig = ConfigFactory.parseString(s""" akka { - actor.provider = "akka.remote.RemoteActorRefProvider" + actor.provider = remote remote.artery.enabled = on remote.artery.hostname = localhost remote.artery.port = 0 diff --git a/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala index 7a8343099b..d52e28da67 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala @@ -32,7 +32,7 @@ class LargeMessagesStreamSpec extends WordSpec with ShouldMatchers with ScalaFut akka { loglevel = ERROR actor { - provider = "akka.remote.RemoteActorRefProvider" + provider = remote } remote.artery { enabled = on diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala index 3d0614dc79..fcfe56ba6f 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala @@ -17,7 +17,7 @@ object RemoteActorRefProviderSpec { val config = ConfigFactory.parseString(s""" akka { - actor.provider = "akka.remote.RemoteActorRefProvider" + actor.provider = remote remote.artery.enabled = on remote.artery.hostname = localhost remote.artery.port = 0 diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala index 297cb9fcc0..8b8ec6a37e 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala @@ -20,7 +20,7 @@ object RemoteDeathWatchSpec { val config = ConfigFactory.parseString(s""" akka { actor { - provider = "akka.remote.RemoteActorRefProvider" + provider = remote deployment { /watchers.remote = "artery://other@localhost:$otherPort" } diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeployerSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeployerSpec.scala index 9020f1951a..5c95b122e8 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeployerSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeployerSpec.scala @@ -12,7 +12,7 @@ import akka.remote.RemoteScope 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/artery/RemoteDeploymentSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala index 8f89196a68..c314cd5e0a 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala @@ -34,7 +34,7 @@ object RemoteDeploymentSpec { class RemoteDeploymentSpec extends AkkaSpec(""" #akka.loglevel=DEBUG - akka.actor.provider = "akka.remote.RemoteActorRefProvider" + akka.actor.provider = remote akka.remote.artery.enabled = on akka.remote.artery.hostname = localhost akka.remote.artery.port = 0 diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala index 3700df47c8..6ce9ba3004 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala @@ -23,7 +23,7 @@ object RemoteRouterSpec { class RemoteRouterSpec extends AkkaSpec(""" akka.loglevel=DEBUG - akka.actor.provider = "akka.remote.RemoteActorRefProvider" + akka.actor.provider = remote akka.remote.artery.enabled = on akka.remote.artery.hostname = localhost akka.remote.artery.port = 0 diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala index d14861415f..3abbbb7309 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala @@ -13,7 +13,7 @@ object RemoteSendConsistencySpec { val config = ConfigFactory.parseString(s""" akka { - actor.provider = "akka.remote.RemoteActorRefProvider" + actor.provider = remote remote.artery.enabled = on remote.artery.hostname = localhost remote.artery.port = 0 diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala index 9a5f11d1f3..7f4027f62a 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala @@ -67,7 +67,7 @@ class RemoteWatcherSpec extends AkkaSpec( """akka { loglevel = INFO log-dead-letters-during-shutdown = false - actor.provider = "akka.remote.RemoteActorRefProvider" + actor.provider = remote remote.artery.enabled = on remote.artery.hostname = localhost remote.artery.port = 0 diff --git a/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala index 5c3364da39..d997cf08fd 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala @@ -14,7 +14,7 @@ object SerializationErrorSpec { val config = ConfigFactory.parseString(s""" akka { - actor.provider = "akka.remote.RemoteActorRefProvider" + actor.provider = remote remote.artery.enabled = on remote.artery.hostname = localhost remote.artery.port = 0 diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index 5f9a3e0b50..b964a1c741 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -37,7 +37,7 @@ object SystemMessageDeliverySpec { val config = ConfigFactory.parseString(s""" akka { - actor.provider = "akka.remote.RemoteActorRefProvider" + actor.provider = remote remote.artery.enabled = on remote.artery.hostname = localhost remote.artery.port = 0 diff --git a/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala index 65798d6956..9b353a3be8 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/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.artery.enabled = on @@ -72,7 +72,7 @@ class UntrustedSpec extends AkkaSpec(""" import UntrustedSpec._ val client = ActorSystem("UntrustedSpec-client", ConfigFactory.parseString(""" - akka.actor.provider = akka.remote.RemoteActorRefProvider + akka.actor.provider = remote akka.remote.artery.enabled = on akka.remote.artery.hostname = localhost akka.remote.artery.port = 0 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 3825bba924..0f283aeb2d 100644 --- a/akka-remote/src/test/scala/akka/remote/transport/AkkaProtocolSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/transport/AkkaProtocolSpec.scala @@ -30,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( """ 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-cluster-scala/src/main/resources/application.conf b/akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf index 58195b14b4..0d00e9b66d 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf +++ b/akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf @@ -1,7 +1,7 @@ #//#snippet akka { actor { - provider = "akka.cluster.ClusterActorRefProvider" + provider = "cluster" } remote.artery { enabled = on diff --git a/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala b/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala index e22a1a281f..dbbdd32e11 100644 --- a/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala +++ b/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala @@ -48,7 +48,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-scala/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala b/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala index dba0965de2..9421e17cbc 100644 --- a/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala +++ b/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala @@ -40,7 +40,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-scala/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala b/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala index cd9324181c..0eb2469d1a 100644 --- a/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala +++ b/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala @@ -42,7 +42,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-scala/tutorial/index.html b/akka-samples/akka-sample-cluster-scala/tutorial/index.html index 1628cb5970..0567e6685f 100644 --- a/akka-samples/akka-sample-cluster-scala/tutorial/index.html +++ b/akka-samples/akka-sample-cluster-scala/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 { From 5c234940c60b1f862f58ed1bb683798b0a4b8cae Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 9 Jun 2016 09:16:44 +0200 Subject: [PATCH 071/186] make remote deployment work with Artery, #20715 There were two related problems with remote deployment when using Artery. * 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. * The first point solves the creation of the remote deployed actor but it's not enough. Resolve of the recipient actor ref may still happen before the actor is created. This is solved by retrying the resolve for the first message of a remote deployed actor. --- .../remote/AttemptSysMsgRedeliverySpec.scala | 13 +-- .../RemoteRestartedQuarantinedSpec.scala | 5 - .../main/scala/akka/remote/RemoteDaemon.scala | 62 ++++++---- .../akka/remote/artery/ArteryTransport.scala | 43 ++++--- .../akka/remote/artery/Association.scala | 18 ++- .../scala/akka/remote/artery/BufferPool.scala | 11 ++ .../scala/akka/remote/artery/Codecs.scala | 107 +++++++++++++++--- .../artery/InboundControlJunctionSpec.scala | 2 +- .../remote/artery/InboundHandshakeSpec.scala | 2 +- .../remote/artery/RemoteDeploymentSpec.scala | 3 +- .../akka/remote/artery/RemoteRouterSpec.scala | 19 +--- .../artery/SystemMessageDeliverySpec.scala | 2 +- .../akka/remote/artery/TestContext.scala | 2 +- 13 files changed, 195 insertions(+), 94 deletions(-) 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 f0a5327991..35badd1cc0 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 @@ -37,13 +37,12 @@ class AttemptSysMsgRedeliveryMultiJvmNode2 extends AttemptSysMsgRedeliverySpec( class AttemptSysMsgRedeliveryMultiJvmNode3 extends AttemptSysMsgRedeliverySpec( new AttemptSysMsgRedeliveryMultiJvmSpec(artery = false)) -// FIXME this test is failing for Artery, a DeathWatchNotification is not delivered as expected? -//class ArteryAttemptSysMsgRedeliveryMultiJvmNode1 extends AttemptSysMsgRedeliverySpec( -// new AttemptSysMsgRedeliveryMultiJvmSpec(artery = true)) -//class ArteryAttemptSysMsgRedeliveryMultiJvmNode2 extends AttemptSysMsgRedeliverySpec( -// new AttemptSysMsgRedeliveryMultiJvmSpec(artery = true)) -//class ArteryAttemptSysMsgRedeliveryMultiJvmNode3 extends AttemptSysMsgRedeliverySpec( -// new AttemptSysMsgRedeliveryMultiJvmSpec(artery = true)) +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 { 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 index 980aa51908..f34fae4742 100644 --- 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 @@ -82,11 +82,6 @@ abstract class RemoteRestartedQuarantinedSpec runOn(first) { val secondAddress = node(second).address - // FIXME this should not be needed, see issue #20566 - within(30.seconds) { - identifyWithUid(second, "subject", 1.seconds) - } - val (uid, ref) = identifyWithUid(second, "subject", 5.seconds) enterBarrier("before-quarantined") 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/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 9edaa01bf2..fdeafef12d 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -79,14 +79,14 @@ import akka.util.OptionVal */ private[akka] object InboundEnvelope { def apply( - recipient: InternalActorRef, + recipient: OptionVal[InternalActorRef], recipientAddress: Address, message: AnyRef, - senderOption: OptionVal[ActorRef], + sender: OptionVal[ActorRef], originUid: Long, association: OptionVal[OutboundContext]): InboundEnvelope = { val env = new ReusableInboundEnvelope - env.init(recipient, recipientAddress, message, senderOption, originUid, association) + env.init(recipient, recipientAddress, message, sender, originUid, association) env } @@ -96,31 +96,33 @@ private[akka] object InboundEnvelope { * INTERNAL API */ private[akka] trait InboundEnvelope { - def recipient: InternalActorRef + def recipient: OptionVal[InternalActorRef] def recipientAddress: Address def message: AnyRef - def senderOption: OptionVal[ActorRef] + def sender: OptionVal[ActorRef] def originUid: Long def association: OptionVal[OutboundContext] def withMessage(message: AnyRef): InboundEnvelope + + def withRecipient(ref: InternalActorRef): InboundEnvelope } /** * INTERNAL API */ private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { - private var _recipient: InternalActorRef = null + private var _recipient: OptionVal[InternalActorRef] = OptionVal.None private var _recipientAddress: Address = null private var _message: AnyRef = null - private var _senderOption: OptionVal[ActorRef] = OptionVal.None + private var _sender: OptionVal[ActorRef] = OptionVal.None private var _originUid: Long = 0L private var _association: OptionVal[OutboundContext] = OptionVal.None - override def recipient: InternalActorRef = _recipient + override def recipient: OptionVal[InternalActorRef] = _recipient override def recipientAddress: Address = _recipientAddress override def message: AnyRef = _message - override def senderOption: OptionVal[ActorRef] = _senderOption + override def sender: OptionVal[ActorRef] = _sender override def originUid: Long = _originUid override def association: OptionVal[OutboundContext] = _association @@ -129,32 +131,37 @@ private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { this } + def withRecipient(ref: InternalActorRef): InboundEnvelope = { + _recipient = OptionVal(ref) + this + } + def clear(): Unit = { - _recipient = null + _recipient = OptionVal.None _recipientAddress = null _message = null - _senderOption = OptionVal.None + _sender = OptionVal.None _originUid = 0L _association = OptionVal.None } def init( - recipient: InternalActorRef, + recipient: OptionVal[InternalActorRef], recipientAddress: Address, message: AnyRef, - senderOption: OptionVal[ActorRef], + sender: OptionVal[ActorRef], originUid: Long, association: OptionVal[OutboundContext]): Unit = { _recipient = recipient _recipientAddress = recipientAddress _message = message - _senderOption = senderOption + _sender = sender _originUid = originUid _association = association } override def toString: String = - s"InboundEnvelope($recipient, $recipientAddress, $message, $senderOption, $originUid, $association)" + s"InboundEnvelope($recipient, $recipientAddress, $message, $sender, $originUid, $association)" } /** @@ -647,7 +654,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R override def sendControl(to: Address, message: ControlMessage) = association(to).sendControl(message) - override def send(message: Any, senderOption: OptionVal[ActorRef], recipient: RemoteActorRef): Unit = { + override def send(message: Any, sender: OptionVal[ActorRef], recipient: RemoteActorRef): Unit = { val cached = recipient.cachedAssociation val a = @@ -658,7 +665,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R a2 } - a.send(message, senderOption, recipient) + a.send(message, sender, recipient) } override def association(remoteAddress: Address): Association = @@ -719,7 +726,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R flightRecorder.createEventSink())) val messageDispatcherSink: Sink[InboundEnvelope, Future[Done]] = Sink.foreach[InboundEnvelope] { m ⇒ - messageDispatcher.dispatch(m.recipient, m.recipientAddress, m.message, m.senderOption) + messageDispatcher.dispatch(m.recipient.get, m.recipientAddress, m.message, m.sender) inboundEnvelopePool.release(m) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index ca90c9a366..c6d27f4c85 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -37,6 +37,7 @@ import akka.util.{ Unsafe, WildcardTree } import org.agrona.concurrent.ManyToOneConcurrentArrayQueue import akka.util.OptionVal import akka.remote.QuarantinedEvent +import akka.remote.DaemonMsgCreate /** * INTERNAL API @@ -159,20 +160,31 @@ private[remote] class Association( override def sendControl(message: ControlMessage): Unit = outboundControlIngress.sendControlMessage(message) - def send(message: Any, senderOption: OptionVal[ActorRef], recipient: RemoteActorRef): Unit = { + def send(message: Any, sender: OptionVal[ActorRef], recipient: RemoteActorRef): Unit = { // allow ActorSelectionMessage to pass through quarantine, to be able to establish interaction with new system // FIXME where is that ActorSelectionMessage check in old remoting? if (message.isInstanceOf[ActorSelectionMessage] || !associationState.isQuarantined() || message == ClearSystemMessageDelivery) { // FIXME: Use a different envelope than the old Send, but make sure the new is handled by deadLetters properly message match { case _: SystemMessage | ClearSystemMessageDelivery ⇒ - val send = Send(message, senderOption, recipient, None) + val send = Send(message, sender, recipient, None) if (!controlQueue.offer(send)) { quarantine(reason = s"Due to overflow of control queue, size [$controlQueueSize]") transport.system.deadLetters ! send } + 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 send1 = Send(message, sender, recipient, None) + if (!controlQueue.offer(send1)) + transport.system.deadLetters ! send1 + val send2 = Send(message, sender, recipient, None) + if (!queue.offer(send2)) + transport.system.deadLetters ! send2 case _ ⇒ - val send = Send(message, senderOption, recipient, None) + val send = Send(message, sender, recipient, None) val offerOk = if (largeMessageChannelEnabled && isLargeMessageDestination(recipient)) largeQueue.offer(send) diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index 8842d548b6..b649fd71ae 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -99,6 +99,9 @@ sealed trait HeaderBuilder { def setNoSender(): Unit def isNoSender: Boolean + def setNoRecipient(): Unit + def isNoRecipient: Boolean + def recipientActorRef_=(ref: String): Unit def recipientActorRef: String @@ -147,6 +150,14 @@ private[remote] final class HeaderBuilderImpl(val compressionTable: LiteralCompr } } + def setNoRecipient(): Unit = { + _recipientActorRef = null + _recipientActorRefIdx = EnvelopeBuffer.DeadLettersCode + } + + def isNoRecipient: Boolean = + (_recipientActorRef eq null) && _recipientActorRefIdx == EnvelopeBuffer.DeadLettersCode + def recipientActorRef_=(ref: String): Unit = { _recipientActorRef = ref _recipientActorRefIdx = compressionTable.compressActorRef(ref) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index db77bc75ae..614a3f92fa 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -1,5 +1,6 @@ package akka.remote.artery +import scala.concurrent.duration._ import scala.util.control.NonFatal import akka.actor.{ ActorRef, InternalActorRef } import akka.actor.ActorSystem @@ -11,9 +12,13 @@ import akka.serialization.{ Serialization, SerializationExtension } import akka.stream._ import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler } import akka.util.OptionVal +import akka.actor.EmptyLocalActorRef +import akka.stream.stage.TimerGraphStageLogic -// TODO: Long UID -class Encoder( +/** + * INTERNAL API + */ +private[remote] class Encoder( uniqueLocalAddress: UniqueAddress, system: ActorSystem, compressionTable: LiteralCompressionTable, @@ -104,7 +109,20 @@ class Encoder( } } -class Decoder( +/** + * INTERNAL API + */ +private[remote] object Decoder { + private final case class RetryResolveRemoteDeployedRecipient( + attemptsLeft: Int, + recipientPath: String, + inboundEnvelope: InboundEnvelope) +} + +/** + * INTERNAL API + */ +private[remote] class Decoder( inboundContext: InboundContext, system: ExtendedActorSystem, resolveActorRefWithLocalAddress: String ⇒ InternalActorRef, @@ -116,7 +134,8 @@ class Decoder( val shape: FlowShape[EnvelopeBuffer, InboundEnvelope] = FlowShape(in, out) override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = - new GraphStageLogic(shape) with InHandler with OutHandler with StageLogging { + new TimerGraphStageLogic(shape) with InHandler with OutHandler with StageLogging { + import Decoder.RetryResolveRemoteDeployedRecipient private val localAddress = inboundContext.localAddress.address private val headerBuilder = HeaderBuilder(compressionTable) private val serialization = SerializationExtension(system) @@ -124,6 +143,9 @@ class Decoder( private val recipientCache = new java.util.HashMap[String, InternalActorRef] private val senderCache = new java.util.HashMap[String, ActorRef] + private val retryResolveRemoteDeployedRecipientInterval = 50.millis + private val retryResolveRemoteDeployedRecipientAttempts = 20 + override protected def logSource = classOf[Decoder] override def onPush(): Unit = { @@ -133,18 +155,8 @@ class Decoder( // FIXME: Instead of using Strings, the headerBuilder should automatically return cached ActorRef instances // in case of compression is enabled // FIXME: Is localAddress really needed? - val recipient: InternalActorRef = recipientCache.get(headerBuilder.recipientActorRef) match { - case null ⇒ - val ref = resolveActorRefWithLocalAddress(headerBuilder.recipientActorRef) - // FIXME we might need an efficient LRU cache, or replaced by compression table - if (recipientCache.size() >= 1000) - recipientCache.clear() - recipientCache.put(headerBuilder.recipientActorRef, ref) - ref - case ref ⇒ ref - } - val senderOption = + val sender = if (headerBuilder.isNoSender) OptionVal.None else { @@ -160,6 +172,12 @@ class Decoder( } } + val recipient = + if (headerBuilder.isNoRecipient) + OptionVal.None + else + resolveRecipient(headerBuilder.recipientActorRef) + val originUid = headerBuilder.uid val association = inboundContext.association(originUid) @@ -172,11 +190,18 @@ class Decoder( recipient, localAddress, // FIXME: Is this needed anymore? What should we do here? deserializedMessage, - senderOption, + sender, originUid, association) - push(out, decoded) + 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 + scheduleOnce(RetryResolveRemoteDeployedRecipient( + retryResolveRemoteDeployedRecipientAttempts, + headerBuilder.recipientActorRef, decoded), retryResolveRemoteDeployedRecipientInterval) + } else + push(out, decoded) } catch { case NonFatal(e) ⇒ log.warning( @@ -188,8 +213,56 @@ class Decoder( } } + private def resolveRecipient(path: String): OptionVal[InternalActorRef] = { + recipientCache.get(path) match { + case null ⇒ + def addToCache(resolved: InternalActorRef): Unit = { + // FIXME we might need an efficient LRU cache, or replaced by compression table + if (recipientCache.size() >= 1000) + recipientCache.clear() + recipientCache.put(path, resolved) + } + + resolveActorRefWithLocalAddress(path) match { + case empty: EmptyLocalActorRef ⇒ + val pathElements = empty.path.elements + if (pathElements.nonEmpty && pathElements.head == "remote") + OptionVal.None + else { + addToCache(empty) + OptionVal(empty) + } + case ref ⇒ + addToCache(ref) + OptionVal(ref) + } + case ref ⇒ OptionVal(ref) + } + } + override def onPull(): Unit = pull(in) + override protected def onTimer(timerKey: Any): Unit = { + timerKey match { + case RetryResolveRemoteDeployedRecipient(attemptsLeft, recipientPath, inboundEnvelope) ⇒ + resolveRecipient(recipientPath) match { + case OptionVal.None ⇒ + if (attemptsLeft > 0) + scheduleOnce(RetryResolveRemoteDeployedRecipient( + attemptsLeft - 1, + headerBuilder.recipientActorRef, inboundEnvelope), retryResolveRemoteDeployedRecipientInterval) + else { + val recipient = resolveActorRefWithLocalAddress(recipientPath) + // only retry for the first message + recipientCache.put(recipientPath, recipient) + push(out, inboundEnvelope.withRecipient(recipient)) + } + case OptionVal.Some(recipient) ⇒ + push(out, inboundEnvelope.withRecipient(recipient)) + } + } + } + setHandlers(in, out, this) } } diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala index 5afbdcec07..203169817c 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala @@ -39,7 +39,7 @@ class InboundControlJunctionSpec extends AkkaSpec with ImplicitSender { "be emitted via side channel" in { val observerProbe = TestProbe() val inboundContext = new TestInboundContext(localAddress = addressB) - val recipient = null.asInstanceOf[InternalActorRef] // not used + val recipient = OptionVal.None // not used val ((upstream, controlSubject), downstream) = TestSource.probe[AnyRef] .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, OptionVal.None, addressA.uid, OptionVal.None)) diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala index fb369d4bd4..ac7b62529b 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala @@ -39,7 +39,7 @@ class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { val addressB = UniqueAddress(Address("artery", "sysB", "hostB", 1002), 2) private def setupStream(inboundContext: InboundContext, timeout: FiniteDuration = 5.seconds): (TestPublisher.Probe[AnyRef], TestSubscriber.Probe[Any]) = { - val recipient = null.asInstanceOf[InternalActorRef] // not used + val recipient = OptionVal.None // not used TestSource.probe[AnyRef] .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, OptionVal.None, addressA.uid, inboundContext.association(addressA.uid))) diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala index c314cd5e0a..004547b081 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala @@ -59,8 +59,7 @@ class RemoteDeploymentSpec extends AkkaSpec(""" "Remoting" must { - // FIXME this test is failing with Artery - "create and supervise children on remote node" ignore { + "create and supervise children on remote node" in { val senderProbe = TestProbe()(masterSystem) val r = masterSystem.actorOf(Props[Echo1], "blub") r.path.toString should ===(s"artery://${system.name}@localhost:${port}/remote/artery/${masterSystem.name}@localhost:${masterPort}/user/blub") diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala index 6ce9ba3004..4d3380d03d 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala @@ -22,7 +22,6 @@ object RemoteRouterSpec { } class RemoteRouterSpec extends AkkaSpec(""" - akka.loglevel=DEBUG akka.actor.provider = remote akka.remote.artery.enabled = on akka.remote.artery.hostname = localhost @@ -99,8 +98,7 @@ class RemoteRouterSpec extends AkkaSpec(""" "A Remote Router" must { - // FIXME this test is failing with Artery - "deploy its children on remote host driven by configuration" ignore { + "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) @@ -112,8 +110,7 @@ class RemoteRouterSpec extends AkkaSpec(""" masterSystem.stop(router) } - // FIXME this test is failing with Artery - "deploy its children on remote host driven by programatic definition" ignore { + "deploy its children on remote host driven by programatic definition" in { val probe = TestProbe()(masterSystem) val router = masterSystem.actorOf(new RemoteRouterConfig( RoundRobinPool(2), @@ -126,8 +123,7 @@ class RemoteRouterSpec extends AkkaSpec(""" masterSystem.stop(router) } - // FIXME this test is failing with Artery - "deploy dynamic resizable number of children on remote host driven by configuration" ignore { + "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) @@ -152,8 +148,7 @@ class RemoteRouterSpec extends AkkaSpec(""" masterSystem.stop(router) } - // FIXME this test is failing with Artery - "deploy remote routers based on explicit deployment" ignore { + "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"artery://${sysName}@localhost:${port}")))), "remote-blub2") @@ -168,8 +163,7 @@ class RemoteRouterSpec extends AkkaSpec(""" masterSystem.stop(router) } - // FIXME this test is failing with Artery - "let remote deployment be overridden by local configuration" ignore { + "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"artery://${sysName}@localhost:${port}")))), "local-blub") @@ -214,8 +208,7 @@ class RemoteRouterSpec extends AkkaSpec(""" masterSystem.stop(router) } - // FIXME this test is failing with Artery - "set supplied supervisorStrategy" ignore { + "set supplied supervisorStrategy" in { val probe = TestProbe()(masterSystem) val escalator = OneForOneStrategy() { case e ⇒ probe.ref ! e; SupervisorStrategy.Escalate diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index b964a1c741..36b8340a4b 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -73,7 +73,7 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi } private def inbound(inboundContext: InboundContext): Flow[Send, InboundEnvelope, NotUsed] = { - val recipient = null.asInstanceOf[InternalActorRef] // not used + val recipient = OptionVal.None // not used Flow[Send] .map { case Send(sysEnv: SystemMessageEnvelope, _, _, _) ⇒ diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala index 94763cda5c..34371d7673 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -85,7 +85,7 @@ private[akka] class TestOutboundContext( override def sendControl(message: ControlMessage) = { controlProbe.foreach(_ ! message) - controlSubject.sendControl(InboundEnvelope(null, remoteAddress, message, OptionVal.None, localAddress.uid, + controlSubject.sendControl(InboundEnvelope(OptionVal.None, remoteAddress, message, OptionVal.None, localAddress.uid, OptionVal.None)) } From 3eceb241e12a104609cfbc32e435f31e4ef1cd3a Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 10 Jun 2016 07:41:36 +0200 Subject: [PATCH 072/186] make cpu vs latency configurable, #20625 * the actual default values will be measured and tuned later --- .../artery/AeronStreamConcistencySpec.scala | 3 +- .../artery/AeronStreamLatencySpec.scala | 8 +++-- .../artery/AeronStreamMaxThroughputSpec.scala | 3 +- .../akka/remote/artery/LatencySpec.scala | 1 + akka-remote/src/main/resources/reference.conf | 8 +++++ .../scala/akka/remote/RemoteSettings.scala | 2 ++ .../scala/akka/remote/artery/AeronSink.scala | 4 +-- .../akka/remote/artery/AeronSource.scala | 4 +-- .../akka/remote/artery/ArteryTransport.scala | 23 +++++++++++++- .../scala/akka/remote/artery/TaskRunner.scala | 30 +++++++++++++++---- .../akka/remote/artery/AeronSinkSpec.scala | 3 +- .../akka/remote/artery/AeronStreamsApp.scala | 3 +- 12 files changed, 75 insertions(+), 17 deletions(-) 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 index bdd82c4061..3521768011 100644 --- 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 @@ -62,8 +62,9 @@ abstract class AeronStreamConsistencySpec 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]) + val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem], idleCpuLevel) r.start() r } 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 index 8c9c504fe5..9dcf1c7167 100644 --- 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 @@ -52,7 +52,10 @@ object AeronStreamLatencySpec extends MultiNodeConfig { serialize-creators = false serialize-messages = false } - remote.artery.enabled = off + remote.artery { + enabled = off + advanced.idle-cpu-level=8 + } } """))) @@ -91,8 +94,9 @@ abstract class AeronStreamLatencySpec 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]) + val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem], idleCpuLevel) r.start() r } 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 index eca9bf7af4..27689d59d3 100644 --- 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 @@ -95,8 +95,9 @@ abstract class AeronStreamMaxThroughputSpec 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]) + val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem], idleCpuLevel) r.start() r } 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 index 1ee34ee518..a6030f71e1 100644 --- 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 @@ -53,6 +53,7 @@ object LatencySpec extends MultiNodeConfig { } remote.artery { enabled = on + advanced.idle-cpu-level=8 } } """))) diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 3c88823a5e..7ed83e7c06 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -126,6 +126,14 @@ akka { # Embedded media driver will use a this directory, or a temporary directory if this # property is not defined (empty). aeron-dir = "" + + # 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 } } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index 3eb9efdb80..9493dcb7df 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -30,6 +30,8 @@ final class RemoteSettings(val config: Config) { val AeronDirectoryName = getString("akka.remote.artery.advanced.aeron-dir") requiring (dir ⇒ EmbeddedMediaDriver || dir.nonEmpty, "aeron-dir must be defined when using external media driver") val TestMode: Boolean = getBoolean("akka.remote.artery.advanced.test-mode") + val IdleCpuLevel: Int = getInt("akka.remote.artery.advanced.idle-cpu-level").requiring(level ⇒ + 1 <= level && level <= 10, "idle-cpu-level must be between 1 and 10") val LogReceive: Boolean = getBoolean("akka.remote.log-received-messages") diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala index ee7adaa1e2..a580e39834 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala @@ -77,8 +77,7 @@ class AeronSink( taskRunner: TaskRunner, pool: EnvelopeBufferPool, giveUpSendAfter: Duration, - flightRecorder: EventSink -) + flightRecorder: EventSink) extends GraphStageWithMaterializedValue[SinkShape[EnvelopeBuffer], Future[Done]] { import AeronSink._ import TaskRunner._ @@ -96,6 +95,7 @@ class AeronSink( private var completedValue: Try[Done] = Success(Done) + // FIXME measure and adjust with IdleCpuLevel private val spinning = 1000 private var backoffCount = spinning private var lastMsgSize = 0 diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala index 4f2947bc5e..93f1dbbe23 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala @@ -67,8 +67,7 @@ class AeronSource( aeron: Aeron, taskRunner: TaskRunner, pool: EnvelopeBufferPool, - flightRecorder: EventSink -) + flightRecorder: EventSink) extends GraphStage[SourceShape[EnvelopeBuffer]] { import AeronSource._ import TaskRunner._ @@ -81,6 +80,7 @@ class AeronSource( new GraphStageLogic(shape) with OutHandler { private val sub = aeron.addSubscription(channel, streamId) + // FIXME measure and adjust with IdleCpuLevel private val spinning = 1000 private val yielding = 0 private val parking = 0 diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index fdeafef12d..a58f88ab2d 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -73,6 +73,9 @@ import scala.collection.JavaConverters._ import akka.stream.ActorMaterializerSettings import scala.annotation.tailrec import akka.util.OptionVal +import io.aeron.driver.ThreadingMode +import org.agrona.concurrent.BackoffIdleStrategy +import org.agrona.concurrent.BusySpinIdleStrategy /** * INTERNAL API @@ -362,7 +365,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val controlStreamId = 1 private val ordinaryStreamId = 3 private val largeStreamId = 4 - private val taskRunner = new TaskRunner(system) + private val taskRunner = new TaskRunner(system, remoteSettings.IdleCpuLevel) private val restartTimeout: FiniteDuration = 5.seconds // FIXME config private val maxRestarts = 5 // FIXME config @@ -431,9 +434,27 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R if (remoteSettings.AeronDirectoryName.nonEmpty) driverContext.aeronDirectoryName(remoteSettings.AeronDirectoryName) // FIXME settings from config + driverContext.conductorIdleStrategy() driverContext.clientLivenessTimeoutNs(SECONDS.toNanos(20)) driverContext.imageLivenessTimeoutNs(SECONDS.toNanos(20)) driverContext.driverTimeoutMs(SECONDS.toNanos(20)) + + if (remoteSettings.IdleCpuLevel == 10) { + driverContext + .threadingMode(ThreadingMode.DEDICATED) + .conductorIdleStrategy(new BackoffIdleStrategy(1, 1, 1, 1)) + .receiverIdleStrategy(new BusySpinIdleStrategy) + .senderIdleStrategy(new BusySpinIdleStrategy); + } else if (remoteSettings.IdleCpuLevel == 1) { + driverContext + .threadingMode(ThreadingMode.SHARED) + //FIXME measure: .sharedIdleStrategy(new BackoffIdleStrategy(20, 50, 1, 200)) + } else if (remoteSettings.IdleCpuLevel <= 5) { + driverContext + .threadingMode(ThreadingMode.SHARED_NETWORK) + //FIXME measure: .sharedNetworkIdleStrategy(new BackoffIdleStrategy(20, 50, 1, 20 * (11 - remoteSettings.IdleCpuLevel))) + } + val driver = MediaDriver.launchEmbedded(driverContext) log.debug("Started embedded media driver in directory [{}]", driver.aeronDirectoryName) topLevelFREvents.loFreq(Transport_MediaDriverStarted, driver.aeronDirectoryName().getBytes("US-ASCII")) diff --git a/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala b/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala index 307399d23c..6778260759 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala @@ -11,6 +11,9 @@ import akka.event.Logging import org.agrona.concurrent.BackoffIdleStrategy import scala.annotation.tailrec import scala.reflect.ClassTag +import org.agrona.concurrent.IdleStrategy +import org.agrona.concurrent.BusySpinIdleStrategy +import akka.dispatch.MonitorableThreadFactory /** * INTERNAL API @@ -82,7 +85,7 @@ private[akka] object TaskRunner { /** * INTERNAL API */ -private[akka] class TaskRunner(system: ExtendedActorSystem) extends Runnable { +private[akka] class TaskRunner(system: ExtendedActorSystem, idleCpuLevel: Int) extends Runnable { import TaskRunner._ private val log = Logging(system, getClass) @@ -91,14 +94,29 @@ private[akka] class TaskRunner(system: ExtendedActorSystem) extends Runnable { private[this] val tasks = new ArrayBag[Task] // TODO the backoff strategy should be measured and tuned - private val spinning = 2000000 - private val yielding = 0 - private val idleStrategy = new BackoffIdleStrategy( - spinning, yielding, TimeUnit.MICROSECONDS.toNanos(1), TimeUnit.MICROSECONDS.toNanos(100)) + private val idleStrategy: IdleStrategy = { + if (idleCpuLevel == 1) { + val maxParkMicros = 400 + new BackoffIdleStrategy(1, 1, TimeUnit.MICROSECONDS.toNanos(1), TimeUnit.MICROSECONDS.toNanos(maxParkMicros)) + } else if (idleCpuLevel == 10) + new BusySpinIdleStrategy + else { + val spinning = 100000 * idleCpuLevel + val yielding = 2 * idleCpuLevel + val maxParkMicros = 40 * (11 - idleCpuLevel) + new BackoffIdleStrategy( + spinning, yielding, TimeUnit.MICROSECONDS.toNanos(1), TimeUnit.MICROSECONDS.toNanos(maxParkMicros)) + } + } private var reset = false def start(): Unit = { - val thread = system.threadFactory.newThread(this) + val tf = system.threadFactory match { + case m: MonitorableThreadFactory ⇒ + m.withName(m.name + "-taskrunner") + case other ⇒ other + } + val thread = tf.newThread(this) thread.start() } diff --git a/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala index 0500299727..368537ee55 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala @@ -32,8 +32,9 @@ class AeronSinkSpec extends AkkaSpec with ImplicitSender { Aeron.connect(ctx) } + val idleCpuLevel = 5 val taskRunner = { - val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem]) + val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem], idleCpuLevel) r.start() r } diff --git a/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala b/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala index c9f94b99ce..6a8bf49089 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala @@ -78,8 +78,9 @@ object AeronStreamsApp { lazy val system = ActorSystem("AeronStreams") lazy implicit val mat = ActorMaterializer()(system) + val idleCpuLevel = 5 lazy val taskRunner = { - val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem]) + val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem], idleCpuLevel) r.start() r } From c8eadd7d08ea376d222689fabf1081b98979e69e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andre=CC=81n?= Date: Wed, 8 Jun 2016 10:04:30 +0200 Subject: [PATCH 073/186] Porting RemotingSpec to artery #20714 --- .../main/scala/akka/actor/ActorSystem.scala | 8 +- .../scala/akka/remote/artery/Codecs.scala | 6 +- .../remote/artery/ArteryMultiNodeSpec.scala | 68 +++++++ .../artery/LargeMessagesStreamSpec.scala | 168 +++++++----------- .../remote/artery/RemoteActorForSpec.scala | 130 ++++++++++++++ .../artery/RemoteActorRefProviderSpec.scala | 34 +--- .../artery/RemoteActorSelectionSpec.scala | 154 ++++++++++++++++ .../remote/artery/RemoteConnectionSpec.scala | 85 +++++++++ .../remote/artery/RemoteDeploymentSpec.scala | 6 +- .../remote/artery/RemoteFailureSpec.scala | 58 ++++++ .../RemoteMessageSerializationSpec.scala | 117 ++++++++++++ .../akka/remote/artery/RemoteRouterSpec.scala | 4 +- .../artery/RemoteSendConsistencySpec.scala | 3 +- .../remote/artery/RemoteWatcherSpec.scala | 2 +- .../artery/SerializationErrorSpec.scala | 3 +- .../artery/SystemMessageDeliverySpec.scala | 9 +- .../akka/remote/artery/UntrustedSpec.scala | 3 +- 17 files changed, 710 insertions(+), 148 deletions(-) create mode 100644 akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/RemoteActorForSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/RemoteConnectionSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/RemoteFailureSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 15f190bbfd..bd9afb83a2 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -168,11 +168,11 @@ object ActorSystem { final val ConfigVersion: String = getString("akka.version") final val ProviderClass: String = getString("akka.actor.provider") match { - case "local" => classOf[LocalActorRefProvider].getName + 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 + case "remote" ⇒ "akka.remote.RemoteActorRefProvider" + case "cluster" ⇒ "akka.cluster.ClusterActorRefProvider" + case fqcn ⇒ fqcn } final val SupervisorStrategyClass: String = getString("akka.actor.guardian-supervisor-strategy") diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 614a3f92fa..0104daa779 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -5,7 +5,7 @@ import scala.util.control.NonFatal import akka.actor.{ ActorRef, InternalActorRef } import akka.actor.ActorSystem import akka.actor.ExtendedActorSystem -import akka.remote.{ MessageSerializer, UniqueAddress } +import akka.remote.{ MessageSerializer, OversizedPayloadException, UniqueAddress } import akka.remote.EndpointManager.Send import akka.remote.artery.SystemMessageDelivery.SystemMessageEnvelope import akka.serialization.{ Serialization, SerializationExtension } @@ -95,6 +95,10 @@ private[remote] class Encoder( case _: SystemMessageEnvelope ⇒ log.error(e, "Failed to serialize system message [{}].", send.message.getClass.getName) throw e + case _ if e.isInstanceOf[java.nio.BufferOverflowException] ⇒ + val reason = new OversizedPayloadException(s"Discarding oversized payload sent to ${send.recipient}: max allowed size ${envelope.byteBuffer.limit()} bytes. Message type [${send.message.getClass.getName}].") + log.error(reason, "Transient association error (association remains live)") + pull(in) case _ ⇒ log.error(e, "Failed to serialize message [{}].", send.message.getClass.getName) pull(in) 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..4640772ec8 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala @@ -0,0 +1,68 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.actor.{ ActorSystem, ExtendedActorSystem, RootActorPath } +import akka.remote.RARP +import akka.testkit.AkkaSpec +import com.typesafe.config.{ Config, ConfigFactory } + +object ArteryMultiNodeSpec { + def defaultConfig = + ConfigFactory.parseString(""" + akka { + actor.provider = remote + actor.warn-about-java-serializer-usage = off + remote.artery { + enabled = on + hostname = localhost + port = 0 + } + } + """) +} + +/** + * 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 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 + } + + override def afterTermination(): Unit = { + remoteSystems.foreach(sys ⇒ shutdown(sys)) + remoteSystems = Vector.empty + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala index d52e28da67..1eaaa5d103 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala @@ -3,13 +3,10 @@ */ package akka.remote.artery -import akka.actor.{ Actor, ActorRef, ActorSelection, ActorSystem, ExtendedActorSystem, Props, RootActorPath } -import akka.remote.{ LargeDestination, RegularDestination, RemoteActorRef } -import akka.testkit.{ SocketUtil, TestKit, TestProbe } +import akka.actor.{ Actor, ActorRef, ActorSelection, Props, RootActorPath } +import akka.remote.{ LargeDestination, RARP, RegularDestination, RemoteActorRef } +import akka.testkit.TestProbe import akka.util.ByteString -import com.typesafe.config.ConfigFactory -import org.scalatest.concurrent.ScalaFutures -import org.scalatest.{ ShouldMatchers, WordSpec } import scala.concurrent.Await import scala.concurrent.duration._ @@ -24,130 +21,99 @@ object LargeMessagesStreamSpec { } } -class LargeMessagesStreamSpec extends WordSpec with ShouldMatchers with ScalaFutures { +class LargeMessagesStreamSpec extends ArteryMultiNodeSpec( + """ + akka { + loglevel = ERROR + remote.artery.large-message-destinations = [ "/user/large" ] + } + """.stripMargin) { + import LargeMessagesStreamSpec._ - val config = ConfigFactory.parseString( - s""" - akka { - loglevel = ERROR - actor { - provider = remote - } - remote.artery { - enabled = on - hostname = localhost - port = 0 - large-message-destinations = [ - "/user/large" - ] - } - } - - """) - "The large message support" should { "not affect regular communication" in { - val systemA = ActorSystem("systemA", config) - val systemB = ActorSystem("systemB", config) + val systemA = localSystem + val systemB = newRemoteSystem() - try { - val senderProbeA = TestProbe()(systemA) - val senderProbeB = TestProbe()(systemB) + 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)) + // 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 addressB = systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress - val rootB = RootActorPath(addressB) - val regularRemote = awaitResolve(systemA.actorSelection(rootB / "user" / "regular")) - regularRemote.tell(Ping(), senderProbeA.ref) - senderProbeA.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].cachedLargeMessageDestinationFlag should ===(RegularDestination) + // flag should be cached now + regularRemote.asInstanceOf[RemoteActorRef].cachedLargeMessageDestinationFlag should ===(RegularDestination) - } finally { - TestKit.shutdownActorSystem(systemA) - TestKit.shutdownActorSystem(systemB) - } } "pass small regular messages over the large-message stream" in { - val systemA = ActorSystem("systemA", config) - val systemB = ActorSystem("systemB", config) + val systemA = localSystem + val systemB = newRemoteSystem() - try { - val senderProbeA = TestProbe()(systemA) - val senderProbeB = TestProbe()(systemB) + 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)) + // 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 = systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress - val rootB = RootActorPath(addressB) - val largeRemote = awaitResolve(systemA.actorSelection(rootB / "user" / "large")) - largeRemote.tell(Ping(), senderProbeA.ref) - senderProbeA.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].cachedLargeMessageDestinationFlag should ===(LargeDestination) + // flag should be cached now + largeRemote.asInstanceOf[RemoteActorRef].cachedLargeMessageDestinationFlag should ===(LargeDestination) - } finally { - TestKit.shutdownActorSystem(systemA) - TestKit.shutdownActorSystem(systemB) - } } "allow for normal communication while simultaneously sending large messages" in { - val systemA = ActorSystem("systemA", config) - val systemB = ActorSystem("systemB", config) + val systemA = localSystem + val systemB = newRemoteSystem() - try { + val senderProbeB = TestProbe()(systemB) - 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)) - // 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)) - 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")) - // both up and running, resolve remote refs - val addressB = systemB.asInstanceOf[ExtendedActorSystem].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) - // 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) - 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)) - // 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].cachedLargeMessageDestinationFlag should ===(LargeDestination) - regularRemote.asInstanceOf[RemoteActorRef].cachedLargeMessageDestinationFlag should ===(RegularDestination) - - } finally { - TestKit.shutdownActorSystem(systemA) - TestKit.shutdownActorSystem(systemB) - } + // cached flags should be set now + largeRemote.asInstanceOf[RemoteActorRef].cachedLargeMessageDestinationFlag should ===(LargeDestination) + regularRemote.asInstanceOf[RemoteActorRef].cachedLargeMessageDestinationFlag should ===(RegularDestination) } } 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..80536c106e --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorForSpec.scala @@ -0,0 +1,130 @@ +/** + * 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) +} + +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"artery://${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("artery://nonexistingsystem@localhost:12346/user/echo") ! "ping" + } + } + + "support ask" in { + remoteSystem.actorOf(TestActors.echoActorProps, "support-ask") + val remoteRef = localSystem.actorFor(s"artery://${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"artery://${remoteSystem.name}@localhost:$remotePort/dead-letters-on-remote") ! "buh" + }(remoteSystem) + } + + // FIXME can't communicate with new ref looked up after starting a new instance (!?!) + "not send to remote re-created actor with same name" ignore { + + def lookItUp() = localSystem.actorFor(s"artery://${remoteSystem.name}@localhost:$remotePort/user/re-created") + + val echo1 = remoteSystem.actorOf(TestActors.echoActorProps, "re-created") + val remoteRef1 = lookItUp() + remoteRef1 ! 2 + expectMsg(2) + + // now stop and start a new actor with the same name + watch(echo1) + remoteSystem.stop(echo1) + expectTerminated(echo1) + + val echo2 = remoteSystem.actorOf(TestActors.echoActorProps, "re-created") + val remoteRef2 = lookItUp() + remoteRef2 ! 2 + expectMsg(2) + + // the old ref should not interact with the + // new actor instance at the same path + remoteRef1 ! 3 + expectNoMsg(1.second) + + // and additionally, but it would have failed already + // if this wasn't true + remoteRef1.path.uid should !==(remoteRef2.path.uid) + } + + // 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 index fcfe56ba6f..5410f0e0d2 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala @@ -3,41 +3,19 @@ */ package akka.remote.artery -import scala.concurrent.duration._ -import akka.actor.{ ActorIdentity, ActorSystem, ExtendedActorSystem, Identify, RootActorPath } -import akka.testkit.{ AkkaSpec, ImplicitSender } -import akka.testkit.TestActors -import com.typesafe.config.ConfigFactory -import akka.testkit.EventFilter -import akka.actor.InternalActorRef +import akka.actor.{ EmptyLocalActorRef, InternalActorRef } import akka.remote.RemoteActorRef -import akka.actor.EmptyLocalActorRef +import akka.testkit.{ EventFilter, TestActors } -object RemoteActorRefProviderSpec { +class RemoteActorRefProviderSpec extends ArteryMultiNodeSpec { - val config = ConfigFactory.parseString(s""" - akka { - actor.provider = remote - remote.artery.enabled = on - remote.artery.hostname = localhost - remote.artery.port = 0 - } - """) - -} - -class RemoteActorRefProviderSpec extends AkkaSpec(RemoteActorRefProviderSpec.config) with ImplicitSender { - import RemoteActorRefProviderSpec._ - - val addressA = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + val addressA = address(localSystem) system.actorOf(TestActors.echoActorProps, "echo") - val systemB = ActorSystem("systemB", system.settings.config) - val addressB = systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + val systemB = newRemoteSystem() + val addressB = address(systemB) systemB.actorOf(TestActors.echoActorProps, "echo") - override def afterTermination(): Unit = shutdown(systemB) - "RemoteActorRefProvider" must { "resolve local actor selection" in { 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..e5ea82fd13 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala @@ -0,0 +1,154 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.actor.{ Actor, ActorIdentity, 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._ + +object RemoteActorSelectionSpec { + final case class ActorSelReq(s: String) + + class SelectionActor extends Actor { + def receive = { + // if we get props and a name, create a child, send ref back + case (p: Props, n: String) ⇒ 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._ + + val systemB = system + + val systemA = { + val remotePort = port(systemB) + val remoteSysName = systemB.name + + val localSysName = "local-" + remoteSysName + val localPort = SocketUtil.temporaryServerAddress(udp = true).getPort + + // nesting the hierarchy across the two systems + newRemoteSystem(Some(s""" + akka { + remote.artery.port = $localPort + actor.deployment { + /looker2/child.remote = "artery://$remoteSysName@localhost:$remotePort" + /looker2/child/grandchild.remote = "artery://$localSysName@localhost:$localPort" + } + } + """)) + } + + "Remote actor selection" should { + + // TODO would like to split up in smaller cases but find it hard + // TODO fails with "received Supervise from unregistered child" when looker2/child is created - akka/akka#20715 + "select actors across node boundaries" ignore { + + val localLooker2 = systemA.actorOf(selectionActorProps, "looker2") + + // child is configured to be deployed on remoteSystem + localLooker2 ! ((selectionActorProps, "child")) + val remoteChild = expectMsgType[ActorRef] + + // grandchild is configured to be deployed on local system but from remote system + remoteChild ! ((selectionActorProps, "grandchild")) + val localGrandchild = expectMsgType[ActorRef] + localGrandchild.asInstanceOf[ActorRefScope].isLocal should ===(true) + localGrandchild ! 53 + expectMsg(53) + + val localGrandchildSelection = systemA.actorSelection(system / "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)) + + systemA.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 + + system.actorSelection(system / "looker2" / "child") ! ActorSelReq("..") + expectMsgType[ActorSelection] ! Identify(None) + expectMsgType[ActorIdentity].ref.get should be theSameInstanceAs localLooker2 + + localGrandchild ! ((TestActors.echoActorProps, "grandgrandchild")) + val grandgrandchild = expectMsgType[ActorRef] + + system.actorSelection("/user/looker2/child") ! Identify("idReq1") + expectMsg(ActorIdentity("idReq1", Some(remoteChild))) + system.actorSelection(remoteChild.path) ! Identify("idReq2") + expectMsg(ActorIdentity("idReq2", Some(remoteChild))) + system.actorSelection("/user/looker2/*") ! Identify("idReq3") + expectMsg(ActorIdentity("idReq3", Some(remoteChild))) + + system.actorSelection("/user/looker2/child/grandchild") ! Identify("idReq4") + expectMsg(ActorIdentity("idReq4", Some(localGrandchild))) + system.actorSelection(remoteChild.path / "grandchild") ! Identify("idReq5") + expectMsg(ActorIdentity("idReq5", Some(localGrandchild))) + system.actorSelection("/user/looker2/*/grandchild") ! Identify("idReq6") + expectMsg(ActorIdentity("idReq6", Some(localGrandchild))) + system.actorSelection("/user/looker2/child/*") ! Identify("idReq7") + expectMsg(ActorIdentity("idReq7", Some(localGrandchild))) + system.actorSelection(remoteChild.path / "*") ! Identify("idReq8") + expectMsg(ActorIdentity("idReq8", Some(localGrandchild))) + + system.actorSelection("/user/looker2/child/grandchild/grandgrandchild") ! Identify("idReq9") + expectMsg(ActorIdentity("idReq9", Some(grandgrandchild))) + system.actorSelection(remoteChild.path / "grandchild" / "grandgrandchild") ! Identify("idReq10") + expectMsg(ActorIdentity("idReq10", Some(grandgrandchild))) + system.actorSelection("/user/looker2/child/*/grandgrandchild") ! Identify("idReq11") + expectMsg(ActorIdentity("idReq11", Some(grandgrandchild))) + system.actorSelection("/user/looker2/child/*/*") ! Identify("idReq12") + expectMsg(ActorIdentity("idReq12", Some(grandgrandchild))) + system.actorSelection(remoteChild.path / "*" / "grandgrandchild") ! Identify("idReq13") + expectMsg(ActorIdentity("idReq13", Some(grandgrandchild))) + + val sel1 = system.actorSelection("/user/looker2/child/grandchild/grandgrandchild") + system.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))) + system.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) + system.actorSelection(system / "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..168e67ad0a --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteConnectionSpec.scala @@ -0,0 +1,85 @@ +/** + * 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"artery://$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.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"artery://$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.port=$remotePort")) + + muteSystem(remoteSystem) + localProbe.expectNoMsg(2.seconds) + val otherProbe = new TestProbe(remoteSystem) + val otherSender = otherProbe.ref + val thisSelection = remoteSystem.actorSelection(s"artery://${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/RemoteDeploymentSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala index 004547b081..c4bbf74cb4 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala @@ -10,7 +10,7 @@ import akka.actor._ import akka.remote.routing._ import com.typesafe.config._ import akka.testkit.TestActors.echoActorProps -import akka.remote.RemoteScope +import akka.remote.{ RARP, RemoteScope } object RemoteDeploymentSpec { class Echo1 extends Actor { @@ -42,7 +42,7 @@ class RemoteDeploymentSpec extends AkkaSpec(""" import RemoteDeploymentSpec._ - val port = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress.port.get + val port = RARP(system).provider.getDefaultAddress.port.get val conf = ConfigFactory.parseString( s""" akka.actor.deployment { @@ -51,7 +51,7 @@ class RemoteDeploymentSpec extends AkkaSpec(""" """).withFallback(system.settings.config) val masterSystem = ActorSystem("Master" + system.name, conf) - val masterPort = masterSystem.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress.port.get + val masterPort = RARP(masterSystem).provider.getDefaultAddress.port.get override def afterTermination(): Unit = { shutdown(masterSystem) 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..d0ed9e5a93 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala @@ -0,0 +1,117 @@ +/** + * 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 + } + } + val maxPayloadBytes = ArteryTransport.MaximumFrameSize +} + +class RemoteMessageSerializationSpec extends ArteryMultiNodeSpec(""" + akka.actor.serialize-messages = off + akka.actor.serialize-creators = off + """) with ImplicitSender { + + import RemoteMessageSerializationSpec._ + + 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](pattern = ".*Discarding oversized payload sent.*", 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"artery://${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 index 4d3380d03d..1f724b3fea 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala @@ -10,7 +10,7 @@ import akka.actor._ import akka.remote.routing._ import com.typesafe.config._ import akka.testkit.TestActors.echoActorProps -import akka.remote.RemoteScope +import akka.remote.{ RARP, RemoteScope } object RemoteRouterSpec { class Parent extends Actor { @@ -43,7 +43,7 @@ class RemoteRouterSpec extends AkkaSpec(""" import RemoteRouterSpec._ - val port = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress.port.get + val port = RARP(system).provider.getDefaultAddress.port.get val sysName = system.name val conf = ConfigFactory.parseString( s""" diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala index 3abbbb7309..a800d86dbc 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala @@ -8,6 +8,7 @@ import akka.actor.{ Actor, ActorIdentity, ActorSystem, Deploy, ExtendedActorSyst import akka.testkit.{ AkkaSpec, ImplicitSender } import com.typesafe.config.ConfigFactory import akka.actor.Actor.Receive +import akka.remote.RARP object RemoteSendConsistencySpec { @@ -25,7 +26,7 @@ object RemoteSendConsistencySpec { class RemoteSendConsistencySpec extends AkkaSpec(RemoteSendConsistencySpec.config) with ImplicitSender { val systemB = ActorSystem("systemB", system.settings.config) - val addressB = systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + val addressB = RARP(systemB).provider.getDefaultAddress println(addressB) val rootB = RootActorPath(addressB) diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala index 7f4027f62a..ede7c7f62e 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala @@ -79,7 +79,7 @@ class RemoteWatcherSpec extends AkkaSpec( override def expectedTestDuration = 2.minutes val remoteSystem = ActorSystem("RemoteSystem", system.settings.config) - val remoteAddress = remoteSystem.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + val remoteAddress = RARP(remoteSystem).provider.getDefaultAddress def remoteAddressUid = AddressUidExtension(remoteSystem).addressUid Seq(system, remoteSystem).foreach(muteDeadLetters( diff --git a/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala index d997cf08fd..a555c51fc6 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala @@ -5,6 +5,7 @@ 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 @@ -40,7 +41,7 @@ class SerializationErrorSpec extends AkkaSpec(SerializationErrorSpec.config) wit """).withFallback(system.settings.config) val systemB = ActorSystem("systemB", configB) systemB.actorOf(TestActors.echoActorProps, "echo") - val addressB = systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + val addressB = RARP(systemB).provider.getDefaultAddress val rootB = RootActorPath(addressB) override def afterTermination(): Unit = shutdown(systemB) diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index 36b8340a4b..5df50600d1 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -4,6 +4,7 @@ package akka.remote.artery import java.util.concurrent.ThreadLocalRandom + import scala.concurrent.Await import scala.concurrent.duration._ import akka.NotUsed @@ -14,10 +15,8 @@ import akka.actor.Identify import akka.actor.InternalActorRef import akka.actor.PoisonPill import akka.actor.RootActorPath -import akka.remote.AddressUidExtension +import akka.remote.{ AddressUidExtension, RARP, RemoteActorRef, UniqueAddress } 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 @@ -52,11 +51,11 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi import SystemMessageDeliverySpec._ val addressA = UniqueAddress( - system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress, + RARP(system).provider.getDefaultAddress, AddressUidExtension(system).addressUid) val systemB = ActorSystem("systemB", system.settings.config) val addressB = UniqueAddress( - systemB.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress, + RARP(systemB).provider.getDefaultAddress, AddressUidExtension(systemB).addressUid) val rootB = RootActorPath(addressB.address) val matSettings = ActorMaterializerSettings(system).withFuzzing(true) diff --git a/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala index 9b353a3be8..837ffd7dbe 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala @@ -23,6 +23,7 @@ import akka.testkit.TestProbe import akka.actor.ActorSelection import akka.testkit.TestEvent import akka.event.Logging +import akka.remote.RARP import akka.testkit.EventFilter object UntrustedSpec { @@ -77,7 +78,7 @@ class UntrustedSpec extends AkkaSpec(""" akka.remote.artery.hostname = localhost akka.remote.artery.port = 0 """)) - val addr = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + val addr = RARP(system).provider.getDefaultAddress val receptionist = system.actorOf(Props(classOf[Receptionist], testActor), "receptionist") From 19ce6711560c338e948b68e580abe0df4fa32c2f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andre=CC=81n?= Date: Fri, 10 Jun 2016 16:59:44 +0200 Subject: [PATCH 074/186] Remote actor selection test cleaned up, but still not passing --- .../artery/RemoteActorSelectionSpec.scala | 107 ++++++++++-------- 1 file changed, 57 insertions(+), 50 deletions(-) diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala index e5ea82fd13..8be763ad6f 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala @@ -3,7 +3,7 @@ */ package akka.remote.artery -import akka.actor.{ Actor, ActorIdentity, ActorRef, ActorRefScope, ActorSelection, ActorSystem, ExtendedActorSystem, Identify, PoisonPill, Props, Terminated } +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 @@ -12,13 +12,17 @@ import scala.concurrent.duration._ object RemoteActorSelectionSpec { final case class ActorSelReq(s: String) + final case class ActorCreateReq(props: Props, name: String) - class SelectionActor extends Actor { + 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 (p: Props, n: String) ⇒ sender() ! context.actorOf(p, n) + 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) + case ActorSelReq(s) ⇒ sender() ! context.actorSelection(s) } } def selectionActorProps = Props(new SelectionActor) @@ -28,47 +32,50 @@ class RemoteActorSelectionSpec extends ArteryMultiNodeSpec with ImplicitSender { import RemoteActorSelectionSpec._ - val systemB = system - - val systemA = { - val remotePort = port(systemB) - val remoteSysName = systemB.name - - val localSysName = "local-" + remoteSysName - val localPort = SocketUtil.temporaryServerAddress(udp = true).getPort - - // nesting the hierarchy across the two systems - newRemoteSystem(Some(s""" - akka { - remote.artery.port = $localPort - actor.deployment { - /looker2/child.remote = "artery://$remoteSysName@localhost:$remotePort" - /looker2/child/grandchild.remote = "artery://$localSysName@localhost:$localPort" - } - } - """)) - } - "Remote actor selection" should { - // TODO would like to split up in smaller cases but find it hard - // TODO fails with "received Supervise from unregistered child" when looker2/child is created - akka/akka#20715 + // TODO fails with not receiving the localGrandchild value, seems to go to dead letters "select actors across node boundaries" ignore { - val localLooker2 = systemA.actorOf(selectionActorProps, "looker2") + 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 = "artery://$remoteSysName@localhost:$remotePort" + /looker2/child/grandchild.remote = "artery://$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 ! ((selectionActorProps, "child")) + localLooker2 ! ActorCreateReq(selectionActorProps, "child") val remoteChild = expectMsgType[ActorRef] // grandchild is configured to be deployed on local system but from remote system - remoteChild ! ((selectionActorProps, "grandchild")) + remoteChild ! ActorCreateReq(selectionActorProps, "grandchild") val localGrandchild = expectMsgType[ActorRef] localGrandchild.asInstanceOf[ActorRefScope].isLocal should ===(true) localGrandchild ! 53 expectMsg(53) - val localGrandchildSelection = systemA.actorSelection(system / "looker2" / "child" / "grandchild") + val localGrandchildSelection = localSystem.actorSelection(localSystem / "looker2" / "child" / "grandchild") localGrandchildSelection ! 54 expectMsg(54) lastSender should ===(localGrandchild) @@ -77,51 +84,51 @@ class RemoteActorSelectionSpec extends ArteryMultiNodeSpec with ImplicitSender { val grandchild2 = expectMsgType[ActorIdentity].ref grandchild2 should ===(Some(localGrandchild)) - systemA.actorSelection("/user/looker2/child") ! Identify(None) + 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 - system.actorSelection(system / "looker2" / "child") ! ActorSelReq("..") + 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] - system.actorSelection("/user/looker2/child") ! Identify("idReq1") + localSystem.actorSelection("/user/looker2/child") ! Identify("idReq1") expectMsg(ActorIdentity("idReq1", Some(remoteChild))) - system.actorSelection(remoteChild.path) ! Identify("idReq2") + localSystem.actorSelection(remoteChild.path) ! Identify("idReq2") expectMsg(ActorIdentity("idReq2", Some(remoteChild))) - system.actorSelection("/user/looker2/*") ! Identify("idReq3") + localSystem.actorSelection("/user/looker2/*") ! Identify("idReq3") expectMsg(ActorIdentity("idReq3", Some(remoteChild))) - system.actorSelection("/user/looker2/child/grandchild") ! Identify("idReq4") + localSystem.actorSelection("/user/looker2/child/grandchild") ! Identify("idReq4") expectMsg(ActorIdentity("idReq4", Some(localGrandchild))) - system.actorSelection(remoteChild.path / "grandchild") ! Identify("idReq5") + localSystem.actorSelection(remoteChild.path / "grandchild") ! Identify("idReq5") expectMsg(ActorIdentity("idReq5", Some(localGrandchild))) - system.actorSelection("/user/looker2/*/grandchild") ! Identify("idReq6") + localSystem.actorSelection("/user/looker2/*/grandchild") ! Identify("idReq6") expectMsg(ActorIdentity("idReq6", Some(localGrandchild))) - system.actorSelection("/user/looker2/child/*") ! Identify("idReq7") + localSystem.actorSelection("/user/looker2/child/*") ! Identify("idReq7") expectMsg(ActorIdentity("idReq7", Some(localGrandchild))) - system.actorSelection(remoteChild.path / "*") ! Identify("idReq8") + localSystem.actorSelection(remoteChild.path / "*") ! Identify("idReq8") expectMsg(ActorIdentity("idReq8", Some(localGrandchild))) - system.actorSelection("/user/looker2/child/grandchild/grandgrandchild") ! Identify("idReq9") + localSystem.actorSelection("/user/looker2/child/grandchild/grandgrandchild") ! Identify("idReq9") expectMsg(ActorIdentity("idReq9", Some(grandgrandchild))) - system.actorSelection(remoteChild.path / "grandchild" / "grandgrandchild") ! Identify("idReq10") + localSystem.actorSelection(remoteChild.path / "grandchild" / "grandgrandchild") ! Identify("idReq10") expectMsg(ActorIdentity("idReq10", Some(grandgrandchild))) - system.actorSelection("/user/looker2/child/*/grandgrandchild") ! Identify("idReq11") + localSystem.actorSelection("/user/looker2/child/*/grandgrandchild") ! Identify("idReq11") expectMsg(ActorIdentity("idReq11", Some(grandgrandchild))) - system.actorSelection("/user/looker2/child/*/*") ! Identify("idReq12") + localSystem.actorSelection("/user/looker2/child/*/*") ! Identify("idReq12") expectMsg(ActorIdentity("idReq12", Some(grandgrandchild))) - system.actorSelection(remoteChild.path / "*" / "grandgrandchild") ! Identify("idReq13") + localSystem.actorSelection(remoteChild.path / "*" / "grandgrandchild") ! Identify("idReq13") expectMsg(ActorIdentity("idReq13", Some(grandgrandchild))) - val sel1 = system.actorSelection("/user/looker2/child/grandchild/grandgrandchild") - system.actorSelection(sel1.toSerializationFormat) ! Identify("idReq18") + val sel1 = localSystem.actorSelection("/user/looker2/child/grandchild/grandgrandchild") + localSystem.actorSelection(sel1.toSerializationFormat) ! Identify("idReq18") expectMsg(ActorIdentity("idReq18", Some(grandgrandchild))) remoteChild ! Identify("idReq14") @@ -134,7 +141,7 @@ class RemoteActorSelectionSpec extends ArteryMultiNodeSpec with ImplicitSender { val child2 = expectMsgType[ActorRef] child2 ! Identify("idReq15") expectMsg(ActorIdentity("idReq15", Some(child2))) - system.actorSelection(remoteChild.path) ! Identify("idReq16") + localSystem.actorSelection(remoteChild.path) ! Identify("idReq16") expectMsg(ActorIdentity("idReq16", Some(child2))) remoteChild ! Identify("idReq17") expectMsg(ActorIdentity("idReq17", None)) @@ -145,7 +152,7 @@ class RemoteActorSelectionSpec extends ArteryMultiNodeSpec with ImplicitSender { child2.path.uid should not be (remoteChild.path.uid) remoteChild ! 56 expectNoMsg(1.second) - system.actorSelection(system / "looker2" / "child") ! 57 + localSystem.actorSelection(localSystem / "looker2" / "child") ! 57 expectMsg(57) } From bdfbffcde53a7eed4cb12ef107b7742ceaee5dc4 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 2 Jun 2016 20:44:27 +0200 Subject: [PATCH 075/186] port remaining remote multi-node tests to Artery --- .../remote/AttemptSysMsgRedeliverySpec.scala | 13 ++-- .../akka/remote/RemoteDeliverySpec.scala | 32 +++++--- .../remote/RemoteNodeDeathWatchSpec.scala | 64 ++++++++++------ .../RemoteNodeRestartDeathWatchSpec.scala | 41 +++++----- .../RemoteNodeShutdownAndComesBackSpec.scala | 8 +- .../akka/remote/RemoteReDeploymentSpec.scala | 76 +++++++++++++------ .../RemoteRestartedQuarantinedSpec.scala | 19 +---- .../remote/routing/RemoteRandomSpec.scala | 45 +++++++---- .../remote/routing/RemoteRoundRobinSpec.scala | 56 +++++++++----- .../routing/RemoteScatterGatherSpec.scala | 40 ++++++---- 10 files changed, 239 insertions(+), 155 deletions(-) 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 35badd1cc0..b9abfe25a2 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 @@ -37,12 +37,13 @@ class AttemptSysMsgRedeliveryMultiJvmNode2 extends AttemptSysMsgRedeliverySpec( 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)) +// FIXME this is failing with Artery +//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 { 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..3e59ee2e87 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 + """))) +} +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,13 @@ 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) +abstract class RemoteDeliverySpec(multiNodeConfig: RemoteDeliveryConfig) + extends MultiNodeSpec(multiNodeConfig) with STMultiNodeSpec with ImplicitSender { - - import RemoteDeliveryMultiJvmSpec._ + import multiNodeConfig._ + import RemoteDeliverySpec._ override def initialParticipants = roles.size @@ -52,7 +60,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/RemoteNodeDeathWatchSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeDeathWatchSpec.scala index be2c79dd61..06dcfe4e8d 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 @@ -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 + """))) +} + +// 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,11 @@ 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) +abstract class RemoteNodeDeathWatchSpec(multiNodeConfig: RemoteNodeDeathWatchConfig) + extends MultiNodeSpec(multiNodeConfig) with STMultiNodeSpec with ImplicitSender { - - import RemoteNodeDeathWatchMultiJvmSpec._ + 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..7ac9d1c531 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,13 @@ 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) +abstract class RemoteNodeRestartDeathWatchSpec(multiNodeConfig: RemoteNodeRestartDeathWatchConfig) + extends MultiNodeSpec(multiNodeConfig) with STMultiNodeSpec with ImplicitSender { - - import RemoteNodeRestartDeathWatchMultiJvmSpec._ + import multiNodeConfig._ + import RemoteNodeRestartDeathWatchSpec._ override def initialParticipants = roles.size @@ -104,11 +113,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.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/RemoteNodeShutdownAndComesBackSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteNodeShutdownAndComesBackSpec.scala index b321462305..9102832593 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 @@ -135,11 +135,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.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..9be5c92b48 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 + """))) + 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,28 +105,7 @@ 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) +abstract class RemoteReDeploymentMultiJvmSpec(multiNodeConfig: RemoteReDeploymentConfig) extends MultiNodeSpec(multiNodeConfig) with STMultiNodeSpec with ImplicitSender { def sleepAfterKill: FiniteDuration @@ -93,6 +113,7 @@ abstract class RemoteReDeploymentMultiJvmSpec extends MultiNodeSpec(RemoteReDepl def initialParticipants = roles.size + import multiNodeConfig._ import RemoteReDeploymentMultiJvmSpec._ "A remote deployment target system" must { @@ -113,6 +134,11 @@ abstract class RemoteReDeploymentMultiJvmSpec extends MultiNodeSpec(RemoteReDepl enterBarrier("first-deployed") + // FIXME When running with Artery: + // [akka://RemoteReDeploymentMultiJvmSpec/user/parent] received Supervise from unregistered child + // Actor[artery://RemoteReDeploymentMultiJvmSpec@localhost:55627/remote/artery/RemoteReDeploymentMultiJvmSpec@localhost:65490/user/parent/hello#-370928728], + // this will not end well + runOn(first) { testConductor.blackhole(second, first, Both).await testConductor.shutdown(second, abort = true).await 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 index f34fae4742..8712d7cb03 100644 --- 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 @@ -30,20 +30,8 @@ object RemoteRestartedQuarantinedSpec extends MultiNodeConfig { ConfigFactory.parseString(""" akka.loglevel = WARNING akka.remote.log-remote-lifecycle-events = WARNING - - # Keep it long, we don't want reconnects - akka.remote.retry-gate-closed-for = 1 s - - # Important, otherwise it is very racy to get a non-writing endpoint: the only way to do it if the two nodes - # associate to each other at the same time. Setting this will ensure that the right scenario happens. - akka.remote.use-passive-connections = off - - # TODO should not be needed, but see TODO at the end of the test - akka.remote.transport-failure-detector.heartbeat-interval = 1 s - akka.remote.transport-failure-detector.acceptable-heartbeat-pause = 10 s - akka.remote.artery.enabled = on - """))) + """))) class Subject extends Actor { def receive = { @@ -134,10 +122,7 @@ abstract class RemoteRestartedQuarantinedSpec val probe = TestProbe()(freshSystem) freshSystem.actorSelection(RootActorPath(firstAddress) / "user" / "subject").tell(Identify("subject"), probe.ref) - // TODO sometimes it takes long time until the new connection is established, - // It seems like there must first be a transport failure detector timeout, that triggers - // "No response from remote. Handshake timed out or transport failure detector triggered". - probe.expectMsgType[ActorIdentity](30.second).ref should not be (None) + 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") 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..0b9331b8e4 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 @@ -14,21 +14,19 @@ 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 + """))) deployOnAll(""" /service-hello { @@ -39,14 +37,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) +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 MultiNodeSpec(multiNodeConfig) with STMultiNodeSpec with ImplicitSender with DefaultTimeout { - import RemoteRandomMultiJvmSpec._ + import multiNodeConfig._ + import RemoteRandomSpec._ def initialParticipants = roles.size @@ -87,7 +99,10 @@ class RemoteRandomSpec extends MultiNodeSpec(RemoteRandomMultiJvmSpec) // "Terminate" to a shut down node system.stop(actor) enterBarrier("done") + + // FIXME this test has problems shutting down actor system when running with Artery + // [akka.actor.ActorSystemImpl(RemoteRandomSpec)] Failed to stop [RemoteRandomSpec] within [5 seconds] } } } -} \ 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..6b1100008d 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 @@ -24,26 +24,19 @@ import akka.routing.Routee import akka.routing.FromConfig import akka.testkit._ import scala.concurrent.duration._ +import com.typesafe.config.ConfigFactory -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 + """))) deployOnAll(""" /service-hello { @@ -56,7 +49,7 @@ object RemoteRoundRobinMultiJvmSpec extends MultiNodeConfig { router = round-robin-pool target.nodes = ["@first@", "@second@", "@third@"] } - + /service-hello3 { router = round-robin-group routees.paths = [ @@ -67,14 +60,37 @@ 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) +// FIXME this test fails with Artery +// [akka://RemoteRoundRobinSpec/user/service-hello2] received Supervise from unregistered child +// Actor[artery://RemoteRoundRobinSpec@localhost:52247/remote/artery/RemoteRoundRobinSpec@localhost:56386/user/service-hello2/c2#-2080820302], +// this will not end well +//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 MultiNodeSpec(multiNodeConfig) with STMultiNodeSpec with ImplicitSender with DefaultTimeout { - import RemoteRoundRobinMultiJvmSpec._ + import multiNodeConfig._ + import RemoteRoundRobinSpec._ 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..351595d837 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 @@ -15,21 +15,19 @@ 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 + """))) deployOnAll(""" /service-hello { @@ -40,14 +38,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) +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 MultiNodeSpec(multiNodeConfig) with STMultiNodeSpec with ImplicitSender with DefaultTimeout { - import RemoteScatterGatherMultiJvmSpec._ + import multiNodeConfig._ + import RemoteScatterGatherSpec._ def initialParticipants = roles.size From 7ca0d53310b34b3f2b8afe500f2d96e51fea1c7a Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 8 Jun 2016 18:38:54 +0200 Subject: [PATCH 076/186] delete embedded media files on shutdown, #20779 --- .../akka/remote/artery/ArteryTransport.scala | 30 +++++++++++++++---- 1 file changed, 25 insertions(+), 5 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index a58f88ab2d..c166e170dd 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -76,6 +76,7 @@ import akka.util.OptionVal import io.aeron.driver.ThreadingMode import org.agrona.concurrent.BackoffIdleStrategy import org.agrona.concurrent.BusySpinIdleStrategy +import scala.util.control.NonFatal /** * INTERNAL API @@ -428,6 +429,10 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R log.info("Remoting started; listening on address: {}", defaultAddress) } + private lazy val stopMediaDriverShutdownHook = new Thread { + override def run(): Unit = stopMediaDriver() + } + private def startMediaDriver(): Unit = { if (remoteSettings.EmbeddedMediaDriver) { val driverContext = new MediaDriver.Context @@ -458,6 +463,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R val driver = MediaDriver.launchEmbedded(driverContext) log.debug("Started embedded media driver in directory [{}]", driver.aeronDirectoryName) topLevelFREvents.loFreq(Transport_MediaDriverStarted, driver.aeronDirectoryName().getBytes("US-ASCII")) + Runtime.getRuntime.addShutdownHook(stopMediaDriverShutdownHook) mediaDriver = Some(driver) } } @@ -467,6 +473,23 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R case None ⇒ remoteSettings.AeronDirectoryName } + private def stopMediaDriver(): Unit = { + mediaDriver.foreach { driver ⇒ + // this is only for embedded media driver + driver.close() + try { + // FIXME it should also be configurable to not delete dir + IoUtil.delete(new File(driver.aeronDirectoryName), false) + } catch { + case NonFatal(e) ⇒ + log.warning( + "Couldn't delete Aeron embedded media driver files in [{}] due to [{}]", + driver.aeronDirectoryName, e.getMessage) + } + } + Try(Runtime.getRuntime.removeShutdownHook(stopMediaDriverShutdownHook)) + } + // TODO: Add FR events private def startAeron(): Unit = { val ctx = new Aeron.Context @@ -642,11 +665,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R topLevelFREvents.loFreq(Transport_AeronErrorLogTaskStopped, NoMetaData) } if (aeron != null) aeron.close() - mediaDriver.foreach { driver ⇒ - // this is only for embedded media driver - driver.close() - // FIXME it should also be configurable to not delete dir - IoUtil.delete(new File(driver.aeronDirectoryName), true) + if (mediaDriver.isDefined) { + stopMediaDriver() topLevelFREvents.loFreq(Transport_MediaFileDeleted, NoMetaData) } topLevelFREvents.loFreq(Transport_FlightRecorderClose, NoMetaData) From e818887bb23ec49ea37c4bef42306e7a8ef44d84 Mon Sep 17 00:00:00 2001 From: Konrad Malawski Date: Thu, 23 Jun 2016 11:58:54 +0200 Subject: [PATCH 077/186] +art #20455 HeavyHitters, CountMinSketch => ActorRef Compression * +art #20455 HeavyHitters and CountMinSketch prepared for Compression * +art #20455 compression tables and integration specs --- .gitignore | 2 + .../scala/akka/io/DirectByteBufferPool.scala | 4 +- .../main/scala/akka/routing/MurmurHash.scala | 10 +- .../scala/akka/util/PrettyByteString.scala | 44 +++ .../akka/remote/artery/CodecBenchmark.scala | 30 +- .../remote/artery/FlightRecorderBench.scala | 3 +- .../compress/HeavyHittersBenchmark.scala | 110 ++++++ .../OutboundCompressionTableBenchmark.scala | 60 +++ .../scala/akka/stream/FlowMapBenchmark.scala | 2 +- .../http/impl/engine/client/PoolGateway.scala | 2 +- .../remote/artery/MaxThroughputSpec.scala | 1 - .../artery/compress/CountMinSketch.java | 342 ++++++++++++++++++ akka-remote/src/main/resources/reference.conf | 35 +- .../akka/remote/RemoteActorRefProvider.scala | 9 +- .../scala/akka/remote/RemoteSettings.scala | 3 + .../akka/remote/artery/AeronSource.scala | 2 +- .../akka/remote/artery/ArteryTransport.scala | 150 ++++---- .../akka/remote/artery/Association.scala | 80 +++- .../scala/akka/remote/artery/BufferPool.scala | 94 +++-- .../scala/akka/remote/artery/Codecs.scala | 149 +++----- .../akka/remote/artery/Compression.scala | 20 - .../remote/artery/FlightRecorderEvents.scala | 6 + .../scala/akka/remote/artery/Handshake.scala | 8 +- .../remote/artery/NoLiteralCompression.scala | 38 ++ .../compress/ActualCompressionTables.scala | 102 ++++++ .../compress/AdvertiseCompressionId.scala | 11 + .../artery/compress/CompressionProtocol.scala | 45 +++ .../artery/compress/CompressionSettings.scala | 36 ++ .../compress/InboundCompressionTable.scala | 161 +++++++++ .../OutboundActorRefCompressionTable.scala | 125 +++++++ .../artery/compress/TopHeavyHitters.scala | 166 +++++++++ .../remote/artery/EnvelopeBufferSpec.scala | 80 ++-- .../remote/artery/HandshakeRetrySpec.scala | 4 +- .../remote/artery/OutboundHandshakeSpec.scala | 2 +- .../akka/remote/artery/TestContext.scala | 4 +- .../compress/CompressionIntegrationSpec.scala | 94 +++++ .../compress/CompressionTestUtils.scala | 17 + ...dshakeShouldDropCompressionTableSpec.scala | 128 +++++++ .../artery/compress/HeavyHittersSpec.scala | 101 ++++++ .../InboundCompressionTableSpec.scala | 62 ++++ .../OutboundCompressionTableSpec.scala | 81 +++++ .../scala/akka/stream/impl/ConstantFun.scala | 1 + .../main/scala/akka/testkit/TestActors.scala | 10 + project/plugins.sbt | 4 +- 44 files changed, 2124 insertions(+), 314 deletions(-) create mode 100644 akka-actor/src/main/scala/akka/util/PrettyByteString.scala create mode 100644 akka-bench-jmh/src/main/scala/akka/remote/compress/HeavyHittersBenchmark.scala create mode 100644 akka-bench-jmh/src/main/scala/akka/remote/compress/OutboundCompressionTableBenchmark.scala create mode 100644 akka-remote/src/main/java/akka/remote/artery/compress/CountMinSketch.java delete mode 100644 akka-remote/src/main/scala/akka/remote/artery/Compression.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/NoLiteralCompression.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressionTables.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/AdvertiseCompressionId.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/CompressionProtocol.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/CompressionSettings.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressionTable.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompressionTable.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/TopHeavyHitters.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTestUtils.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/compress/HeavyHittersSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/compress/InboundCompressionTableSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/compress/OutboundCompressionTableSpec.scala diff --git a/.gitignore b/.gitignore index 8e716eb2d4..cb0c517fff 100755 --- a/.gitignore +++ b/.gitignore @@ -1,4 +1,6 @@ *# +*.log +*.orig *.iml *.ipr *.iws diff --git a/akka-actor/src/main/scala/akka/io/DirectByteBufferPool.scala b/akka-actor/src/main/scala/akka/io/DirectByteBufferPool.scala index fdde031dda..3f68e35ba9 100644 --- a/akka-actor/src/main/scala/akka/io/DirectByteBufferPool.scala +++ b/akka-actor/src/main/scala/akka/io/DirectByteBufferPool.scala @@ -83,10 +83,10 @@ private[akka] class DirectByteBufferPool(defaultBufferSize: Int, maxPoolEntries: */ private final def tryCleanDirectByteBuffer(toBeDestroyed: ByteBuffer): Unit = try { if (toBeDestroyed.isDirect) { - val cleanerMethod = toBeDestroyed.getClass().getMethod("cleaner") + val cleanerMethod = toBeDestroyed.getClass.getMethod("cleaner") cleanerMethod.setAccessible(true) val cleaner = cleanerMethod.invoke(toBeDestroyed) - val cleanMethod = cleaner.getClass().getMethod("clean") + val cleanMethod = cleaner.getClass.getMethod("clean") cleanMethod.setAccessible(true) cleanMethod.invoke(cleaner) } 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/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-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala index b02ca87672..9a1da92bd6 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -3,18 +3,21 @@ */ 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.ActorSystem -import akka.actor.ExtendedActorSystem -import akka.actor.InternalActorRef -import akka.actor.Props -import akka.actor.RootActorPath +import akka.actor._ import akka.remote.AddressUidExtension import akka.remote.EndpointManager.Send import akka.remote.RARP @@ -56,8 +59,9 @@ class CodecBenchmark { 16, create = () ⇒ new ReusableInboundEnvelope, clear = inEnvelope ⇒ inEnvelope.asInstanceOf[ReusableInboundEnvelope].clear() ) - val compression = new Compression(system) - val headerIn = HeaderBuilder(compression) + + val compressionOut = NoOutboundCompression + val headerIn = HeaderBuilder.in(NoopInboundCompression) val envelopeTemplateBuffer = ByteBuffer.allocate(ArteryTransport.MaximumFrameSize).order(ByteOrder.LITTLE_ENDIAN) val uniqueLocalAddress = UniqueAddress( @@ -102,8 +106,8 @@ class CodecBenchmark { headerIn.version = 1 headerIn.uid = 42 headerIn.serializer = 4 - headerIn.senderActorRef = senderStringA - headerIn.recipientActorRef = recipientStringB + headerIn.senderActorRef = actorOnSystemA + headerIn.recipientActorRef = remoteRefB headerIn.manifest = "" envelope.writeHeader(headerIn) envelope.byteBuffer.put(payload) @@ -136,7 +140,7 @@ class CodecBenchmark { val N = 100000 val encoder: Flow[Send, EnvelopeBuffer, NotUsed] = - Flow.fromGraph(new Encoder(uniqueLocalAddress, system, compression, envelopePool)) + Flow.fromGraph(new Encoder(uniqueLocalAddress, system, compressionOut, envelopePool)) Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) .map(_ ⇒ Send(payload, OptionVal.None, remoteRefB, None)) @@ -165,7 +169,7 @@ class CodecBenchmark { val decoder: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = Flow.fromGraph(new Decoder(inboundContext, system.asInstanceOf[ExtendedActorSystem], - resolveActorRefWithLocalAddress, compression, envelopePool, inboundEnvelopePool)) + resolveActorRefWithLocalAddress, NoopInboundCompression, envelopePool, inboundEnvelopePool)) Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) .map { _ => @@ -193,7 +197,7 @@ class CodecBenchmark { val N = 100000 val encoder: Flow[Send, EnvelopeBuffer, NotUsed] = - Flow.fromGraph(new Encoder(uniqueLocalAddress, system, compression, envelopePool)) + Flow.fromGraph(new Encoder(uniqueLocalAddress, system, compressionOut, envelopePool)) val localRecipient = resolvedRef.path.toSerializationFormatWithAddress(uniqueLocalAddress.address) val provider = RARP(system).provider @@ -206,7 +210,7 @@ class CodecBenchmark { val decoder: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = Flow.fromGraph(new Decoder(inboundContext, system.asInstanceOf[ExtendedActorSystem], - resolveActorRefWithLocalAddress, compression, envelopePool, inboundEnvelopePool)) + resolveActorRefWithLocalAddress, NoopInboundCompression, envelopePool, inboundEnvelopePool)) Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) .map(_ ⇒ Send(payload, OptionVal.None, remoteRefB, None)) 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 index 9b286f34c0..be6cdfcb67 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/FlightRecorderBench.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/FlightRecorderBench.scala @@ -6,8 +6,9 @@ package akka.remote.artery import java.io.File import java.nio.channels.FileChannel import java.nio.file.StandardOpenOption -import java.util.concurrent.CountDownLatch +import java.util.concurrent.{ CountDownLatch, TimeUnit } import java.util.concurrent.TimeUnit + import org.openjdk.jmh.annotations.{ OperationsPerInvocation, _ } @State(Scope.Benchmark) 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..4297dbb171 --- /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 + + 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 = 0 + while (i < 8192) { + blackhole.consume(topN.update("HEAVY_HITTER", 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/remote/compress/OutboundCompressionTableBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/remote/compress/OutboundCompressionTableBenchmark.scala new file mode 100644 index 0000000000..7f7da57b01 --- /dev/null +++ b/akka-bench-jmh/src/main/scala/akka/remote/compress/OutboundCompressionTableBenchmark.scala @@ -0,0 +1,60 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.compress + +import java.util.Random + +import akka.actor.{ ActorSystem, Address } +import akka.event.NoLogging +import akka.remote.artery.compress.{ OutboundCompressionTable, TopHeavyHitters } +import org.openjdk.jmh.annotations._ +import org.openjdk.jmh.infra.Blackhole + +@State(Scope.Benchmark) +@BenchmarkMode(Array(Mode.Throughput)) +@Fork(2) +class OutboundCompressionTableBenchmark { + + @Param(Array("512", "8192")) + var registered: Int = 0 + + implicit val system = ActorSystem("TestSystem") + + var outgoingCompression: OutboundCompressionTable[String] = _ + + val rand = new Random(1001021) + + var preallocatedNums: Array[Long] = _ + var preallocatedStrings: Array[String] = _ + + var i = 0 + + @Setup + def init(): Unit = { + preallocatedNums = Array.ofDim(registered) + preallocatedStrings = Array.ofDim(8192) + + outgoingCompression = new OutboundCompressionTable(system, Address("akka", "remote-system")) + + var i = 0 + while (i < registered) { + outgoingCompression.register(i.toString, i) + preallocatedNums(i) = rand.nextLong() + preallocatedStrings(i) = i.toString + i += 1 + } + } + + // @Benchmark + // @BenchmarkMode(Array(Mode.SingleShotTime)) + // def registerThenCompress(): Int = { + // outgoingCompression.register("new", i) + // outgoingCompression.compress("new") + // } + + @Benchmark + def compressKnown(): Int = + outgoingCompression.compress("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-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-remote-tests/src/multi-jvm/scala/akka/remote/artery/MaxThroughputSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/MaxThroughputSpec.scala index 65f966091d..9e081ff638 100644 --- 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 @@ -6,7 +6,6 @@ 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._ 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..577b8718c2 --- /dev/null +++ b/akka-remote/src/main/java/akka/remote/artery/compress/CountMinSketch.java @@ -0,0 +1,342 @@ +/* + * Copyright (C) 2009-2016 Lightbend Inc. + */ + +package akka.remote.artery.compress; + +import akka.actor.Actor; +import akka.actor.ActorRef; + +import java.io.UnsupportedEncodingException; +import java.util.Random; + +/** + * INTERNAL API: Count-Min Sketch datastructure. + * + * 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 { + + public static final long PRIME_MODULUS = (1L << 31) - 1; + + private int depth; + private int width; + private long[][] table; + private long[] hashA; + private long size; + private double eps; + private double confidence; + + public CountMinSketch(int depth, int width, int seed) { + this.depth = depth; + this.width = width; + this.eps = 2.0 / width; + this.confidence = 1 - 1 / Math.pow(2, depth); + initTablesWith(depth, width, seed); + } + + public CountMinSketch(double epsOfTotalCount, double confidence, int seed) { + // 2/w = eps ; w = 2/eps + // 1/2^depth <= 1-confidence ; depth >= -log2 (1-confidence) + this.eps = epsOfTotalCount; + this.confidence = confidence; + this.width = (int) Math.ceil(2 / epsOfTotalCount); + this.depth = (int) Math.ceil(-Math.log(1 - confidence) / Math.log(2)); + initTablesWith(depth, width, seed); + } + + CountMinSketch(int depth, int width, int size, long[] hashA, long[][] table) { + this.depth = depth; + this.width = width; + this.eps = 2.0 / width; + this.confidence = 1 - 1 / Math.pow(2, depth); + this.hashA = hashA; + this.table = table; + this.size = size; + } + + private void initTablesWith(int depth, int width, int seed) { + this.table = new long[depth][width]; + this.hashA = new long[depth]; + Random r = new Random(seed); + // We're using a linear hash functions + // of the form (a*x+b) mod p. + // a,b are chosen independently for each hash function. + // However we can set b = 0 as all it does is shift the results + // without compromising their uniformity or independence with + // the other hashes. + for (int i = 0; i < depth; ++i) { + hashA[i] = r.nextInt(Integer.MAX_VALUE); + } + } + + /** Referred to as {@code epsilon} in the whitepaper */ + public double getRelativeError() { + return eps; + } + + public double getConfidence() { + return confidence; + } + + private int hash(long item, int i) { + long hash = hashA[i] * item; + // A super fast way of computing x mod 2^p-1 + // See http://www.cs.princeton.edu/courses/archive/fall09/cos521/Handouts/universalclasses.pdf + // page 149, right after Proposition 7. + hash += hash >> 32; + hash &= PRIME_MODULUS; + // Doing "%" after (int) conversion is ~2x faster than %'ing longs. + return ((int) hash) % width; + } + + public void add(long 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"); + } + for (int i = 0; i < depth; ++i) { + table[i][hash(item, i)] += count; + } + size += count; + } + + public void add(String 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"); + } + // TODO we could reuse the arrays + final int[] buckets = MurmurHash.hashBuckets(item, depth, width); // TODO replace with Scala's Murmur3, it's much faster + for (int i = 0; i < depth; ++i) { + table[i][buckets[i]] += count; + } + size += count; + } + + /** + * 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 addAndEstimateCount(String 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"); + } + final int[] buckets = MurmurHash.hashBuckets(item, depth, width); + for (int i = 0; i < depth; ++i) { + table[i][buckets[i]] += count; + } + size += count; + return estimateCount(buckets); + } + + public long size() { + return size; + } + + /** + * The estimate is correct within {@code 'epsilon' * (total item count)}, + * with probability {@code confidence}. + */ + public long estimateCount(long item) { + long res = Long.MAX_VALUE; + for (int i = 0; i < depth; ++i) { + res = Math.min(res, table[i][hash(item, i)]); + } + return res; + } + + /** + * The estimate is correct within {@code 'epsilon' * (total item count)}, + * with probability {@code confidence}. + */ + public long estimateCount(String item) { + int[] buckets = MurmurHash.hashBuckets(item, depth, width); + return estimateCount(buckets); + } + + /** + * 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; + } + + /** + * This is copied from https://github.com/addthis/stream-lib/blob/master/src/main/java/com/clearspring/analytics/hash/MurmurHash.java + * Which is Apache V2 licensed. + *

+ * This is a very fast, non-cryptographic hash suitable for general hash-based + * lookup. See http://murmurhash.googlepages.com/ for more details. + *

+ *

+ * The C version of MurmurHash 2.0 found at that site was ported to Java by + * Andrzej Bialecki (ab at getopt org). + *

+ */ + // TODO replace with Scala's Murmur3, it's much faster + private static class MurmurHash { + + public static int hash(Object o) { + if (o == null) { + return 0; + } + if (o instanceof String) { + return hash(((String) o).getBytes()); + } + // TODO consider calling hashCode on ActorRef here directly? It is just a random number though so possibly not as evenly distributed...? + if (o instanceof Long) { + return hashLong((Long) o); + } + if (o instanceof Integer) { + return hashLong((Integer) o); + } + if (o instanceof Double) { + return hashLong(Double.doubleToRawLongBits((Double) o)); + } + if (o instanceof Float) { + return hashLong(Float.floatToRawIntBits((Float) o)); + } + if (o instanceof byte[]) { + return hash((byte[]) o); + } + return hash(o.toString()); + } + + public static int hash(byte[] data) { + return hash(data, data.length, -1); + } + + public static int hash(byte[] data, int seed) { + return hash(data, data.length, seed); + } + + public static int hash(byte[] data, int length, int seed) { + int m = 0x5bd1e995; + int r = 24; + + int h = seed ^ length; + + int len_4 = length >> 2; + + for (int i = 0; i < len_4; i++) { + int i_4 = i << 2; + int k = data[i_4 + 3]; + k = k << 8; + k = k | (data[i_4 + 2] & 0xff); + k = k << 8; + k = k | (data[i_4 + 1] & 0xff); + k = k << 8; + k = k | (data[i_4 + 0] & 0xff); + k *= m; + k ^= k >>> r; + k *= m; + h *= m; + h ^= k; + } + + // avoid calculating modulo + int len_m = len_4 << 2; + int left = length - len_m; + + if (left != 0) { + if (left >= 3) { + h ^= (int) data[length - 3] << 16; + } + if (left >= 2) { + h ^= (int) data[length - 2] << 8; + } + if (left >= 1) { + h ^= (int) data[length - 1]; + } + + h *= m; + } + + h ^= h >>> 13; + h *= m; + h ^= h >>> 15; + + return h; + } + + public static int hashLong(long data) { + int m = 0x5bd1e995; + int r = 24; + + int h = 0; + + int k = (int) data * m; + k ^= k >>> r; + h ^= k * m; + + k = (int) (data >> 32) * m; + k ^= k >>> r; + h *= m; + h ^= k * m; + + h ^= h >>> 13; + h *= m; + h ^= h >>> 15; + + return h; + } + + // Murmur is faster than an SHA-based approach and provides as-good collision + // resistance. The combinatorial generation approach described in + // http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf + // does prove to work in actual tests, and is obviously faster + // than performing further iterations of murmur. + public static int[] hashBuckets(String key, int hashCount, int max) { + byte[] b; + try { + b = key.getBytes("UTF-16");// TODO Use the Unsafe trick @patriknw used to access the backing array directly -- via Endre + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + return hashBuckets(b, hashCount, max); + } + + static int[] hashBuckets(byte[] b, int hashCount, int max) { + // TODO we could reuse the arrays + int[] result = new int[hashCount]; + int hash1 = hash(b, b.length, 0); + int hash2 = hash(b, b.length, hash1); + for (int i = 0; i < hashCount; i++) { + result[i] = Math.abs((hash1 + i * hash2) % max); + } + return result; + } + } + + @Override + public String toString() { + return "CountMinSketch{" + + "confidence=" + confidence + + ", size=" + size + + ", depth=" + depth + + ", width=" + width + + '}'; + } +} diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 7ed83e7c06..d3425fbd81 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -110,8 +110,8 @@ akka { # 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} + # Settings for the materializer that is used for the remote streams. + materializer = ${akka.stream.materializer} materializer { dispatcher = "akka.remote.default-remote-dispatcher" } @@ -134,7 +134,38 @@ akka { # Level 1 strongly prefer low CPU consumption over low latency. # Level 10 strongly prefer low latency over low CPU consumption. idle-cpu-level = 5 + + # compression of common strings in remoting messages, like actor destinations, serializers etc + compression { + # global flag to disable all compression + enabled = off + + # unlocks additional very verbose debug logging of compression events (on DEBUG log level) + debug = off + + actor-refs { + enabled = off # TODO possibly remove on/off option once we have battle proven it? + + # 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 + } + manifests { + enabled = off # TODO possibly remove on/off option once we have battle proven it? + + # 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 + } + } } + } ### General settings diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 70bf13e572..9561ac1d55 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -217,7 +217,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 = @@ -361,10 +361,10 @@ 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) + if (hasAddress(address)) + local.resolveActorRef(rootGuardian, elems) else try { - new RemoteActorRef(transport, localAddress, RootActorPath(address) / elems, - Nobody, props = None, deploy = None) + 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) @@ -479,7 +479,6 @@ private[akka] class RemoteActorRef private[akka] ( 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 a separate stream for large messages diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index 9493dcb7df..77fd6b18cb 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -3,6 +3,7 @@ */ package akka.remote +import akka.remote.artery.compress.CompressionSettings import com.typesafe.config.Config import scala.concurrent.duration._ import akka.util.Timeout @@ -33,6 +34,8 @@ final class RemoteSettings(val config: Config) { val IdleCpuLevel: Int = getInt("akka.remote.artery.advanced.idle-cpu-level").requiring(level ⇒ 1 <= level && level <= 10, "idle-cpu-level must be between 1 and 10") + val ArteryCompressionSettings = CompressionSettings(getConfig("akka.remote.artery.advanced.compression")) + 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/artery/AeronSource.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala index 93f1dbbe23..629ddf5b66 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala @@ -43,7 +43,7 @@ object AeronSource { class MessageHandler(pool: EnvelopeBufferPool) { def reset(): Unit = messageReceived = null - var messageReceived: EnvelopeBuffer = null + private[remote] var messageReceived: EnvelopeBuffer = null // private to avoid scalac warning about exposing EnvelopeBuffer val fragmentsHandler = new Fragments(data ⇒ messageReceived = data, pool) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index c166e170dd..3cb59d8e97 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -3,12 +3,10 @@ */ package akka.remote.artery -import java.io.File -import java.net.InetSocketAddress -import java.nio.channels.DatagramChannel import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.TimeUnit -import java.util.concurrent.atomic.AtomicLong +import akka.remote.artery.compress.CompressionProtocol.CompressionMessage + import scala.collection.JavaConverters._ import scala.concurrent.Future import scala.concurrent.Promise @@ -18,11 +16,8 @@ import scala.util.Success import scala.util.Try import akka.Done import akka.NotUsed -import akka.actor.ActorRef -import akka.actor.Address +import akka.actor._ import akka.actor.Cancellable -import akka.actor.ExtendedActorSystem -import akka.actor.InternalActorRef import akka.event.Logging import akka.event.LoggingAdapter import akka.remote.AddressUidExtension @@ -40,9 +35,9 @@ import akka.remote.artery.InboundControlJunction.ControlMessageSubject import akka.remote.artery.OutboundControlJunction.OutboundControlIngress import akka.remote.transport.AkkaPduCodec import akka.remote.transport.AkkaPduProtobufCodec +import akka.remote.artery.compress.{ AdvertiseCompressionId, InboundCompressionImpl, CompressionProtocol } import akka.stream.AbruptTerminationException import akka.stream.ActorMaterializer -import akka.stream.ActorMaterializerSettings import akka.stream.KillSwitches import akka.stream.Materializer import akka.stream.SharedKillSwitch @@ -55,7 +50,6 @@ import akka.util.Helpers.Requiring import akka.util.WildcardTree import io.aeron.Aeron import io.aeron.AvailableImageHandler -import io.aeron.CncFileDescriptor import io.aeron.Image import io.aeron.UnavailableImageHandler import io.aeron.driver.MediaDriver @@ -68,7 +62,6 @@ import java.nio.channels.{ DatagramChannel, FileChannel } import akka.remote.artery.OutboundControlJunction.OutboundControlIngress import io.aeron.CncFileDescriptor import java.util.concurrent.atomic.AtomicLong -import akka.actor.Cancellable import scala.collection.JavaConverters._ import akka.stream.ActorMaterializerSettings import scala.annotation.tailrec @@ -206,8 +199,11 @@ private[akka] trait InboundContext { */ private[akka] object AssociationState { def apply(): AssociationState = - new AssociationState(incarnation = 1, uniqueRemoteAddressPromise = Promise(), - quarantined = ImmutableLongMap.empty[QuarantinedTimestamp]) + new AssociationState( + incarnation = 1, + uniqueRemoteAddressPromise = Promise(), + quarantined = ImmutableLongMap.empty[QuarantinedTimestamp], + outboundCompression = NoOutboundCompression) final case class QuarantinedTimestamp(nanoTime: Long) { override def toString: String = @@ -221,7 +217,8 @@ private[akka] object AssociationState { private[akka] final class AssociationState( val incarnation: Int, val uniqueRemoteAddressPromise: Promise[UniqueAddress], - val quarantined: ImmutableLongMap[AssociationState.QuarantinedTimestamp]) { + val quarantined: ImmutableLongMap[AssociationState.QuarantinedTimestamp], + val outboundCompression: OutboundCompression) { import AssociationState.QuarantinedTimestamp @@ -247,14 +244,17 @@ private[akka] final class AssociationState( } } - def newIncarnation(remoteAddressPromise: Promise[UniqueAddress]): AssociationState = - new AssociationState(incarnation + 1, remoteAddressPromise, quarantined) + def newIncarnation(remoteAddressPromise: Promise[UniqueAddress], compression: OutboundCompression): AssociationState = + new AssociationState(incarnation + 1, remoteAddressPromise, quarantined, compression) def newQuarantined(): AssociationState = uniqueRemoteAddressPromise.future.value match { case Some(Success(a)) ⇒ - new AssociationState(incarnation, uniqueRemoteAddressPromise, - quarantined = quarantined.updated(a.uid, QuarantinedTimestamp(System.nanoTime()))) + new AssociationState( + incarnation, + uniqueRemoteAddressPromise, + quarantined = quarantined.updated(a.uid, QuarantinedTimestamp(System.nanoTime())), + outboundCompression = NoOutboundCompression) // after quarantine no compression needed anymore, drop it case _ ⇒ this } @@ -384,10 +384,6 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // !!! WARNING !!! This is *NOT* thread safe, private val topLevelFREvents = flightRecorder.createEventSink() - // FIXME: Compression table must be owned by each channel instead - // of having a global one - val compression = new Compression(system) - private val associationRegistry = new AssociationRegistry( remoteAddress ⇒ new Association(this, materializer, remoteAddress, controlSubject, largeMessageDestinations)) @@ -538,19 +534,22 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundStreams(): Unit = { - runInboundControlStream() - runInboundOrdinaryMessagesStream() + val noCompression = new NoInboundCompression(system) // TODO possibly enable on other channels too https://github.com/akka/akka/pull/20546/files#r68074082 + val compression = createInboundCompressionTable(this) + + runInboundControlStream(noCompression) + runInboundOrdinaryMessagesStream(compression) if (largeMessageDestinationsEnabled) { runInboundLargeMessagesStream() } } - private def runInboundControlStream(): Unit = { + private def runInboundControlStream(compression: InboundCompression): Unit = { val (ctrl, completed) = if (remoteSettings.TestMode) { val (mgmt, (ctrl, completed)) = aeronSource(controlStreamId, envelopePool) - .via(inboundFlow) + .via(inboundFlow(compression)) .viaMat(inboundTestFlow)(Keep.right) .toMat(inboundControlSink)(Keep.both) .run()(materializer) @@ -558,26 +557,13 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R (ctrl, completed) } else { aeronSource(controlStreamId, envelopePool) - .via(inboundFlow) + .via(inboundFlow(compression)) .toMat(inboundControlSink)(Keep.right) .run()(materializer) } controlSubject = ctrl - controlSubject.attach(new ControlMessageObserver { - override def notify(inboundEnvelope: InboundEnvelope): Unit = { - inboundEnvelope.message match { - case Quarantined(from, to) if to == localAddress ⇒ - val lifecycleEvent = ThisActorSystemQuarantinedEvent(localAddress.address, from.address) - publishLifecycleEvent(lifecycleEvent) - // quarantine the other system from here - association(from.address).quarantine(lifecycleEvent.toString, Some(from.uid)) - case _ ⇒ // not interesting - } - } - }) - // ordinary messages stream controlSubject.attach(new ControlMessageObserver { override def notify(inboundEnvelope: InboundEnvelope): Unit = { @@ -592,14 +578,34 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } }) - attachStreamRestart("Inbound control stream", completed, () ⇒ runInboundControlStream()) + // compression messages + controlSubject.attach(new ControlMessageObserver { + override def notify(inboundEnvelope: InboundEnvelope): Unit = + inboundEnvelope.message match { + case m: CompressionMessage ⇒ + m match { + case CompressionProtocol.ActorRefCompressionAdvertisement(from, ref, id) ⇒ + log.debug("Incoming ActorRef compression advertisement from [{}], allocating: [{} => {}]", from, ref, id) + association(from.address).compression.allocateActorRefCompressionId(ref, id) + system.eventStream.publish(CompressionProtocol.Events.ReceivedCompressionAdvertisement(from, ref, id)) + + case CompressionProtocol.ClassManifestCompressionAdvertisement(from, manifest, id) ⇒ + log.debug("Incoming Class Manifest compression advertisement from [{}], allocating: [{} => {}]", from, manifest, id) + association(from.address).compression.allocateClassManifestCompressionId(manifest, id) + system.eventStream.publish(CompressionProtocol.Events.ReceivedCompressionAdvertisement(from, manifest, id)) + } + case _ ⇒ // not interested in non CompressionMessages + } + }) + + attachStreamRestart("Inbound control stream", completed, () ⇒ runInboundControlStream(compression)) } - private def runInboundOrdinaryMessagesStream(): Unit = { + private def runInboundOrdinaryMessagesStream(compression: InboundCompression): Unit = { val completed = if (remoteSettings.TestMode) { val (mgmt, c) = aeronSource(ordinaryStreamId, envelopePool) - .via(inboundFlow) + .via(inboundFlow(compression)) .viaMat(inboundTestFlow)(Keep.right) .toMat(inboundSink)(Keep.both) .run()(materializer) @@ -607,19 +613,21 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R c } else { aeronSource(ordinaryStreamId, envelopePool) - .via(inboundFlow) + .via(inboundFlow(compression)) .toMat(inboundSink)(Keep.right) .run()(materializer) } - attachStreamRestart("Inbound message stream", completed, () ⇒ runInboundOrdinaryMessagesStream()) + attachStreamRestart("Inbound message stream", completed, () ⇒ runInboundOrdinaryMessagesStream(compression)) } private def runInboundLargeMessagesStream(): Unit = { + val compression = new NoInboundCompression(system) // no compression on large message stream for now + val completed = if (remoteSettings.TestMode) { val (mgmt, c) = aeronSource(largeStreamId, largeEnvelopePool) - .via(inboundLargeFlow) + .via(inboundLargeFlow(compression)) .viaMat(inboundTestFlow)(Keep.right) .toMat(inboundSink)(Keep.both) .run()(materializer) @@ -627,7 +635,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R c } else { aeronSource(largeStreamId, largeEnvelopePool) - .via(inboundLargeFlow) + .via(inboundLargeFlow(compression)) .toMat(inboundSink)(Keep.right) .run()(materializer) } @@ -678,7 +686,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R Future.successful(Done) } - private[remote] def isShutdown(): Boolean = _shutdown + private[remote] def isShutdown: Boolean = _shutdown override def managementCommand(cmd: Any): Future[Boolean] = { if (testStages.isEmpty) @@ -709,8 +717,10 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R a.send(message, sender, recipient) } - override def association(remoteAddress: Address): Association = + override def association(remoteAddress: Address): Association = { + require(remoteAddress != localAddress.address, "Attemted association with self address!") associationRegistry.association(remoteAddress) + } override def association(uid: Long): OptionVal[Association] = associationRegistry.association(uid) @@ -728,39 +738,46 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R association(remoteAddress).quarantine(reason = "", uid.map(_.toLong)) } - def outbound(outboundContext: OutboundContext): Sink[Send, Future[Done]] = { + def outbound(outboundContext: OutboundContext, compression: OutboundCompression): Sink[Send, Future[Done]] = { Flow.fromGraph(killSwitch.flow[Send]) - .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) - .via(encoder) + .via(new OutboundHandshake(system, outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) + .via(encoder(compression)) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), ordinaryStreamId, aeron, taskRunner, envelopePool, giveUpSendAfter, flightRecorder.createEventSink()))(Keep.right) } - def outboundLarge(outboundContext: OutboundContext): Sink[Send, Future[Done]] = { + def outboundLarge(outboundContext: OutboundContext, compression: OutboundCompression): Sink[Send, Future[Done]] = { Flow.fromGraph(killSwitch.flow[Send]) - .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) - .via(createEncoder(largeEnvelopePool)) + .via(new OutboundHandshake(system, outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) + .via(createEncoder(largeEnvelopePool, compression)) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), largeStreamId, aeron, taskRunner, envelopePool, giveUpSendAfter, flightRecorder.createEventSink()))(Keep.right) } - def outboundControl(outboundContext: OutboundContext): Sink[Send, (OutboundControlIngress, Future[Done])] = { + def outboundControl(outboundContext: OutboundContext, compression: OutboundCompression): Sink[Send, (OutboundControlIngress, Future[Done])] = { Flow.fromGraph(killSwitch.flow[Send]) - .via(new OutboundHandshake(outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) + .via(new OutboundHandshake(system, outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) .via(new SystemMessageDelivery(outboundContext, system.deadLetters, systemMessageResendInterval, remoteSettings.SysMsgBufferSize)) .viaMat(new OutboundControlJunction(outboundContext))(Keep.right) - .via(encoder) + .via(encoder(compression)) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), controlStreamId, aeron, taskRunner, envelopePool, Duration.Inf, flightRecorder.createEventSink()))(Keep.both) // FIXME we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages } - def createEncoder(bufferPool: EnvelopeBufferPool): Flow[Send, EnvelopeBuffer, NotUsed] = + def createEncoder(compression: OutboundCompression, bufferPool: EnvelopeBufferPool): Flow[Send, EnvelopeBuffer, NotUsed] = Flow.fromGraph(new Encoder(localAddress, system, compression, bufferPool)) - def encoder: Flow[Send, EnvelopeBuffer, NotUsed] = createEncoder(envelopePool) + private def createInboundCompressionTable(inboundContext: InboundContext): InboundCompression = + if (remoteSettings.ArteryCompressionSettings.enabled) new InboundCompressionImpl(system, inboundContext) + else new NoInboundCompression(system) + + def createEncoder(pool: EnvelopeBufferPool, compression: OutboundCompression): Flow[Send, EnvelopeBuffer, NotUsed] = + Flow.fromGraph(new Encoder(localAddress, system, compression, pool)) + + def encoder(compression: OutboundCompression): Flow[Send, EnvelopeBuffer, NotUsed] = createEncoder(envelopePool, compression) def aeronSource(streamId: Int, pool: EnvelopeBufferPool): Source[EnvelopeBuffer, NotUsed] = Source.fromGraph(new AeronSource(inboundChannel, streamId, aeron, taskRunner, pool, @@ -771,14 +788,15 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R inboundEnvelopePool.release(m) } - def createDecoder(bufferPool: EnvelopeBufferPool): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { + def createDecoder(compression: InboundCompression, bufferPool: EnvelopeBufferPool): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { val resolveActorRefWithLocalAddress: String ⇒ InternalActorRef = recipient ⇒ provider.resolveActorRefWithLocalAddress(recipient, localAddress.address) Flow.fromGraph(new Decoder(this, system, resolveActorRefWithLocalAddress, compression, bufferPool, inboundEnvelopePool)) } - def decoder: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = createDecoder(envelopePool) + def decoder(compression: InboundCompression): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = + createDecoder(compression, envelopePool) def inboundSink: Sink[InboundEnvelope, Future[Done]] = Flow[InboundEnvelope] @@ -786,16 +804,16 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .via(new InboundQuarantineCheck(this)) .toMat(messageDispatcherSink)(Keep.right) - def inboundFlow: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { + def inboundFlow(compression: InboundCompression): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { Flow[EnvelopeBuffer] .via(killSwitch.flow) - .via(decoder) + .via(decoder(compression)) } - def inboundLargeFlow: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { + def inboundLargeFlow(compression: InboundCompression): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { Flow[EnvelopeBuffer] .via(killSwitch.flow) - .via(createDecoder(largeEnvelopePool)) + .via(createDecoder(compression, largeEnvelopePool)) } def inboundControlSink: Sink[InboundEnvelope, (ControlMessageSubject, Future[Done])] = { diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index c6d27f4c85..4a73807ba8 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -9,6 +9,8 @@ import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.CountDownLatch import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicReference +import akka.remote.artery.compress.{ OutboundCompressionImpl, CompressionProtocol } + import scala.annotation.tailrec import scala.concurrent.Future import scala.concurrent.Promise @@ -79,6 +81,9 @@ private[remote] class Association( // the `SendQueue` after materialization. Using same underlying queue. This makes it possible to // start sending (enqueuing) to the Association immediate after construction. + /** Accesses the currently active outbound compression. */ + def compression: OutboundCompression = associationState.outboundCompression + def createQueue(capacity: Int): Queue[Send] = new ManyToOneConcurrentArrayQueue[Send](capacity) @@ -135,13 +140,13 @@ private[remote] class Association( def completeHandshake(peer: UniqueAddress): Unit = { require( remoteAddress == peer.address, - s"wrong remote address in completeHandshake, got ${peer.address}, expected ${remoteAddress}") + s"wrong remote address in completeHandshake, got ${peer.address}, expected $remoteAddress") val current = associationState current.uniqueRemoteAddressPromise.trySuccess(peer) current.uniqueRemoteAddressValue() match { case Some(`peer`) ⇒ // our value case _ ⇒ - val newState = current.newIncarnation(Promise.successful(peer)) + val newState = current.newIncarnation(Promise.successful(peer), createOutboundCompressionTable(remoteAddress)) if (swapState(current, newState)) { current.uniqueRemoteAddressValue() match { case Some(old) ⇒ @@ -215,7 +220,10 @@ private[remote] class Association( // FIXME we should be able to Send without a recipient ActorRef override val dummyRecipient: RemoteActorRef = - transport.provider.resolveActorRef(RootActorPath(remoteAddress) / "system" / "dummy").asInstanceOf[RemoteActorRef] + try transport.provider.resolveActorRef(RootActorPath(remoteAddress) / "system" / "dummy").asInstanceOf[RemoteActorRef] + catch { + case ex: Exception ⇒ throw new Exception("Bad dummy recipient! RemoteAddress: " + remoteAddress, ex) + } // OutboundContext override def quarantine(reason: String): Unit = { @@ -270,18 +278,24 @@ private[remote] class Association( def associate(): Unit = { if (!controlQueue.isInstanceOf[QueueWrapper]) throw new IllegalStateException("associate() must only be called once") + runOutboundStreams() + } + + private def runOutboundStreams(): Unit = { + // TODO no compression for control / large streams currently + val disableCompression = NoOutboundCompression // it's important to materialize the outboundControl stream first, // so that outboundControlIngress is ready when stages for all streams start - runOutboundControlStream() - runOutboundOrdinaryMessagesStream() + runOutboundControlStream(disableCompression) + runOutboundOrdinaryMessagesStream(CurrentAssociationStateOutboundCompressionProxy) if (largeMessageChannelEnabled) { - runOutboundLargeMessagesStream() + runOutboundLargeMessagesStream(disableCompression) } } - private def runOutboundControlStream(): Unit = { + private def runOutboundControlStream(compression: OutboundCompression): Unit = { // stage in the control stream may access the outboundControlIngress before returned here // using CountDownLatch to make sure that materialization is completed before accessing outboundControlIngress materializing = new CountDownLatch(1) @@ -294,13 +308,13 @@ private[remote] class Association( val ((queueValue, mgmt), (control, completed)) = Source.fromGraph(new SendQueue[Send]) .viaMat(transport.outboundTestFlow(this))(Keep.both) - .toMat(transport.outboundControl(this))(Keep.both) + .toMat(transport.outboundControl(this, compression))(Keep.both) .run()(materializer) _testStages.add(mgmt) (queueValue, (control, completed)) } else { Source.fromGraph(new SendQueue[Send]) - .toMat(transport.outboundControl(this))(Keep.both) + .toMat(transport.outboundControl(this, compression))(Keep.both) .run()(materializer) } @@ -310,7 +324,7 @@ private[remote] class Association( _outboundControlIngress = control materializing.countDown() attachStreamRestart("Outbound control stream", completed, cause ⇒ { - runOutboundControlStream() + runOutboundControlStream(compression) cause match { case _: HandshakeTimeoutException ⇒ // ok, quarantine not possible without UID case _ ⇒ quarantine("Outbound control stream restarted") @@ -326,7 +340,7 @@ private[remote] class Association( QueueWrapper(createQueue(capacity)) } - private def runOutboundOrdinaryMessagesStream(): Unit = { + private def runOutboundOrdinaryMessagesStream(compression: OutboundCompression): Unit = { val wrapper = getOrCreateQueueWrapper(queue, queueSize) queue = wrapper // use new underlying queue immediately for restarts @@ -334,13 +348,13 @@ private[remote] class Association( if (transport.remoteSettings.TestMode) { val ((queueValue, mgmt), completed) = Source.fromGraph(new SendQueue[Send]) .viaMat(transport.outboundTestFlow(this))(Keep.both) - .toMat(transport.outbound(this))(Keep.both) + .toMat(transport.outbound(this, compression))(Keep.both) .run()(materializer) _testStages.add(mgmt) (queueValue, completed) } else { Source.fromGraph(new SendQueue[Send]) - .toMat(transport.outbound(this))(Keep.both) + .toMat(transport.outbound(this, compression))(Keep.both) .run()(materializer) } @@ -348,10 +362,10 @@ private[remote] class Association( // replace with the materialized value, still same underlying queue queue = queueValue - attachStreamRestart("Outbound message stream", completed, _ ⇒ runOutboundOrdinaryMessagesStream()) + attachStreamRestart("Outbound message stream", completed, _ ⇒ runOutboundOrdinaryMessagesStream(compression)) } - private def runOutboundLargeMessagesStream(): Unit = { + private def runOutboundLargeMessagesStream(compression: OutboundCompression): Unit = { val wrapper = getOrCreateQueueWrapper(queue, largeQueueSize) largeQueue = wrapper // use new underlying queue immediately for restarts @@ -359,20 +373,20 @@ private[remote] class Association( if (transport.remoteSettings.TestMode) { val ((queueValue, mgmt), completed) = Source.fromGraph(new SendQueue[Send]) .viaMat(transport.outboundTestFlow(this))(Keep.both) - .toMat(transport.outboundLarge(this))(Keep.both) + .toMat(transport.outboundLarge(this, compression))(Keep.both) .run()(materializer) _testStages.add(mgmt) (queueValue, completed) } else { Source.fromGraph(new SendQueue[Send]) - .toMat(transport.outboundLarge(this))(Keep.both) + .toMat(transport.outboundLarge(this, compression))(Keep.both) .run()(materializer) } queueValue.inject(wrapper.queue) // replace with the materialized value, still same underlying queue largeQueue = queueValue - attachStreamRestart("Outbound large message stream", completed, _ ⇒ runOutboundLargeMessagesStream()) + attachStreamRestart("Outbound large message stream", completed, _ ⇒ runOutboundLargeMessagesStream(compression)) } private def attachStreamRestart(streamName: String, streamCompleted: Future[Done], restart: Throwable ⇒ Unit): Unit = { @@ -396,7 +410,35 @@ private[remote] class Association( } } - override def toString(): String = + // TODO: Make sure that once other channels use Compression, each gets it's own + private def createOutboundCompressionTable(remoteAddress: Address): OutboundCompression = { + if (transport.provider.remoteSettings.ArteryCompressionSettings.enabled) { + val compression = new OutboundCompressionImpl(transport.system, remoteAddress) + // FIXME should use verion number of table instead of hashCode + log.info("Creating Outbound compression table ({}) to [{}]", compression.hashCode, remoteAddress) + compression + } else NoOutboundCompression + } + + /* + * This proxy uses the current associationStates compression table, which is reset for a new incarnation. + * This way the same outgoing stream will switch to using the new table without the need of restarting it. + */ + object CurrentAssociationStateOutboundCompressionProxy extends OutboundCompression { + override final def allocateActorRefCompressionId(ref: ActorRef, id: Int): Unit = + associationState.outboundCompression.allocateActorRefCompressionId(ref, id) + + override final def allocateClassManifestCompressionId(manifest: String, id: Int): Unit = + associationState.outboundCompression.allocateClassManifestCompressionId(manifest, id) + + override final def compressActorRef(ref: ActorRef): Int = + associationState.outboundCompression.compressActorRef(ref) + + override final def compressClassManifest(manifest: String): Int = + associationState.outboundCompression.compressClassManifest(manifest) + } + + override def toString: String = s"Association($localAddress -> $remoteAddress with $associationState)" } diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index b649fd71ae..ee32b9a187 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -4,13 +4,14 @@ package akka.remote.artery -import java.lang.reflect.Field import java.nio.charset.Charset import java.nio.{ ByteBuffer, ByteOrder } +import akka.actor.{ Address, ActorRef } +import akka.remote.artery.compress.{ NoopOutboundCompression, NoopInboundCompression } +import akka.serialization.Serialization import org.agrona.concurrent.{ ManyToManyConcurrentArrayQueue, UnsafeBuffer } -import sun.misc.Cleaner -import akka.util.Unsafe +import akka.util.{ OptionVal, Unsafe } import scala.util.control.NonFatal @@ -68,19 +69,37 @@ private[remote] object EnvelopeBuffer { /** * INTERNAL API + * Decompress and cause compression advertisements. */ -private[remote] trait LiteralCompressionTable { +private[remote] trait InboundCompression { + def hitActorRef(remote: Address, ref: ActorRef): Unit + def decompressActorRef(idx: Int): OptionVal[ActorRef] - def compressActorRef(ref: String): Int - def decompressActorRef(idx: Int): String + def hitClassManifest(remote: Address, manifest: String): Unit + def decompressClassManifest(idx: Int): OptionVal[String] +} +/** + * INTERNAL API + * Compress outgoing data and handle compression advertisements to fill compression table. + */ +private[remote] trait OutboundCompression { + def allocateActorRefCompressionId(ref: ActorRef, id: Int): Unit + def compressActorRef(ref: ActorRef): Int + def allocateClassManifestCompressionId(manifest: String, id: Int): Unit def compressClassManifest(manifest: String): Int - def decompressClassManifest(idx: Int): String - } object HeaderBuilder { - def apply(compressionTable: LiteralCompressionTable): HeaderBuilder = new HeaderBuilderImpl(compressionTable) + + // 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: InboundCompression): HeaderBuilder = new HeaderBuilderImpl(compression, NoopOutboundCompression) + def out(compression: OutboundCompression): HeaderBuilder = new HeaderBuilderImpl(NoopInboundCompression, compression) + + /** INTERNAL API, FOR TESTING ONLY */ + private[remote] def bothWays(in: InboundCompression, out: OutboundCompression): HeaderBuilder = new HeaderBuilderImpl(in, out) } /** @@ -93,8 +112,9 @@ sealed trait HeaderBuilder { def uid_=(u: Long): Unit def uid: Long - def senderActorRef_=(ref: String): Unit - def senderActorRef: String + def senderActorRef_=(ref: ActorRef): Unit + def senderActorRef: OptionVal[ActorRef] + def senderActorRefPath: String def setNoSender(): Unit def isNoSender: Boolean @@ -102,8 +122,9 @@ sealed trait HeaderBuilder { def setNoRecipient(): Unit def isNoRecipient: Boolean - def recipientActorRef_=(ref: String): Unit - def recipientActorRef: String + def recipientActorRef_=(ref: ActorRef): Unit + def recipientActorRef: OptionVal[ActorRef] + def recipientActorRefPath: String def serializer_=(serializer: Int): Unit def serializer: Int @@ -115,7 +136,7 @@ sealed trait HeaderBuilder { /** * INTERNAL API */ -private[remote] final class HeaderBuilderImpl(val compressionTable: LiteralCompressionTable) extends HeaderBuilder { +private[remote] final class HeaderBuilderImpl(inboundCompression: InboundCompression, outboundCompression: OutboundCompression) extends HeaderBuilder { var version: Int = _ var uid: Long = _ @@ -129,64 +150,61 @@ private[remote] final class HeaderBuilderImpl(val compressionTable: LiteralCompr var _manifest: String = null var _manifestIdx: Int = -1 - def senderActorRef_=(ref: String): Unit = { - _senderActorRef = ref - _senderActorRefIdx = compressionTable.compressActorRef(ref) + def senderActorRef_=(ref: ActorRef): Unit = { + _senderActorRefIdx = outboundCompression.compressActorRef(ref) + if (_senderActorRefIdx == -1) _senderActorRef = Serialization.serializedActorPath(ref) // includes local address from `currentTransportInformation` } - def setNoSender(): Unit = { _senderActorRef = null _senderActorRefIdx = EnvelopeBuffer.DeadLettersCode } - def isNoSender: Boolean = (_senderActorRef eq null) && _senderActorRefIdx == EnvelopeBuffer.DeadLettersCode - - def senderActorRef: String = { + def senderActorRef: OptionVal[ActorRef] = + if (_senderActorRef eq null) inboundCompression.decompressActorRef(_senderActorRefIdx) + else OptionVal.None + def senderActorRefPath: String = if (_senderActorRef ne null) _senderActorRef else { - _senderActorRef = compressionTable.decompressActorRef(_senderActorRefIdx) + _senderActorRef = inboundCompression.decompressActorRef(_senderActorRefIdx).get.path.toSerializationFormat _senderActorRef } - } def setNoRecipient(): Unit = { _recipientActorRef = null _recipientActorRefIdx = EnvelopeBuffer.DeadLettersCode } - def isNoRecipient: Boolean = (_recipientActorRef eq null) && _recipientActorRefIdx == EnvelopeBuffer.DeadLettersCode - def recipientActorRef_=(ref: String): Unit = { - _recipientActorRef = ref - _recipientActorRefIdx = compressionTable.compressActorRef(ref) + def recipientActorRef_=(ref: ActorRef): Unit = { + _recipientActorRefIdx = outboundCompression.compressActorRef(ref) + if (_recipientActorRefIdx == -1) _recipientActorRef = ref.path.toSerializationFormat } - - def recipientActorRef: String = { + def recipientActorRef: OptionVal[ActorRef] = + if (_recipientActorRef eq null) inboundCompression.decompressActorRef(_recipientActorRefIdx) + else OptionVal.None + def recipientActorRefPath: String = if (_recipientActorRef ne null) _recipientActorRef else { - _recipientActorRef = compressionTable.decompressActorRef(_recipientActorRefIdx) + _recipientActorRef = inboundCompression.decompressActorRef(_recipientActorRefIdx).get.path.toSerializationFormat _recipientActorRef } - } override def serializer_=(serializer: Int): Unit = { _serializer = serializer } - override def serializer: Int = _serializer override def manifest_=(manifest: String): Unit = { - _manifest = manifest - _manifestIdx = compressionTable.compressClassManifest(manifest) + _manifestIdx = outboundCompression.compressClassManifest(manifest) + if (_manifestIdx == -1) _manifest = manifest } - override def manifest: String = { if (_manifest ne null) _manifest else { - _manifest = compressionTable.decompressClassManifest(_manifestIdx) + _manifest = inboundCompression.decompressClassManifest(_manifestIdx).get _manifest } } @@ -341,10 +359,10 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { */ def tryCleanDirectByteBuffer(): Unit = try { if (byteBuffer.isDirect) { - val cleanerMethod = byteBuffer.getClass().getMethod("cleaner") + val cleanerMethod = byteBuffer.getClass.getMethod("cleaner") cleanerMethod.setAccessible(true) val cleaner = cleanerMethod.invoke(byteBuffer) - val cleanMethod = cleaner.getClass().getMethod("clean") + val cleanMethod = cleaner.getClass.getMethod("clean") cleanMethod.setAccessible(true) cleanMethod.invoke(cleaner) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 0104daa779..61ae1368df 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -1,17 +1,19 @@ + +/** + * Copyright (C) 2016 Lightbend Inc. + */ package akka.remote.artery import scala.concurrent.duration._ import scala.util.control.NonFatal -import akka.actor.{ ActorRef, InternalActorRef } -import akka.actor.ActorSystem -import akka.actor.ExtendedActorSystem +import akka.actor._ import akka.remote.{ MessageSerializer, OversizedPayloadException, UniqueAddress } import akka.remote.EndpointManager.Send 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.OptionVal +import akka.util.{ ByteString, OptionVal } import akka.actor.EmptyLocalActorRef import akka.stream.stage.TimerGraphStageLogic @@ -21,7 +23,7 @@ import akka.stream.stage.TimerGraphStageLogic private[remote] class Encoder( uniqueLocalAddress: UniqueAddress, system: ActorSystem, - compressionTable: LiteralCompressionTable, + compression: OutboundCompression, bufferPool: EnvelopeBufferPool) extends GraphStage[FlowShape[Send, EnvelopeBuffer]] { @@ -32,58 +34,35 @@ private[remote] class Encoder( override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) with InHandler with OutHandler with StageLogging { - private val headerBuilder = HeaderBuilder(compressionTable) + private val headerBuilder = HeaderBuilder.out(compression) headerBuilder.version = ArteryTransport.Version headerBuilder.uid = uniqueLocalAddress.uid private val localAddress = uniqueLocalAddress.address private val serialization = SerializationExtension(system) private val serializationInfo = Serialization.Information(localAddress, system) - private val senderCache = new java.util.HashMap[ActorRef, String] - private var recipientCache = new java.util.HashMap[ActorRef, String] - override protected def logSource = classOf[Encoder] override def onPush(): Unit = { val send = grab(in) val envelope = bufferPool.acquire() - val recipientStr = recipientCache.get(send.recipient) match { - case null ⇒ - val s = send.recipient.path.toSerializationFormat - // FIXME this cache will be replaced by compression table - if (recipientCache.size() >= 1000) - recipientCache.clear() - recipientCache.put(send.recipient, s) - s - case s ⇒ s - } - headerBuilder.recipientActorRef = recipientStr - - send.senderOption match { - case OptionVal.None ⇒ headerBuilder.setNoSender() - case OptionVal.Some(sender) ⇒ - val senderStr = senderCache.get(sender) match { - case null ⇒ - val s = sender.path.toSerializationFormatWithAddress(localAddress) - // FIXME we might need an efficient LRU cache, or replaced by compression table - if (senderCache.size() >= 1000) - senderCache.clear() - senderCache.put(sender, s) - s - case s ⇒ s - } - headerBuilder.senderActorRef = senderStr - } + // internally compression is applied by the builder: + headerBuilder.recipientActorRef = send.recipient try { // avoiding currentTransportInformation.withValue due to thunk allocation val oldValue = Serialization.currentTransportInformation.value try { Serialization.currentTransportInformation.value = serializationInfo + + send.senderOption match { + case OptionVal.None ⇒ headerBuilder.setNoSender() + case OptionVal.Some(s) ⇒ headerBuilder.senderActorRef = s + } + MessageSerializer.serializeForArtery(serialization, send.message.asInstanceOf[AnyRef], headerBuilder, envelope) - } finally - Serialization.currentTransportInformation.value = oldValue + } finally Serialization.currentTransportInformation.value = oldValue envelope.byteBuffer.flip() push(out, envelope) @@ -130,7 +109,7 @@ private[remote] class Decoder( inboundContext: InboundContext, system: ExtendedActorSystem, resolveActorRefWithLocalAddress: String ⇒ InternalActorRef, - compressionTable: LiteralCompressionTable, + compression: InboundCompression, bufferPool: EnvelopeBufferPool, inEnvelopePool: ObjectPool[InboundEnvelope]) extends GraphStage[FlowShape[EnvelopeBuffer, InboundEnvelope]] { val in: Inlet[EnvelopeBuffer] = Inlet("Artery.Decoder.in") @@ -141,12 +120,9 @@ private[remote] class Decoder( new TimerGraphStageLogic(shape) with InHandler with OutHandler with StageLogging { import Decoder.RetryResolveRemoteDeployedRecipient private val localAddress = inboundContext.localAddress.address - private val headerBuilder = HeaderBuilder(compressionTable) + private val headerBuilder = HeaderBuilder.in(compression) private val serialization = SerializationExtension(system) - private val recipientCache = new java.util.HashMap[String, InternalActorRef] - private val senderCache = new java.util.HashMap[String, ActorRef] - private val retryResolveRemoteDeployedRecipientInterval = 50.millis private val retryResolveRemoteDeployedRecipientAttempts = 20 @@ -156,35 +132,32 @@ private[remote] class Decoder( val envelope = grab(in) envelope.parseHeader(headerBuilder) - // FIXME: Instead of using Strings, the headerBuilder should automatically return cached ActorRef instances - // in case of compression is enabled - // FIXME: Is localAddress really needed? - - val sender = - if (headerBuilder.isNoSender) - OptionVal.None - else { - senderCache.get(headerBuilder.senderActorRef) match { - case null ⇒ - val ref = resolveActorRefWithLocalAddress(headerBuilder.senderActorRef) - // FIXME this cache will be replaced by compression table - if (senderCache.size() >= 1000) - senderCache.clear() - senderCache.put(headerBuilder.senderActorRef, ref) - OptionVal(ref) - case ref ⇒ OptionVal(ref) - } - } - - val recipient = - if (headerBuilder.isNoRecipient) - OptionVal.None - else - resolveRecipient(headerBuilder.recipientActorRef) - val originUid = headerBuilder.uid val association = inboundContext.association(originUid) + val recipient: OptionVal[InternalActorRef] = headerBuilder.recipientActorRef match { + case OptionVal.Some(ref) ⇒ OptionVal(ref.asInstanceOf[InternalActorRef]) + case OptionVal.None ⇒ resolveRecipient(headerBuilder.recipientActorRefPath) + } + + val sender: InternalActorRef = headerBuilder.senderActorRef match { + case OptionVal.Some(ref) ⇒ ref.asInstanceOf[InternalActorRef] + case OptionVal.None ⇒ resolveActorRefWithLocalAddress(headerBuilder.senderActorRefPath) + } + + // --- hit refs and manifests for heavy-hitter counting + association match { + case OptionVal.Some(assoc) ⇒ + val remoteAddress = assoc.remoteAddress + compression.hitActorRef(remoteAddress, sender) + if (recipient.isDefined) compression.hitActorRef(remoteAddress, recipient.get) + compression.hitClassManifest(remoteAddress, headerBuilder.manifest) + 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 + try { val deserializedMessage = MessageSerializer.deserializeForArtery( system, serialization, headerBuilder, envelope) @@ -194,7 +167,7 @@ private[remote] class Decoder( recipient, localAddress, // FIXME: Is this needed anymore? What should we do here? deserializedMessage, - sender, + OptionVal.Some(sender), // FIXME: No need for an option, decode simply to deadLetters instead originUid, association) @@ -203,7 +176,7 @@ private[remote] class Decoder( // recipient for the first message that is sent to it, best effort retry scheduleOnce(RetryResolveRemoteDeployedRecipient( retryResolveRemoteDeployedRecipientAttempts, - headerBuilder.recipientActorRef, decoded), retryResolveRemoteDeployedRecipientInterval) + headerBuilder.recipientActorRefPath, decoded), retryResolveRemoteDeployedRecipientInterval) } else push(out, decoded) } catch { @@ -218,28 +191,12 @@ private[remote] class Decoder( } private def resolveRecipient(path: String): OptionVal[InternalActorRef] = { - recipientCache.get(path) match { - case null ⇒ - def addToCache(resolved: InternalActorRef): Unit = { - // FIXME we might need an efficient LRU cache, or replaced by compression table - if (recipientCache.size() >= 1000) - recipientCache.clear() - recipientCache.put(path, resolved) - } - - resolveActorRefWithLocalAddress(path) match { - case empty: EmptyLocalActorRef ⇒ - val pathElements = empty.path.elements - if (pathElements.nonEmpty && pathElements.head == "remote") - OptionVal.None - else { - addToCache(empty) - OptionVal(empty) - } - case ref ⇒ - addToCache(ref) - OptionVal(ref) - } + resolveActorRefWithLocalAddress(path) match { + case empty: EmptyLocalActorRef ⇒ + val pathElements = empty.path.elements + // FIXME remote deployment corner case, please fix @patriknw (see also below, in onTimer) + if (pathElements.nonEmpty && pathElements.head == "remote") OptionVal.None + else OptionVal(empty) case ref ⇒ OptionVal(ref) } } @@ -254,11 +211,10 @@ private[remote] class Decoder( if (attemptsLeft > 0) scheduleOnce(RetryResolveRemoteDeployedRecipient( attemptsLeft - 1, - headerBuilder.recipientActorRef, inboundEnvelope), retryResolveRemoteDeployedRecipientInterval) + recipientPath, inboundEnvelope), retryResolveRemoteDeployedRecipientInterval) else { val recipient = resolveActorRefWithLocalAddress(recipientPath) - // only retry for the first message - recipientCache.put(recipientPath, recipient) + // FIXME only retry for the first message, need to keep them in a cache push(out, inboundEnvelope.withRecipient(recipient)) } case OptionVal.Some(recipient) ⇒ @@ -270,3 +226,4 @@ private[remote] class Decoder( setHandlers(in, out, this) } } + diff --git a/akka-remote/src/main/scala/akka/remote/artery/Compression.scala b/akka-remote/src/main/scala/akka/remote/artery/Compression.scala deleted file mode 100644 index 50206833a1..0000000000 --- a/akka-remote/src/main/scala/akka/remote/artery/Compression.scala +++ /dev/null @@ -1,20 +0,0 @@ -package akka.remote.artery - -import akka.actor.ActorSystem - -/** - * INTERNAL API - */ -// FIXME: Dummy compression table, needs to be replaced by the real deal -// Currently disables all compression -private[remote] class Compression(system: ActorSystem) extends LiteralCompressionTable { - // FIXME: Of course it is foolish to store this as String, but this is a stub - val deadLettersString = system.deadLetters.path.toSerializationFormat - - override def compressActorRef(ref: String): Int = -1 - override def decompressActorRef(idx: Int): String = ??? - - override def compressClassManifest(manifest: String): Int = -1 - override def decompressClassManifest(idx: Int): String = ??? - -} diff --git a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala index 1f61bb2063..93d034622c 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala @@ -35,4 +35,10 @@ object FlightRecorderEvents { val AeronSource_Received = 23 val AeronSource_DelegateToTaskRunner = 24 + // Compression events + val Compression_CompressedActorRef = 25 + val Compression_AllocatedActorRefCompressionId = 26 + val Compression_CompressedManifest = 27 + val Compression_AllocatedManifestCompressionId = 28 + } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala index 22b633208f..8a66226d6f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -3,6 +3,8 @@ */ package akka.remote.artery +import akka.actor.ActorSystem + import scala.concurrent.duration._ import scala.util.control.NoStackTrace import akka.remote.EndpointManager.Send @@ -47,8 +49,10 @@ private[akka] object OutboundHandshake { /** * INTERNAL API */ -private[akka] class OutboundHandshake(outboundContext: OutboundContext, timeout: FiniteDuration, - retryInterval: FiniteDuration, injectHandshakeInterval: FiniteDuration) +private[akka] class OutboundHandshake( + system: ActorSystem, + outboundContext: OutboundContext, timeout: FiniteDuration, + retryInterval: FiniteDuration, injectHandshakeInterval: FiniteDuration) extends GraphStage[FlowShape[Send, Send]] { val in: Inlet[Send] = Inlet("OutboundHandshake.in") diff --git a/akka-remote/src/main/scala/akka/remote/artery/NoLiteralCompression.scala b/akka-remote/src/main/scala/akka/remote/artery/NoLiteralCompression.scala new file mode 100644 index 0000000000..be70489b85 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/NoLiteralCompression.scala @@ -0,0 +1,38 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.actor.{ Address, InternalActorRef, ActorSystem, ActorRef } +import akka.util.OptionVal + +/** + * INTERNAL API + * + * Literarily, no compression! + */ +final class NoInboundCompression(system: ActorSystem) extends InboundCompression { + override def hitActorRef(address: Address, ref: ActorRef): Unit = () + override def decompressActorRef(idx: Int): OptionVal[ActorRef] = + if (idx == -1) throw new IllegalArgumentException("Attemted decompression of illegal compression id: -1") + else if (idx == 0) OptionVal.Some(system.deadLetters) // special case deadLetters + else OptionVal.None + + override def hitClassManifest(address: Address, manifest: String): Unit = () + override def decompressClassManifest(idx: Int): OptionVal[String] = + if (idx == -1) throw new IllegalArgumentException("Attemted decompression of illegal compression id: -1") + else OptionVal.None +} + +/** + * INTERNAL API + * + * Literarily, no compression! + */ +object NoOutboundCompression extends OutboundCompression { + override def allocateActorRefCompressionId(ref: ActorRef, id: Int): Unit = () + override def compressActorRef(ref: ActorRef): Int = -1 + + override def allocateClassManifestCompressionId(manifest: String, id: Int): Unit = () + override def compressClassManifest(manifest: String): Int = -1 +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressionTables.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressionTables.scala new file mode 100644 index 0000000000..9aebf26f62 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressionTables.scala @@ -0,0 +1,102 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import akka.actor.{ Address, ActorRef, ActorSystem } +import akka.remote.artery._ +import akka.remote.artery.compress.CompressionProtocol.Events +import akka.serialization.Serialization +import akka.stream.impl.ConstantFun +import akka.util.OptionVal + +/** INTERNAL API */ +private[remote] final class OutboundCompressionImpl(system: ActorSystem, remoteAddress: Address) extends OutboundCompression { + + private val settings = CompressionSettings(system) + + private val actorRefsOut = new OutboundActorRefCompressionTable(system, remoteAddress) + + private val classManifestsOut = new OutboundCompressionTable[String](system, remoteAddress) + + // actor ref compression --- + + override def allocateActorRefCompressionId(ref: ActorRef, id: Int): Unit = actorRefsOut.register(ref, id) + override def compressActorRef(ref: ActorRef): Int = actorRefsOut.compress(ref) + + // class manifest compression --- + + override def compressClassManifest(manifest: String): Int = classManifestsOut.compress(manifest) + override def allocateClassManifestCompressionId(manifest: String, id: Int): Unit = classManifestsOut.register(manifest, id) +} + +/** INTERNAL API */ +private[remote] final class InboundCompressionImpl( + system: ActorSystem, + inboundContext: InboundContext +) extends InboundCompression { + + private val settings = CompressionSettings(system) + private val log = system.log + + private val localAddress = inboundContext.localAddress + + // TODO maybe use inbound context to get remoteAddress instead? + val advertiseActorRef = new AdvertiseCompressionId[ActorRef] { + override def apply(remoteAddress: Address, ref: ActorRef, id: Int): Unit = { + + log.debug(s"Advertise ActorRef compression [$ref => $id] to [$remoteAddress]") + // TODO could use remote address via association lookup??? could be more lookups though + inboundContext.sendControl(remoteAddress, CompressionProtocol.ActorRefCompressionAdvertisement(inboundContext.localAddress, ref, id)) + } + } + val advertiseManifest = new AdvertiseCompressionId[String] { + override def apply(remoteAddress: Address, man: String, id: Int): Unit = { + log.error(s"Advertise ClassManifest compression [$man => $id] to [$remoteAddress]") + inboundContext.sendControl(remoteAddress, CompressionProtocol.ClassManifestCompressionAdvertisement(localAddress, man, id)) + } + } + + private val actorRefHitters = new TopHeavyHitters[ActorRef](settings.actorRefs.max) + private val actorRefsIn = new InboundActorRefCompressionTable(system, actorRefHitters, advertiseActorRef) + + private val manifestHitters = new TopHeavyHitters[String](settings.manifests.max) + private val classManifestsIn = new InboundCompressionTable[String](system, manifestHitters, ConstantFun.scalaIdentityFunction, advertiseManifest) + + // actor ref compression --- + + override def decompressActorRef(idx: Int): OptionVal[ActorRef] = { + val value = actorRefsIn.decompress(idx) + OptionVal.Some(value) + } + override def hitActorRef(address: Address, ref: ActorRef): Unit = { + actorRefsIn.increment(address, ref, 1L) + } + + // class manifest compression --- + + override def decompressClassManifest(idx: Int): OptionVal[String] = { + val value = classManifestsIn.decompress(idx) + OptionVal.Some(value) + } + override def hitClassManifest(address: Address, manifest: String): Unit = { + classManifestsIn.increment(address, manifest, 1L) + } +} + +object NoopInboundCompression extends InboundCompression { + override def hitActorRef(remote: Address, ref: ActorRef): Unit = () + override def decompressActorRef(idx: Int): OptionVal[ActorRef] = OptionVal.None + + override def hitClassManifest(remote: Address, manifest: String): Unit = () + override def decompressClassManifest(idx: Int): OptionVal[String] = OptionVal.None +} + +object NoopOutboundCompression extends OutboundCompression { + override def compressActorRef(ref: ActorRef): Int = -1 + override def allocateActorRefCompressionId(ref: ActorRef, id: Int): Unit = () + + override def compressClassManifest(manifest: String): Int = -1 + override def allocateClassManifestCompressionId(manifest: String, id: Int): Unit = () +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/AdvertiseCompressionId.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/AdvertiseCompressionId.scala new file mode 100644 index 0000000000..92ef0a5840 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/AdvertiseCompressionId.scala @@ -0,0 +1,11 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery.compress + +import akka.actor.Address + +/** Callback invoked when a compression id allocation should be advertised to the remote actor system. */ +trait AdvertiseCompressionId[T] { + def apply(remoteAddress: Address, ref: T, id: Int): Unit +} 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..4dfe2763ce --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionProtocol.scala @@ -0,0 +1,45 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import akka.actor.{ ActorRef, Address } +import akka.remote.UniqueAddress +import akka.remote.artery.ControlMessage + +// FIXME serialization +/** INTERNAL API */ +object CompressionProtocol { + + /** INTERNAL API */ + sealed trait CompressionMessage + + /** + * 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, ref: ActorRef, id: Int) + 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, manifest: String, id: Int) + extends ControlMessage with CompressionMessage + + /** INTERNAL API */ + private[akka] object Events { + /** INTERNAL API */ + private[akka] sealed trait Event + + /** INTERNAL API */ + final case class HeavyHitterDetected(key: Any, id: Int, count: Long) extends Event + + /** INTERNAL API */ + final case class ReceivedCompressionAdvertisement(from: UniqueAddress, key: Any, id: Int) extends Event + + } + +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionSettings.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionSettings.scala new file mode 100644 index 0000000000..2e27d244d5 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionSettings.scala @@ -0,0 +1,36 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery.compress + +import akka.actor.ActorSystem +import com.typesafe.config.Config + +/** INTERNAL API */ +private[akka] class CompressionSettings(_config: Config) { + val enabled = _config.getBoolean("enabled") + @inline private def globalEnabled = enabled + + val debug = _config.getBoolean("debug") + + object actorRefs { + private val c = _config.getConfig("actor-refs") + + val enabled = globalEnabled && c.getBoolean("enabled") + val max = c.getInt("max") + } + object manifests { + private val c = _config.getConfig("manifests") + + val enabled = globalEnabled && c.getBoolean("enabled") + val max = c.getInt("max") + } +} + +/** INTERNAL API */ +private[akka] object CompressionSettings { // TODO make it an extension + def apply(config: Config): CompressionSettings = new CompressionSettings(config) + def apply(system: ActorSystem): CompressionSettings = + new CompressionSettings( + system.settings.config.getConfig("akka.remote.artery.advanced.compression")) +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressionTable.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressionTable.scala new file mode 100644 index 0000000000..bae6264695 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressionTable.scala @@ -0,0 +1,161 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import akka.actor.{ Address, ActorRef, ActorSystem } +import akka.event.Logging + +final class InboundActorRefCompressionTable( + system: ActorSystem, + heavyHitters: TopHeavyHitters[ActorRef], + onNewHeavyHitterDetected: AdvertiseCompressionId[ActorRef] +) extends InboundCompressionTable[ActorRef](system, heavyHitters, _.path.toSerializationFormat, onNewHeavyHitterDetected) { + + preAllocate( + system.deadLetters + ) + + /* Since the table is empty here, anything we increment here becomes a heavy hitter immediately. */ + def preAllocate(allocations: ActorRef*): Unit = { + allocations foreach { case ref ⇒ increment(null, ref, 100000) } + } + + override def shouldAdvertiseCompressionId(idx: Int): Boolean = + idx > 0 // 0 is special => deadLetters + + override def decompress(idx: Int): ActorRef = + if (idx == 0) system.deadLetters + else super.decompress(idx) +} + +/** + * Handles counting and detecting of heavy-hitters and compressing them via a table lookup. + * Mutable and not thread-safe. + * + * Compression flow goes like: + * [1] on each message we add the actor path here + * [2] if it becomes a heavy hitter, we allocate an identifier for it and invoke the callback + * [3]> the callback for example then triggers an CompressionAdvertisement to the receiving side + */ +// TODO should the onHeavyHitter be inside HeavyHitters? +class InboundCompressionTable[T]( + system: ActorSystem, + heavyHitters: TopHeavyHitters[T], + convertKeyToString: T ⇒ String, + onNewHeavyHitterDetected: AdvertiseCompressionId[T]) { + require(heavyHitters != null, "heavyHitters must not be null") + + private val settings = CompressionSettings(system) + val log = Logging(system, "InboundCompressionTable") + + // TODO calibrate properly (h/w have direct relation to preciseness and max capacity) + private[this] val cms = new CountMinSketch(100, 100, System.currentTimeMillis().toInt) + + @volatile private[this] var compressionAllocations = Map.empty[Int, T] // TODO replace with a specialized LongMap + private[this] var currentCompressionId = InboundCompressionTable.CompressionAllocationCounterStart + + /** + * Decompress given identifier into original String representation. + * + * @throws UnknownCompressedIdException if given id is not known, this may indicate a bug – such situation should not happen. + */ + def decompress(idx: Int): T = { + if (settings.debug) log.debug(s"Decompress [{}] => {}", idx, compressionAllocations.get(idx)) + compressionAllocations.get(idx) match { + case Some(value) ⇒ value + case None ⇒ throw new UnknownCompressedIdException(idx) + } + } + + /** + * Add `n` occurance for the given key and call `heavyHittedDetected` if element has become a heavy hitter. + * Empty keys are omitted. + */ + // TODO not so happy about passing around address here, but in incoming there's no other earlier place to get it? + def increment(remoteAddress: Address, value: T, n: Long): Unit = { + val key = convertKeyToString(value) + if (shouldIgnore(key)) { + // ignore... + } else { + // val countBefore = cms.estimateCount(key) + val count = cms.addAndEstimateCount(key, n) + // log.warning(s"HIT: increment $key + $n => ($countBefore->) $count; (addAndCheckIfheavyHitterDetected(value, count) = ${addAndCheckIfheavyHitterDetected(value, count)}); (!wasCompressedPreviously(key) = ${!wasCompressedPreviously(key)})") + + // TODO optimise order of these, what is more expensive? (now the `previous` is, but if aprox datatype there it would be faster)... Needs pondering. + if (addAndCheckIfheavyHitterDetected(value, count) && !wasCompressedPreviously(key)) { + val idx = allocateCompressedId(value) + log.debug("Allocated compression id [" + idx + "] for [" + value + "], in association with [" + remoteAddress + "]") + if (shouldAdvertiseCompressionId(idx)) { // TODO change to "time based accumulate new table => advertise it" + // TODO guard with if + log.debug(s"Inbound: Heavy hitter detected: [{} => $idx], {} hits recorded for it (confidence: {}, relative error (eps) {}).\n" + + s"All allocations: ${compressionAllocations}", key, count, cms.getConfidence, cms.getRelativeError) + onNewHeavyHitterDetected(remoteAddress, value, idx) // would be used to signal via side-channel to OutboundCompression that we want to send a ActorRefCompressionAdvertisement + } + } + } + } + + /** Some compression IDs are special and known upfront by both sides, thus need not be advertised (e.g. deadLetters => 0) */ + def shouldAdvertiseCompressionId(idx: Int): Boolean = + true // TODO this will be different in the "advertise entire table mode", it will be "once table is big enough or much time passed" + + private def shouldIgnore(key: String) = { // TODO this is hacky, if we'd do this we trigger compression too early (before association exists, so control messages fail) + key match { + case null ⇒ true + case "" ⇒ true // empty class manifest for example + case _ ⇒ key.endsWith("/system/dummy") || key.endsWith("/") // TODO dummy likely shouldn't exist? can we remove it? + } + } + + // TODO this must be optimised, we really don't want to scan the entire key-set each time to make sure + private def wasCompressedPreviously(key: String): Boolean = + compressionAllocations.values.exists(_ == key) // TODO expensive, aprox or something sneakier? + + /** Mutates heavy hitters */ + private def addAndCheckIfheavyHitterDetected(value: T, count: Long): Boolean = { + heavyHitters.update(value, count) + } + + private def allocateCompressedId(value: T): Int = { + val idx = nextCompressionId() + compressionAllocations.get(idx) match { + case Some(previouslyCompressedValue) ⇒ + // should never really happen, but let's not assume that + throw new ExistingcompressedIdReuseAttemptException(idx, previouslyCompressedValue) + + case None ⇒ + // good, the idx is not used so we can allocate it + compressionAllocations = compressionAllocations.updated(idx, value) + idx + } + } + + private def nextCompressionId(): Int = { + val id = currentCompressionId + currentCompressionId += 1 + id + } + + override def toString = + s"""${getClass.getSimpleName}(countMinSketch: $cms, heavyHitters: $heavyHitters)""" + +} + +object InboundCompressionTable { + val CompressionAllocationCounterStart = 0 + // val CompressionAllocationCounterStart = 64L // we leave 64 slots (0 counts too) for pre-allocated Akka compressions +} + +final class ExistingcompressedIdReuseAttemptException(id: Long, value: Any) + extends RuntimeException( + s"Attempted to re-allocate compressedId [$id] which is still in use for compressing [$value]! " + + s"This should never happen and is likely an implementation bug.") + +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.") diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompressionTable.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompressionTable.scala new file mode 100644 index 0000000000..0b22effa2b --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompressionTable.scala @@ -0,0 +1,125 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import akka.actor.{ Address, ActorRef, ActorSystem } +import akka.event.Logging + +import scala.annotation.tailrec + +final class OutboundActorRefCompressionTable(system: ActorSystem, remoteAddress: Address) + extends OutboundCompressionTable[ActorRef](system, remoteAddress) { + + preAllocate( + system.deadLetters → 0 + ) + + // if (system.toString.contains("systemB")) + // system.log.error(new Throwable, "new OutboundActorRefCompressionTable = " + this.hashCode()) + + def preAllocate(allocations: (ActorRef, Int)*): Unit = + allocations foreach { case (ref, id) ⇒ register(ref, id) } +} + +/** + * Base class for all outgoing compression. + * Encapsulates the compressedId registration and lookup. + * + * Not thread safe. + */ +class OutboundCompressionTable[T](system: ActorSystem, remoteAddress: Address) { + import OutboundCompressionTable._ + + private val settings = CompressionSettings(system) + + private val log = system.log + + // TODO can we specialize this? (tuning due here) + @volatile private[this] var backing = Map.empty[T, Int] // TODO could use unsafe to swap the map instead of volatile + + // mapping guarding + private[this] var compressionIdsAllocated = -1 + private[this] var aheadAllocatedCompressionIds = Set.empty[Int] + + def register(value: T, id: Int): Unit = { + backing.get(value) match { + case None if isNextCompressionId(id) ⇒ + log.debug("Outbound: Registering new compression from [{}] to [{}].", value, id) // TODO should be debug + addFastForwardCompressionIdsAllocatedCounter() + backing = backing.updated(value, id) + + if (settings.debug) log.debug("Outgoing: Updated compression table state: \n{}", toDebugString) // TODO debug + + case None ⇒ + // TODO could be wrong? since we can not guarantee alocations come in sequence? + if (compressionIdAlreadyAllocated(id)) + throw new AllocatedSameIdMultipleTimesException(id, backing.find(_._2 == id).get._1, value) + + aheadAllocatedCompressionIds += id + backing = backing.updated(value, id) + + case Some(existingId) ⇒ + throw new ConflictingCompressionException(value, id, existingId) + } + } + + def compressionIdAlreadyAllocated(id: Int): Boolean = + id <= compressionIdsAllocated || aheadAllocatedCompressionIds.contains(id) + + def compress(value: T): Int = { + backing.get(value) match { // TODO possibly optimise avoid the Option? Depends on used Map + case None ⇒ NotCompressedId + case Some(id) ⇒ id + } + } + + private def isNextCompressionId(id: Int): Boolean = + id == compressionIdsAllocated + 1 + + private def addFastForwardCompressionIdsAllocatedCounter(): Unit = { + @tailrec def fastForwardConsume(): Unit = { + val nextId = compressionIdsAllocated + 1 + if (aheadAllocatedCompressionIds.contains(nextId)) { + aheadAllocatedCompressionIds = aheadAllocatedCompressionIds.filterNot(_ == nextId) + compressionIdsAllocated += 1 + fastForwardConsume() + } else () + } + + compressionIdsAllocated += 1 + fastForwardConsume() + } + + def toDebugString: String = { + val pad = backing.keys.iterator.map(_.toString.length).max + s"""${Logging.simpleName(getClass)}( + | hashCode: ${this.hashCode()} to [$remoteAddress] + | compressionIdsAllocated: ${compressionIdsAllocated + 1}, + | aheadAllocatedCompressionIds: $aheadAllocatedCompressionIds) + | + | ${backing.map { case (k, v) ⇒ k.toString.padTo(pad, " ").mkString("") + " => " + v }.mkString("\n ")} + |)""".stripMargin + } + + override def toString = + s"""${Logging.simpleName(getClass)}(compressionIdsAllocated: ${compressionIdsAllocated + 1}, aheadAllocatedCompressionIds: $aheadAllocatedCompressionIds)""" +} +object OutboundCompressionTable { + // format: OFF + final val DeadLettersId = 0 + final val NotCompressedId = -1 + // format: ON +} + +final class ConflictingCompressionException(value: Any, id: Int, existingId: Int) + extends IllegalStateException( + s"Value [$value] was already given a compression id [$id], " + + s"yet new compressionId for it was given: $existingId. This could lead to inconsistencies!") + +final class AllocatedSameIdMultipleTimesException(id: Int, previousValue: Any, conflictingValue: Any) + extends IllegalStateException( + s"Attempted to allocate compression id [$id] second time, " + + s"was already bound to value [$previousValue], " + + s"tried to bind to [$conflictingValue]!") 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..70fb236702 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/TopHeavyHitters.scala @@ -0,0 +1,166 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery.compress + +import java.util +import java.util.Objects + +import akka.japi.Util + +import scala.annotation.{ switch, tailrec } +import scala.collection.immutable + +/** + * 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](val max: Int) { + import TopHeavyHitters._ + private[this] var _lowestHitterIdx: Int = 0 + + private[this] val hashes: Array[Int] = Array.ofDim(max) + private[this] val items: Array[T] = Array.ofDim[Object](max).asInstanceOf[Array[T]] + private[this] val weights: Array[Long] = Array.ofDim(max) + + /** Slow operation, mostly exposed for testing and debugging purposes, avoid using in hot paths. */ + def itemsSnapshot: immutable.Seq[T] = Util.immutableSeq(items).filter(_ != null) + + def toDebugString = + s"""TopHeavyHitters( + | max: $max, + | lowestHitterIdx: $lowestHitterIdx (weight: $lowestHitterWeight) + | + | hashes: ${hashes.toList.mkString("[", ", ", "]")} + | weights: ${weights.toList.mkString("[", ", ", "]")} + | items: ${items.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 + (findHashIdx(0, hashCode): @switch) match { // worst case O(n), 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(1) + rarely O(n) if needs to update lowest hitter + + 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) + O(n), we simply replace the current lowest heavy hitter + 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 + + @tailrec private def findItemIdx(searchFromIndex: Int, hashCode: HashCodeVal, o: T): Int = + if (searchFromIndex == -1) -1 + else if (Objects.equals(items(searchFromIndex), o)) searchFromIndex + else findItemIdx(findHashIdx(searchFromIndex + 1, hashCode), 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)) { + putCount(foundHashIndex, count) // we don't need to change `hashCode` or `item`, those remain the same + if (foundHashIndex == lowestHitterIdx) updateLowestHitterIdx() // need to update the lowestHitter since we just bumped its count + 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) + + /** + * 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): Boolean = { + put(_lowestHitterIdx, hashCode, item, count) + updateLowestHitterIdx() + true + } + + /** + * Only update the count for a given index, e.g. if value and hashCode remained the same. + */ + private def putCount(idx: Int, count: Long): Unit = + weights(idx) = count + + private def put(idx: Int, hashCode: HashCodeVal, item: T, count: Long): Unit = { + hashes(idx) = hashCode.get + items(idx) = item + weights(idx) = count + } + + /** Perform a scan for the lowest hitter (by weight). */ + private def updateLowestHitterIdx(): Int = { + _lowestHitterIdx = findIndexOfMinimum(weights) + _lowestHitterIdx + } + + /** Weight of lowest heavy hitter, if a new inserted item has a weight greater than this it is a heavy hitter. */ + def lowestHitterWeight: Long = + weights(_lowestHitterIdx) + + // do not expose we're array based + private def lowestHitterIdx: Int = + _lowestHitterIdx + + private def findEqIndex(hashes: Array[Int], searchFromIndex: Int, hashCode: Int): Int = { + var i: Int = searchFromIndex + while (i < hashes.length) { + if (hashes(i) == hashCode) return i + i += 1 + } + -1 + } + + private def findIndexOfMinimum(weights: Array[Long]): Int = { + var _lowestHitterIdx: Int = -1 + var min: Long = Long.MaxValue + var i: Int = 0 + while (i < weights.length) { + if (weights(i) < min) { + min = weights(i) + _lowestHitterIdx = i + } + i += 1 + } + _lowestHitterIdx + } + + override def toString = + s"${getClass.getSimpleName}(max:$max)" +} + +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/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala index 0184920d37..1fea6b4629 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala @@ -2,17 +2,20 @@ package akka.remote.artery import java.nio.{ ByteBuffer, ByteOrder } +import akka.actor._ +import akka.remote.artery.compress.CompressionTestUtils import akka.testkit.AkkaSpec -import akka.util.ByteString +import akka.util.{ OptionVal, ByteString } class EnvelopeBufferSpec extends AkkaSpec { + import CompressionTestUtils._ - object TestCompressor extends LiteralCompressionTable { - val refToIdx = Map( - "compressable0" → 0, - "compressable1" → 1, - "reallylongcompressablestring" → 2) - val idxToRef = refToIdx.map(_.swap) + object TestCompressor extends InboundCompression with OutboundCompression { + 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, @@ -24,15 +27,20 @@ class EnvelopeBufferSpec extends AkkaSpec { "manifest1" → 1) val idxToManifest = manifestToIdx.map(_.swap) - override def compressActorRef(ref: String): Int = refToIdx.getOrElse(ref, -1) - override def decompressActorRef(idx: Int): String = idxToRef(idx) + override def allocateActorRefCompressionId(ref: ActorRef, id: Int): Unit = ??? // dynamic allocating not implemented here + override def compressActorRef(ref: ActorRef): Int = refToIdx.getOrElse(ref, -1) + override def hitActorRef(address: Address, ref: ActorRef): Unit = () + override def decompressActorRef(idx: Int): OptionVal[ActorRef] = OptionVal.Some(idxToRef(idx)) + + override def allocateClassManifestCompressionId(manifest: String, id: Int): Unit = ??? // dynamic allocating not implemented here override def compressClassManifest(manifest: String): Int = manifestToIdx.getOrElse(manifest, -1) - override def decompressClassManifest(idx: Int): String = idxToManifest(idx) + override def hitClassManifest(address: Address, manifest: String): Unit = () + override def decompressClassManifest(idx: Int) = OptionVal.Some(idxToManifest(idx)) } "EnvelopeBuffer" must { - val headerIn = HeaderBuilder(TestCompressor) - val headerOut = HeaderBuilder(TestCompressor) + val headerIn = HeaderBuilder.bothWays(TestCompressor, TestCompressor) + val headerOut = HeaderBuilder.bothWays(TestCompressor, TestCompressor) val byteBuffer = ByteBuffer.allocate(1024).order(ByteOrder.LITTLE_ENDIAN) val envelope = new EnvelopeBuffer(byteBuffer) @@ -41,8 +49,8 @@ class EnvelopeBufferSpec extends AkkaSpec { headerIn.version = 1 headerIn.uid = 42 headerIn.serializer = 4 - headerIn.senderActorRef = "compressable0" - headerIn.recipientActorRef = "compressable1" + headerIn.senderActorRef = minimalRef("compressable0") + headerIn.recipientActorRef = minimalRef("compressable1") headerIn.manifest = "manifest1" envelope.writeHeader(headerIn) @@ -54,8 +62,8 @@ class EnvelopeBufferSpec extends AkkaSpec { headerOut.version should ===(1) headerOut.uid should ===(42) headerOut.serializer should ===(4) - headerOut.senderActorRef should ===("compressable0") - headerOut.recipientActorRef should ===("compressable1") + headerOut.senderActorRefPath should ===("akka://EnvelopeBufferSpec/compressable0") + headerOut.recipientActorRefPath should ===("akka://EnvelopeBufferSpec/compressable1") headerOut.manifest should ===("manifest1") } @@ -63,14 +71,14 @@ class EnvelopeBufferSpec extends AkkaSpec { headerIn.version = 1 headerIn.uid = 42 headerIn.serializer = 4 - headerIn.senderActorRef = "uncompressable0" - headerIn.recipientActorRef = "uncompressable11" + headerIn.senderActorRef = minimalRef("uncompressable0") + headerIn.recipientActorRef = minimalRef("uncompressable11") headerIn.manifest = "uncompressable3333" val expectedHeaderLength = EnvelopeBuffer.LiteralsSectionOffset + // Constant header part - 2 + headerIn.senderActorRef.length + // Length field + literal - 2 + headerIn.recipientActorRef.length + // Length field + literal + 2 + headerIn.senderActorRefPath.length + // Length field + literal + 2 + headerIn.recipientActorRefPath.length + // Length field + literal 2 + headerIn.manifest.length // Length field + literal envelope.writeHeader(headerIn) @@ -82,8 +90,8 @@ class EnvelopeBufferSpec extends AkkaSpec { headerOut.version should ===(1) headerOut.uid should ===(42) headerOut.serializer should ===(4) - headerOut.senderActorRef should ===("uncompressable0") - headerOut.recipientActorRef should ===("uncompressable11") + headerOut.senderActorRefPath should ===("akka://EnvelopeBufferSpec/uncompressable0") + headerOut.recipientActorRefPath should ===("akka://EnvelopeBufferSpec/uncompressable11") headerOut.manifest should ===("uncompressable3333") } @@ -91,14 +99,14 @@ class EnvelopeBufferSpec extends AkkaSpec { headerIn.version = 1 headerIn.uid = 42 headerIn.serializer = 4 - headerIn.senderActorRef = "reallylongcompressablestring" - headerIn.recipientActorRef = "uncompressable1" + headerIn.senderActorRef = minimalRef("reallylongcompressablestring") + headerIn.recipientActorRef = minimalRef("uncompressable1") headerIn.manifest = "manifest1" envelope.writeHeader(headerIn) envelope.byteBuffer.position() should ===( EnvelopeBuffer.LiteralsSectionOffset + - 2 + headerIn.recipientActorRef.length) + 2 + headerIn.recipientActorRefPath.length) envelope.byteBuffer.flip() envelope.parseHeader(headerOut) @@ -106,21 +114,21 @@ class EnvelopeBufferSpec extends AkkaSpec { headerOut.version should ===(1) headerOut.uid should ===(42) headerOut.serializer should ===(4) - headerOut.senderActorRef should ===("reallylongcompressablestring") - headerOut.recipientActorRef should ===("uncompressable1") + headerOut.senderActorRefPath should ===("akka://EnvelopeBufferSpec/reallylongcompressablestring") + headerOut.recipientActorRefPath should ===("akka://EnvelopeBufferSpec/uncompressable1") headerOut.manifest should ===("manifest1") headerIn.version = 3 headerIn.uid = Long.MinValue headerIn.serializer = -1 - headerIn.senderActorRef = "uncompressable0" - headerIn.recipientActorRef = "reallylongcompressablestring" + headerIn.senderActorRef = minimalRef("uncompressable0") + headerIn.recipientActorRef = minimalRef("reallylongcompressablestring") headerIn.manifest = "longlonglongliteralmanifest" envelope.writeHeader(headerIn) envelope.byteBuffer.position() should ===( EnvelopeBuffer.LiteralsSectionOffset + - 2 + headerIn.senderActorRef.length + + 2 + headerIn.senderActorRefPath.length + 2 + headerIn.manifest.length) envelope.byteBuffer.flip() @@ -129,8 +137,8 @@ class EnvelopeBufferSpec extends AkkaSpec { headerOut.version should ===(3) headerOut.uid should ===(Long.MinValue) headerOut.serializer should ===(-1) - headerOut.senderActorRef should ===("uncompressable0") - headerOut.recipientActorRef should ===("reallylongcompressablestring") + headerOut.senderActorRefPath should ===("akka://EnvelopeBufferSpec/uncompressable0") + headerOut.recipientActorRefPath should ===("akka://EnvelopeBufferSpec/reallylongcompressablestring") headerOut.manifest should ===("longlonglongliteralmanifest") } @@ -140,8 +148,8 @@ class EnvelopeBufferSpec extends AkkaSpec { headerIn.version = 1 headerIn.uid = 42 headerIn.serializer = 4 - headerIn.senderActorRef = "reallylongcompressablestring" - headerIn.recipientActorRef = "uncompressable1" + headerIn.senderActorRef = minimalRef("reallylongcompressablestring") + headerIn.recipientActorRef = minimalRef("uncompressable1") headerIn.manifest = "manifest1" envelope.writeHeader(headerIn) @@ -153,8 +161,8 @@ class EnvelopeBufferSpec extends AkkaSpec { headerOut.version should ===(1) headerOut.uid should ===(42) headerOut.serializer should ===(4) - headerOut.senderActorRef should ===("reallylongcompressablestring") - headerOut.recipientActorRef should ===("uncompressable1") + headerOut.senderActorRefPath should ===("akka://EnvelopeBufferSpec/reallylongcompressablestring") + headerOut.recipientActorRefPath should ===("akka://EnvelopeBufferSpec/uncompressable1") headerOut.manifest should ===("manifest1") ByteString.fromByteBuffer(envelope.byteBuffer) should ===(payload) diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala index 44c79967b3..16a7605a17 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala @@ -5,7 +5,7 @@ package akka.remote.artery import scala.concurrent.duration._ -import akka.actor.{ ActorIdentity, ActorSystem, Identify } +import akka.actor._ import akka.testkit.{ AkkaSpec, ImplicitSender } import akka.testkit.SocketUtil import akka.testkit.TestActors @@ -20,9 +20,11 @@ object HandshakeRetrySpec { akka { actor.provider = remote remote.artery.enabled = on + remote.artery.compression.enabled = on remote.artery.hostname = localhost remote.artery.port = 0 remote.handshake-timeout = 10s + } """) diff --git a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala index 1b3adef27d..3dd7227cb6 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala @@ -39,7 +39,7 @@ class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { val destination = null.asInstanceOf[RemoteActorRef] // not used TestSource.probe[String] .map(msg ⇒ Send(msg, OptionVal.None, destination, None)) - .via(new OutboundHandshake(outboundContext, timeout, retryInterval, injectHandshakeInterval)) + .via(new OutboundHandshake(system, outboundContext, timeout, retryInterval, injectHandshakeInterval)) .map { case Send(msg, _, _, _) ⇒ msg } .toMat(TestSink.probe[Any])(Keep.both) .run() diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala index 34371d7673..0035043f87 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -75,7 +75,7 @@ private[akka] class TestOutboundContext( _associationState.uniqueRemoteAddress.value match { case Some(Success(`peer`)) ⇒ // our value case _ ⇒ - _associationState = _associationState.newIncarnation(Promise.successful(peer)) + _associationState = _associationState.newIncarnation(Promise.successful(peer), NoOutboundCompression) } } @@ -96,7 +96,7 @@ private[akka] class TestOutboundContext( private[akka] class TestControlMessageSubject extends ControlMessageSubject { - private var observers = new CopyOnWriteArrayList[ControlMessageObserver] + private val observers = new CopyOnWriteArrayList[ControlMessageObserver] override def attach(observer: ControlMessageObserver): Future[Done] = { observers.add(observer) 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..8d2c230ec8 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala @@ -0,0 +1,94 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import akka.actor._ +import akka.remote.artery.compress.CompressionProtocol.Events +import akka.testkit._ +import akka.util.Timeout +import com.typesafe.config.ConfigFactory +import org.scalatest.BeforeAndAfter +import akka.pattern.ask + +import scala.concurrent.Await +import scala.concurrent.duration._ + +object CompressionIntegrationSpec { + // need the port before systems are 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.advanced { + compression.enabled = on + compression.debug = on + } + remote.artery.hostname = localhost + remote.artery.port = 0 + remote.handshake-timeout = 10s + + } + """) + + val configB = ConfigFactory.parseString(s"akka.remote.artery.port = $portB") + .withFallback(commonConfig) +} + +class CompressionIntegrationSpec extends AkkaSpec(CompressionIntegrationSpec.commonConfig) + with ImplicitSender with BeforeAndAfter { + import CompressionIntegrationSpec._ + + implicit val t = Timeout(3.seconds) + var systemB: ActorSystem = null + + before { + systemB = ActorSystem("systemB", configB) + } + + "Outgoing compression table" must { + "compress chatty actor" in { + val messagesToExchange = 10 + + // listen for compression table events + val aProbe = TestProbe() + val b1Probe = TestProbe() + system.eventStream.subscribe(aProbe.ref, classOf[CompressionProtocol.Events.Event]) + systemB.eventStream.subscribe(b1Probe.ref, classOf[CompressionProtocol.Events.Event]) + + def voidSel = system.actorSelection(s"artery://systemB@localhost:$portB/user/void") + systemB.actorOf(TestActors.blackholeProps, "void") + + // cause testActor-1 to become a heavy hitter + (1 to messagesToExchange).foreach { i ⇒ voidSel ! "hello" } // does not reply, but a hot receiver should be advertised + + val a1 = aProbe.expectMsgType[Events.ReceivedCompressionAdvertisement](10.seconds) + info("System [A] received: " + a1) + a1.id should ===(1) + a1.key.toString should include(testActor.path.name) + } + } + + def identify(_system: String, port: Int, name: String) = { + val selection = + system.actorSelection(s"artery://${_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/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..9549072906 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala @@ -0,0 +1,128 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import akka.actor.{ ActorIdentity, ActorSystem, Identify } +import akka.remote.artery.compress.CompressionProtocol.Events +import akka.testkit._ +import akka.util.Timeout +import akka.pattern.ask +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.advanced { + compression.enabled = on + compression.debug = on + } + remote.artery.hostname = localhost + remote.artery.port = 0 + remote.handshake-timeout = 10s + + } + """) + + val configB = ConfigFactory.parseString(s"akka.remote.artery.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 + + // listen for compression table events + val aProbe = TestProbe() + val a1Probe = TestProbe() + val aNew2Probe = TestProbe() + val b1Probe = TestProbe() + system.eventStream.subscribe(aProbe.ref, classOf[CompressionProtocol.Events.Event]) + systemB.eventStream.subscribe(b1Probe.ref, classOf[CompressionProtocol.Events.Event]) + + def voidSel = system.actorSelection(s"artery://systemB@localhost:$portB/user/void") + systemB.actorOf(TestActors.blackholeProps, "void") + + // cause testActor-1 to become a heavy hitter + (1 to messagesToExchange).foreach { i ⇒ voidSel ! "hello" } // does not reply, but a hot receiver should be advertised + // give it enough time to advertise first table + val a0 = aProbe.expectMsgType[Events.ReceivedCompressionAdvertisement](10.seconds) + info("System [A] received: " + a0) + a0.id should ===(1) + a0.key.toString should include(testActor.path.name) + + // cause a1Probe to become a heavy hitter (we want to not have it in the 2nd compression table later) + (1 to messagesToExchange).foreach { i ⇒ voidSel.tell("hello", a1Probe.ref) } // does not reply, but a hot receiver should be advertised + // give it enough time to advertise first table + val a1 = aProbe.expectMsgType[Events.ReceivedCompressionAdvertisement](10.seconds) + info("System [A] received: " + a1) + a1.id should ===(2) + a1.key.toString should include(a1Probe.ref.path.name) + + log.warning("SHUTTING DOWN system {}...", systemB) + shutdown(systemB) + systemB = ActorSystem("systemB", configB) + Thread.sleep(5000) + log.warning("SYSTEM READY {}...", systemB) + + systemB.actorOf(TestActors.blackholeProps, "void") // start it again + (1 to messagesToExchange).foreach { i ⇒ voidSel ! "hello" } // does not reply, but a hot receiver should be advertised + // compression triggered again + val a2 = aProbe.expectMsgType[Events.ReceivedCompressionAdvertisement](10.seconds) + info("System [A] received: " + a2) + a2.id should ===(1) + a2.key.toString should include(testActor.path.name) + + (1 to messagesToExchange).foreach { i ⇒ voidSel.tell("hello", aNew2Probe.ref) } // does not reply, but a hot receiver should be advertised + // compression triggered again + val a3 = aProbe.expectMsgType[Events.ReceivedCompressionAdvertisement](10.seconds) + info("Received second compression: " + a3) + a3.id should ===(2) + a3.key.toString should include(aNew2Probe.ref.path.name) + } + + } + + def identify(_system: String, port: Int, name: String) = { + val selection = + system.actorSelection(s"artery://${_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..3120140b93 --- /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](3) + hitters.update("A", 10) shouldBe true + hitters.itemsSnapshot.toSet should ===(Set("A")) + + hitters.update("B", 20) shouldBe true + hitters.itemsSnapshot.toSet should ===(Set("A", "B")) + + hitters.update("C", 1) shouldBe true + hitters.itemsSnapshot.toSet should ===(Set("A", "B", "C")) + + hitters.update("D", 100) shouldBe true + hitters.itemsSnapshot.toSet should ===(Set("A", "B", "D")) + + hitters.update("E", 200) shouldBe true + hitters.itemsSnapshot.toSet should ===(Set("B", "D", "E")) + + hitters.update("BB", 22) shouldBe true + hitters.itemsSnapshot.toSet should ===(Set("BB", "D", "E")) + + hitters.update("a", 1) shouldBe false + hitters.itemsSnapshot.toSet should ===(Set("BB", "D", "E")) + } + + "correctly replace a hitter" in { + val hitters = new TopHeavyHitters[String](3) + hitters.update("A", 10) shouldBe true + hitters.itemsSnapshot.toSet should ===(Set("A")) + + hitters.update("A", 12) shouldBe false + hitters.update("A", 22) shouldBe false + hitters.itemsSnapshot.toSet should ===(Set("A")) + } + + "correctly drop least heavy hitter when more than N are inserted" in { + val hitters = new TopHeavyHitters[String](3) + + hitters.update("A", 1) shouldBe true + hitters.itemsSnapshot.toSet should ===(Set("A")) + + hitters.update("B", 22) shouldBe true + hitters.itemsSnapshot.toSet should ===(Set("A", "B")) + + hitters.update("C", 33) shouldBe true + hitters.itemsSnapshot.toSet should ===(Set("A", "B", "C")) + hitters.lowestHitterWeight should ===(1) + + // first item which forces dropping least heavy hitter + hitters.update("D", 100) shouldBe true + hitters.itemsSnapshot.toSet should ===(Set("B", "C", "D")) + + // second item which forces dropping least heavy hitter + hitters.update("X", 999) shouldBe true + hitters.itemsSnapshot.toSet should ===(Set("X", "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.itemsSnapshot.toSet should ===(Set(a1)) + hitters.lowestHitterWeight should ===(0) + + hitters.update(b1, 2) + hitters.itemsSnapshot.toSet should ===(Set(a1, b1)) + hitters.lowestHitterWeight should ===(1) + + hitters.update(a1, 10) + hitters.itemsSnapshot.toSet should ===(Set(a1, b1)) + hitters.lowestHitterWeight should ===(2) + + hitters.update(b1, 100) + hitters.itemsSnapshot.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/InboundCompressionTableSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/InboundCompressionTableSpec.scala new file mode 100644 index 0000000000..7b7d4688eb --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/InboundCompressionTableSpec.scala @@ -0,0 +1,62 @@ +/* + * Copyright (C) 2009-2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import akka.actor.Address +import akka.stream.impl.ConstantFun +import akka.testkit.AkkaSpec + +class InboundCompressionTableSpec extends AkkaSpec { + + "InboundCompressionTable" must { + val NoChange: (String, Int) = null + + "invoke callback when compression triggered" in { + var p: (String, Int) = NoChange + val heavyHitters = new TopHeavyHitters[String](2) + val advertiseCompressionId = new AdvertiseCompressionId[String] { + override def apply(remoteAddress: Address, ref: String, id: Int): Unit = + p = ref → id + } + val table = new InboundCompressionTable[String](system, heavyHitters, ConstantFun.scalaIdentityFunction, advertiseCompressionId) + + table.increment(null, "A", 1L) + p should ===("A" → 0) + + table.increment(null, "B", 1L) + p should ===("B" → 1) + + p = NoChange + table.increment(null, "A", 1L) // again, yet was already compressed (A count == 2), thus no need to compress (call callback) again + p should ===(NoChange) // no change + + table.increment(null, "B", 1L) // again, yet was already compressed (B count == 2), thus no need to compress (call callback) again + p should ===(NoChange) // no change + + table.increment(null, "C", 1L) // max hitters = 2; [A=2, B=2] C=1 + p should ===(NoChange) // no change + + table.increment(null, "C", 1L) // max hitters = 2; [A=2, B=2] C=2 – causes compression of C! + p should ===(NoChange) // no change + table.increment(null, "C", 1L) // max hitters = 2; [..., C=3] – causes compression of C! + p should ===("C" → 2) // allocated + + p = NoChange + table.increment(null, "A", 1L) // again! + p should ===(NoChange) + + p = NoChange + table.increment(null, "B", 1L) // again! + p should ===(NoChange) + + // and again and again... won't be signalled again since already compressed + table.increment(null, "A", 1L) + table.increment(null, "A", 1L) + table.increment(null, "A", 1L) + p should ===(NoChange) + } + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/compress/OutboundCompressionTableSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/OutboundCompressionTableSpec.scala new file mode 100644 index 0000000000..5af8be2fbe --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/OutboundCompressionTableSpec.scala @@ -0,0 +1,81 @@ +/* + * Copyright (C) 2009-2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import akka.actor._ +import akka.testkit.AkkaSpec + +class OutboundCompressionTableSpec extends AkkaSpec { + import CompressionTestUtils._ + + val remoteAddress = Address("artery", "example", "localhost", 0) + + "OutboundCompressionTable" must { + "not compress not-known values" in { + val table = new OutboundActorRefCompressionTable(system, remoteAddress) + table.compress(minimalRef("banana")) should ===(-1) + } + } + + "OutboundActorRefCompressionTable" must { + val alice = minimalRef("alice") + val bob = minimalRef("bob") + + "always compress /deadLetters" in { + val table = new OutboundActorRefCompressionTable(system, remoteAddress) + table.compress(system.deadLetters) should ===(0) + } + + "not compress unknown actor ref" in { + val table = new OutboundActorRefCompressionTable(system, remoteAddress) + table.compress(alice) should ===(-1) // not compressed + } + + "compress previously registered actor ref" in { + val table = new OutboundActorRefCompressionTable(system, remoteAddress) + table.register(alice, 1) + table.compress(alice) should ===(1) // compressed + + table.compress(bob) should ===(-1) // not compressed + } + + "fail if same id attempted to be registered twice" in { + val table = new OutboundActorRefCompressionTable(system, remoteAddress) + table.register(alice, 1) + val ex = intercept[AllocatedSameIdMultipleTimesException] { + table.register(bob, 1) + } + + ex.getMessage should include("Attempted to allocate compression id [1] second time, " + + "was already bound to value [Actor[akka://OutboundCompressionTableSpec/alice]], " + + "tried to bind to [Actor[akka://OutboundCompressionTableSpec/bob]]!") + } + + "survive compression ahead-allocation, and then fast forward allocated Ids counter when able to (compact storage)" in { + val table = new OutboundActorRefCompressionTable(system, remoteAddress) + table.register(alice, 1) + table.compressionIdAlreadyAllocated(1) should ===(true) + + table.register(bob, 3) // ahead allocated + table.compressionIdAlreadyAllocated(2) should ===(false) + table.compressionIdAlreadyAllocated(3) should ===(true) + + table.register(minimalRef("oogie-boogie"), 4) // ahead allocated (we're able to survive re-delivery of allocation messages) + table.compressionIdAlreadyAllocated(2) should ===(false) + table.compressionIdAlreadyAllocated(4) should ===(true) + + table.register(minimalRef("jack-skellington"), 2) // missing allocation was re-delivered, cause fast-forward + + table.compressionIdAlreadyAllocated(2) should ===(true) + + table.register(minimalRef("jack-sparrow"), 5) // immediate next, after fast-forward + } + + // FIXME "fast forward" concept will not exist once we use "advertise entire table", possibly remove mentions of that + // TODO cover more cases of holes in the redeliveries of advertisements + // TODO ^ to cover the fast forward logic a bit more + } + +} 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/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/plugins.sbt b/project/plugins.sbt index e863d3bbdb..a8d01a6513 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -25,7 +25,9 @@ addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.3") addSbtPlugin("com.thoughtworks.sbt-api-mappings" % "sbt-api-mappings" % "0.2.2") -addSbtPlugin("pl.project13.scala" % "sbt-jmh" % "0.2.3") +addSbtPlugin("pl.project13.scala" % "sbt-jmh" % "0.2.6") + +addSbtPlugin("pl.project13.sbt" % "sbt-jol" % "0.1.1") addSbtPlugin("com.typesafe.sbt" % "sbt-native-packager" % "1.0.0-RC1") From b6a94e1758548fba5c680fc03d2d925771ca4744 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 23 Jun 2016 16:36:55 +0200 Subject: [PATCH 078/186] fix bug in SystemMessageAcker, #20709 (#20792) * sequence numbers must, of course, be tracked by origin system * add unit test for SystemMessageAcker stage * enable ArteryRemoteRoundRobinSpec --- .../remote/AttemptSysMsgRedeliverySpec.scala | 13 +- .../akka/remote/RemoteReDeploymentSpec.scala | 5 - .../remote/routing/RemoteRoundRobinSpec.scala | 12 +- .../remote/artery/SystemMessageDelivery.scala | 17 ++- .../akka/remote/artery/RemoteRouterSpec.scala | 1 - .../artery/SystemMessageAckerSpec.scala | 132 ++++++++++++++++++ 6 files changed, 153 insertions(+), 27 deletions(-) create mode 100644 akka-remote/src/test/scala/akka/remote/artery/SystemMessageAckerSpec.scala 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 b9abfe25a2..35badd1cc0 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 @@ -37,13 +37,12 @@ class AttemptSysMsgRedeliveryMultiJvmNode2 extends AttemptSysMsgRedeliverySpec( class AttemptSysMsgRedeliveryMultiJvmNode3 extends AttemptSysMsgRedeliverySpec( new AttemptSysMsgRedeliveryMultiJvmSpec(artery = false)) -// FIXME this is failing with Artery -//class ArteryAttemptSysMsgRedeliveryMultiJvmNode1 extends AttemptSysMsgRedeliverySpec( -// new AttemptSysMsgRedeliveryMultiJvmSpec(artery = true)) -//class ArteryAttemptSysMsgRedeliveryMultiJvmNode2 extends AttemptSysMsgRedeliverySpec( -// new AttemptSysMsgRedeliveryMultiJvmSpec(artery = true)) -//class ArteryAttemptSysMsgRedeliveryMultiJvmNode3 extends AttemptSysMsgRedeliverySpec( -// new AttemptSysMsgRedeliveryMultiJvmSpec(artery = true)) +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 { 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 9be5c92b48..456d8a262a 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 @@ -134,11 +134,6 @@ abstract class RemoteReDeploymentMultiJvmSpec(multiNodeConfig: RemoteReDeploymen enterBarrier("first-deployed") - // FIXME When running with Artery: - // [akka://RemoteReDeploymentMultiJvmSpec/user/parent] received Supervise from unregistered child - // Actor[artery://RemoteReDeploymentMultiJvmSpec@localhost:55627/remote/artery/RemoteReDeploymentMultiJvmSpec@localhost:65490/user/parent/hello#-370928728], - // this will not end well - runOn(first) { testConductor.blackhole(second, first, Both).await testConductor.shutdown(second, abort = true).await 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 6b1100008d..157c8fb0e3 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 @@ -65,14 +65,10 @@ class RemoteRoundRobinMultiJvmNode2 extends RemoteRoundRobinSpec(new RemoteRound class RemoteRoundRobinMultiJvmNode3 extends RemoteRoundRobinSpec(new RemoteRoundRobinConfig(artery = false)) class RemoteRoundRobinMultiJvmNode4 extends RemoteRoundRobinSpec(new RemoteRoundRobinConfig(artery = false)) -// FIXME this test fails with Artery -// [akka://RemoteRoundRobinSpec/user/service-hello2] received Supervise from unregistered child -// Actor[artery://RemoteRoundRobinSpec@localhost:52247/remote/artery/RemoteRoundRobinSpec@localhost:56386/user/service-hello2/c2#-2080820302], -// this will not end well -//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)) +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 { diff --git a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala index de2ee9da60..12bb70cec0 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -233,7 +233,8 @@ private[akka] class SystemMessageAcker(inboundContext: InboundContext) extends G override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) with InHandler with OutHandler { - var seqNo = 1L + // TODO we might need have to prune old unused entries + var sequenceNumbers = Map.empty[UniqueAddress, Long] def localAddress = inboundContext.localAddress @@ -242,16 +243,20 @@ private[akka] class SystemMessageAcker(inboundContext: InboundContext) extends G val env = grab(in) env.message match { case sysEnv @ SystemMessageEnvelope(_, n, ackReplyTo) ⇒ - if (n == seqNo) { + val expectedSeqNo = sequenceNumbers.get(ackReplyTo) match { + case None ⇒ 1L + case Some(seqNo) ⇒ seqNo + } + if (n == expectedSeqNo) { inboundContext.sendControl(ackReplyTo.address, Ack(n, localAddress)) - seqNo += 1 + sequenceNumbers = sequenceNumbers.updated(ackReplyTo, n + 1) val unwrapped = env.withMessage(sysEnv.message) push(out, unwrapped) - } else if (n < seqNo) { - inboundContext.sendControl(ackReplyTo.address, Ack(n, localAddress)) + } else if (n < expectedSeqNo) { + inboundContext.sendControl(ackReplyTo.address, Ack(expectedSeqNo - 1, localAddress)) pull(in) } else { - inboundContext.sendControl(ackReplyTo.address, Nack(seqNo - 1, localAddress)) + inboundContext.sendControl(ackReplyTo.address, Nack(expectedSeqNo - 1, localAddress)) pull(in) } case _ ⇒ diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala index 1f724b3fea..d55b842c53 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala @@ -102,7 +102,6 @@ class RemoteRouterSpec extends AkkaSpec(""" val probe = TestProbe()(masterSystem) val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps), "blub") val replies = collectRouteePaths(probe, router, 5) - println(s"# replies $replies") // FIXME val children = replies.toSet children should have size 2 children.map(_.parent) should have size 1 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..58585a2e6a --- /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("artery", "sysA", "hostA", 1001), 1) + val addressB = UniqueAddress(Address("artery", "sysB", "hostB", 1002), 2) + val addressC = UniqueAddress(Address("artery", "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, addressA.address, 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() + } + + } + +} From 7c79b40deacf589b07196afef11778b972ce350a Mon Sep 17 00:00:00 2001 From: Konrad Malawski Date: Fri, 24 Jun 2016 11:21:21 +0200 Subject: [PATCH 079/186] +tes introduce simple way to gather flamegraphs from multinode specs --- .../akka/remote/testkit/MultiNodeSpec.scala | 9 ++- .../remote/testkit/PerfFlamesSupport.scala | 58 +++++++++++++++++++ .../remote/artery/MaxThroughputSpec.scala | 8 ++- 3 files changed, 70 insertions(+), 5 deletions(-) create mode 100644 akka-multi-node-testkit/src/main/scala/akka/remote/testkit/PerfFlamesSupport.scala 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 9e637341c1..dabd86f341 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 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..34311bd1b1 --- /dev/null +++ b/akka-multi-node-testkit/src/main/scala/akka/remote/testkit/PerfFlamesSupport.scala @@ -0,0 +1,58 @@ +/* + * 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-remote-tests/src/multi-jvm/scala/akka/remote/artery/MaxThroughputSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/MaxThroughputSpec.scala index 9e081ff638..19bf6c8e99 100644 --- 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 @@ -6,13 +6,14 @@ package akka.remote.artery import java.nio.ByteBuffer import java.util.concurrent.Executors import java.util.concurrent.TimeUnit.NANOSECONDS -import scala.concurrent.duration._ +import scala.concurrent.duration._ import akka.actor._ import akka.remote.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 @@ -199,7 +200,8 @@ class MaxThroughputSpecMultiJvmNode2 extends MaxThroughputSpec abstract class MaxThroughputSpec extends MultiNodeSpec(MaxThroughputSpec) - with STMultiNodeSpec with ImplicitSender { + with STMultiNodeSpec with ImplicitSender + with PerfFlamesSupport { import MaxThroughputSpec._ @@ -269,6 +271,8 @@ abstract class MaxThroughputSpec 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) { From d99274a51ffdb374636fa800fe79574684d2d1ab Mon Sep 17 00:00:00 2001 From: Konrad Malawski Date: Fri, 1 Jul 2016 11:54:57 +0200 Subject: [PATCH 080/186] =art #20455 compression tables advertised as a whole "2 tables mode" (#20863) Squashed commit of the following: commit 6dc45364eb285338885bc8a5f1c4f293a29a53fb Author: Konrad Malawski Date: Fri Jun 24 19:03:58 2016 +0200 =art moved successfuly to 2 table mode envelope format prepared, versioned tables 2 table mode working commit 517723c5d61969988a9a93b99666824bf5bccb52 Author: Konrad Malawski Date: Fri Jun 24 10:28:02 2016 +0200 WIP commit 3e05a733e087e0d5bd8df9cc4fff0d4bc1314ec8 Author: Konrad Malawski Date: Wed May 18 02:28:12 2016 +0200 commit b51f1766a94b202cd42fcc9d5402327ad0800d2d Author: Konrad Malawski Date: Thu Apr 28 10:56:58 2016 +0200 +art #20455 HeavyHitters and CountMinSketch prepared for Compression --- .../main/scala/akka/util/PrettyDuration.scala | 3 +- .../akka/remote/artery/CodecBenchmark.scala | 20 +- .../InvertCompressionTableBenchmark.scala | 27 ++ .../OutboundCompressionTableBenchmark.scala | 60 ---- .../remote/artery/MaxThroughputSpec.scala | 7 +- .../artery/compress/CountMinSketch.java | 8 +- akka-remote/src/main/resources/reference.conf | 10 +- .../scala/akka/remote/MessageSerializer.scala | 10 +- .../akka/remote/artery/ArteryTransport.scala | 67 +++-- .../akka/remote/artery/Association.scala | 34 +-- .../scala/akka/remote/artery/BufferPool.scala | 179 ++++++++---- .../scala/akka/remote/artery/Codecs.scala | 42 +-- .../remote/artery/NoLiteralCompression.scala | 19 +- .../compress/ActualCompressionTables.scala | 102 ------- .../artery/compress/ActualCompressions.scala | 99 +++++++ .../artery/compress/CompressionProtocol.scala | 7 +- .../artery/compress/CompressionSettings.scala | 6 + .../artery/compress/CompressionTable.scala | 30 ++ .../artery/compress/DecompressionTable.scala | 28 ++ .../compress/InboundCompressionTable.scala | 161 ---------- .../artery/compress/InboundCompressions.scala | 276 ++++++++++++++++++ .../OutboundActorRefCompression.scala | 106 +++++++ .../OutboundActorRefCompressionTable.scala | 125 -------- .../artery/compress/TopHeavyHitters.scala | 9 +- .../remote/artery/EnvelopeBufferSpec.scala | 137 +++++---- .../akka/remote/artery/TestContext.scala | 2 +- .../compress/CompressionIntegrationSpec.scala | 20 +- .../compress/CompressionTableSpec.scala | 36 +++ .../artery/compress/CompressionTestKit.scala | 17 ++ ...dshakeShouldDropCompressionTableSpec.scala | 38 +-- .../artery/compress/HeavyHittersSpec.scala | 36 +-- .../InboundCompressionTableSpec.scala | 62 ---- .../compress/OutboundCompressionSpec.scala | 50 ++++ .../OutboundCompressionTableSpec.scala | 81 ----- 34 files changed, 1062 insertions(+), 852 deletions(-) create mode 100644 akka-bench-jmh/src/main/scala/akka/remote/artery/compress/InvertCompressionTableBenchmark.scala delete mode 100644 akka-bench-jmh/src/main/scala/akka/remote/compress/OutboundCompressionTableBenchmark.scala delete mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressionTables.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressions.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala delete mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressionTable.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompression.scala delete mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompressionTable.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTableSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTestKit.scala delete mode 100644 akka-remote/src/test/scala/akka/remote/artery/compress/InboundCompressionTableSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/compress/OutboundCompressionSpec.scala delete mode 100644 akka-remote/src/test/scala/akka/remote/artery/compress/OutboundCompressionTableSpec.scala 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/CodecBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala index 9a1da92bd6..0454c0e1c1 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -60,8 +60,8 @@ class CodecBenchmark { create = () ⇒ new ReusableInboundEnvelope, clear = inEnvelope ⇒ inEnvelope.asInstanceOf[ReusableInboundEnvelope].clear() ) - val compressionOut = NoOutboundCompression - val headerIn = HeaderBuilder.in(NoopInboundCompression) + val compressionOut = NoOutboundCompressions + val headerIn = HeaderBuilder.in(NoopInboundCompressions) val envelopeTemplateBuffer = ByteBuffer.allocate(ArteryTransport.MaximumFrameSize).order(ByteOrder.LITTLE_ENDIAN) val uniqueLocalAddress = UniqueAddress( @@ -103,12 +103,12 @@ class CodecBenchmark { recipientStringB = remoteRefB.path.toSerializationFormatWithAddress(addressB) val envelope = new EnvelopeBuffer(envelopeTemplateBuffer) - headerIn.version = 1 - headerIn.uid = 42 - headerIn.serializer = 4 - headerIn.senderActorRef = actorOnSystemA - headerIn.recipientActorRef = remoteRefB - headerIn.manifest = "" + 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() @@ -169,7 +169,7 @@ class CodecBenchmark { val decoder: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = Flow.fromGraph(new Decoder(inboundContext, system.asInstanceOf[ExtendedActorSystem], - resolveActorRefWithLocalAddress, NoopInboundCompression, envelopePool, inboundEnvelopePool)) + resolveActorRefWithLocalAddress, NoopInboundCompressions, envelopePool, inboundEnvelopePool)) Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) .map { _ => @@ -210,7 +210,7 @@ class CodecBenchmark { val decoder: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = Flow.fromGraph(new Decoder(inboundContext, system.asInstanceOf[ExtendedActorSystem], - resolveActorRefWithLocalAddress, NoopInboundCompression, envelopePool, inboundEnvelopePool)) + resolveActorRefWithLocalAddress, NoopInboundCompressions, envelopePool, inboundEnvelopePool)) Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) .map(_ ⇒ Send(payload, OptionVal.None, remoteRefB, None)) 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..bc6cf2eba1 --- /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(2, Map(Vector.fill[String](256)(randomName).zipWithIndex: _*)) + val compTable_1024 = CompressionTable(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/OutboundCompressionTableBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/remote/compress/OutboundCompressionTableBenchmark.scala deleted file mode 100644 index 7f7da57b01..0000000000 --- a/akka-bench-jmh/src/main/scala/akka/remote/compress/OutboundCompressionTableBenchmark.scala +++ /dev/null @@ -1,60 +0,0 @@ -/** - * Copyright (C) 2016 Lightbend Inc. - */ -package akka.remote.compress - -import java.util.Random - -import akka.actor.{ ActorSystem, Address } -import akka.event.NoLogging -import akka.remote.artery.compress.{ OutboundCompressionTable, TopHeavyHitters } -import org.openjdk.jmh.annotations._ -import org.openjdk.jmh.infra.Blackhole - -@State(Scope.Benchmark) -@BenchmarkMode(Array(Mode.Throughput)) -@Fork(2) -class OutboundCompressionTableBenchmark { - - @Param(Array("512", "8192")) - var registered: Int = 0 - - implicit val system = ActorSystem("TestSystem") - - var outgoingCompression: OutboundCompressionTable[String] = _ - - val rand = new Random(1001021) - - var preallocatedNums: Array[Long] = _ - var preallocatedStrings: Array[String] = _ - - var i = 0 - - @Setup - def init(): Unit = { - preallocatedNums = Array.ofDim(registered) - preallocatedStrings = Array.ofDim(8192) - - outgoingCompression = new OutboundCompressionTable(system, Address("akka", "remote-system")) - - var i = 0 - while (i < registered) { - outgoingCompression.register(i.toString, i) - preallocatedNums(i) = rand.nextLong() - preallocatedStrings(i) = i.toString - i += 1 - } - } - - // @Benchmark - // @BenchmarkMode(Array(Mode.SingleShotTime)) - // def registerThenCompress(): Int = { - // outgoingCompression.register("new", i) - // outgoingCompression.compress("new") - // } - - @Benchmark - def compressKnown(): Int = - outgoingCompression.compress("1") - -} 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 index 19bf6c8e99..121a166dc6 100644 --- 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 @@ -6,7 +6,6 @@ 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.RemoteActorRefProvider @@ -55,6 +54,12 @@ object MaxThroughputSpec extends MultiNodeConfig { # See akka-remote-tests/src/test/resources/aeron.properties #advanced.embedded-media-driver = off #advanced.aeron-dir = "target/aeron" + + #advanced.compression { + # enabled = on + # actor-refs.enabled = on + # manifests.enabled = on + #} } } """))) 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 index 577b8718c2..37b2baffad 100644 --- a/akka-remote/src/main/java/akka/remote/artery/compress/CountMinSketch.java +++ b/akka-remote/src/main/java/akka/remote/artery/compress/CountMinSketch.java @@ -198,14 +198,20 @@ public class CountMinSketch { // TODO replace with Scala's Murmur3, it's much faster private static class MurmurHash { + // FIXME: This overload isn't actually ever used 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? + return o.hashCode(); + } if (o instanceof String) { return hash(((String) o).getBytes()); } - // TODO consider calling hashCode on ActorRef here directly? It is just a random number though so possibly not as evenly distributed...? if (o instanceof Long) { return hashLong((Long) o); } diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index d3425fbd81..21f1eae9a7 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -151,7 +151,11 @@ akka { # 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 + 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 { enabled = off # TODO possibly remove on/off option once we have battle proven it? @@ -162,6 +166,10 @@ akka { # 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 } } } diff --git a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala index a52fb00c50..981b2b5bed 100644 --- a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala @@ -52,7 +52,7 @@ private[akka] object MessageSerializer { def serializeForArtery(serialization: Serialization, message: AnyRef, headerBuilder: HeaderBuilder, envelope: EnvelopeBuffer): Unit = { val serializer = serialization.findSerializerFor(message) - headerBuilder.serializer = serializer.identifier + headerBuilder setSerializer serializer.identifier def manifest: String = serializer match { case ser: SerializerWithStringManifest ⇒ ser.manifest(message) @@ -61,21 +61,21 @@ private[akka] object MessageSerializer { serializer match { case ser: ByteBufferSerializer ⇒ - headerBuilder.manifest = manifest + headerBuilder setManifest manifest envelope.writeHeader(headerBuilder) ser.toBinary(message, envelope.byteBuffer) case _ ⇒ - headerBuilder.manifest = manifest + headerBuilder setManifest manifest envelope.writeHeader(headerBuilder) envelope.byteBuffer.put(serializer.toBinary(message)) } } - def deserializeForArtery(system: ExtendedActorSystem, serialization: Serialization, headerBuilder: HeaderBuilder, + def deserializeForArtery(system: ExtendedActorSystem, originUid: Long, serialization: Serialization, headerBuilder: HeaderBuilder, envelope: EnvelopeBuffer): AnyRef = { serialization.deserializeByteBuffer( envelope.byteBuffer, headerBuilder.serializer, - headerBuilder.manifest) + headerBuilder.manifest(originUid)) // FIXME currently compression will not work for manifests } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 3cb59d8e97..10417fdc2f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -32,10 +32,9 @@ import akka.remote.ThisActorSystemQuarantinedEvent import akka.remote.UniqueAddress import akka.remote.artery.InboundControlJunction.ControlMessageObserver import akka.remote.artery.InboundControlJunction.ControlMessageSubject -import akka.remote.artery.OutboundControlJunction.OutboundControlIngress import akka.remote.transport.AkkaPduCodec import akka.remote.transport.AkkaPduProtobufCodec -import akka.remote.artery.compress.{ AdvertiseCompressionId, InboundCompressionImpl, CompressionProtocol } +import akka.remote.artery.compress.{ InboundCompressionsImpl, CompressionProtocol } import akka.stream.AbruptTerminationException import akka.stream.ActorMaterializer import akka.stream.KillSwitches @@ -203,7 +202,7 @@ private[akka] object AssociationState { incarnation = 1, uniqueRemoteAddressPromise = Promise(), quarantined = ImmutableLongMap.empty[QuarantinedTimestamp], - outboundCompression = NoOutboundCompression) + outboundCompression = NoOutboundCompressions) final case class QuarantinedTimestamp(nanoTime: Long) { override def toString: String = @@ -218,7 +217,7 @@ private[akka] final class AssociationState( val incarnation: Int, val uniqueRemoteAddressPromise: Promise[UniqueAddress], val quarantined: ImmutableLongMap[AssociationState.QuarantinedTimestamp], - val outboundCompression: OutboundCompression) { + val outboundCompression: OutboundCompressions) { import AssociationState.QuarantinedTimestamp @@ -244,7 +243,7 @@ private[akka] final class AssociationState( } } - def newIncarnation(remoteAddressPromise: Promise[UniqueAddress], compression: OutboundCompression): AssociationState = + def newIncarnation(remoteAddressPromise: Promise[UniqueAddress], compression: OutboundCompressions): AssociationState = new AssociationState(incarnation + 1, remoteAddressPromise, quarantined, compression) def newQuarantined(): AssociationState = @@ -254,7 +253,7 @@ private[akka] final class AssociationState( incarnation, uniqueRemoteAddressPromise, quarantined = quarantined.updated(a.uid, QuarantinedTimestamp(System.nanoTime())), - outboundCompression = NoOutboundCompression) // after quarantine no compression needed anymore, drop it + outboundCompression = NoOutboundCompressions) // after quarantine no compression needed anymore, drop it case _ ⇒ this } @@ -534,17 +533,17 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundStreams(): Unit = { - val noCompression = new NoInboundCompression(system) // TODO possibly enable on other channels too https://github.com/akka/akka/pull/20546/files#r68074082 - val compression = createInboundCompressionTable(this) + val noCompressions = new NoInboundCompressions(system) // TODO possibly enable on other channels too https://github.com/akka/akka/pull/20546/files#r68074082 + val compressions = createInboundCompressions(this) - runInboundControlStream(noCompression) - runInboundOrdinaryMessagesStream(compression) + runInboundControlStream(noCompressions) + runInboundOrdinaryMessagesStream(compressions) if (largeMessageDestinationsEnabled) { runInboundLargeMessagesStream() } } - private def runInboundControlStream(compression: InboundCompression): Unit = { + private def runInboundControlStream(compression: InboundCompressions): Unit = { val (ctrl, completed) = if (remoteSettings.TestMode) { val (mgmt, (ctrl, completed)) = @@ -584,15 +583,15 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R inboundEnvelope.message match { case m: CompressionMessage ⇒ m match { - case CompressionProtocol.ActorRefCompressionAdvertisement(from, ref, id) ⇒ - log.debug("Incoming ActorRef compression advertisement from [{}], allocating: [{} => {}]", from, ref, id) - association(from.address).compression.allocateActorRefCompressionId(ref, id) - system.eventStream.publish(CompressionProtocol.Events.ReceivedCompressionAdvertisement(from, ref, id)) + case CompressionProtocol.ActorRefCompressionAdvertisement(from, table) ⇒ + log.debug("Incoming ActorRef compression advertisement from [{}], table: [{}]", from, table) + association(from.address).compression.applyActorRefCompressionTable(table) + system.eventStream.publish(CompressionProtocol.Events.ReceivedCompressionTable(from, table)) - case CompressionProtocol.ClassManifestCompressionAdvertisement(from, manifest, id) ⇒ - log.debug("Incoming Class Manifest compression advertisement from [{}], allocating: [{} => {}]", from, manifest, id) - association(from.address).compression.allocateClassManifestCompressionId(manifest, id) - system.eventStream.publish(CompressionProtocol.Events.ReceivedCompressionAdvertisement(from, manifest, id)) + case CompressionProtocol.ClassManifestCompressionAdvertisement(from, table) ⇒ + log.debug("Incoming Class Manifest compression advertisement from [{}], table: [{}]", from, table) + association(from.address).compression.applyClassManifestCompressionTable(table) + system.eventStream.publish(CompressionProtocol.Events.ReceivedCompressionTable(from, table)) } case _ ⇒ // not interested in non CompressionMessages } @@ -601,7 +600,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R attachStreamRestart("Inbound control stream", completed, () ⇒ runInboundControlStream(compression)) } - private def runInboundOrdinaryMessagesStream(compression: InboundCompression): Unit = { + private def runInboundOrdinaryMessagesStream(compression: InboundCompressions): Unit = { val completed = if (remoteSettings.TestMode) { val (mgmt, c) = aeronSource(ordinaryStreamId, envelopePool) @@ -622,7 +621,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundLargeMessagesStream(): Unit = { - val compression = new NoInboundCompression(system) // no compression on large message stream for now + val compression = new NoInboundCompressions(system) // no compression on large message stream for now val completed = if (remoteSettings.TestMode) { @@ -738,7 +737,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R association(remoteAddress).quarantine(reason = "", uid.map(_.toLong)) } - def outbound(outboundContext: OutboundContext, compression: OutboundCompression): Sink[Send, Future[Done]] = { + def outbound(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[Send, Future[Done]] = { Flow.fromGraph(killSwitch.flow[Send]) .via(new OutboundHandshake(system, outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) .via(encoder(compression)) @@ -746,7 +745,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R envelopePool, giveUpSendAfter, flightRecorder.createEventSink()))(Keep.right) } - def outboundLarge(outboundContext: OutboundContext, compression: OutboundCompression): Sink[Send, Future[Done]] = { + def outboundLarge(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[Send, Future[Done]] = { Flow.fromGraph(killSwitch.flow[Send]) .via(new OutboundHandshake(system, outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) .via(createEncoder(largeEnvelopePool, compression)) @@ -754,7 +753,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R envelopePool, giveUpSendAfter, flightRecorder.createEventSink()))(Keep.right) } - def outboundControl(outboundContext: OutboundContext, compression: OutboundCompression): Sink[Send, (OutboundControlIngress, Future[Done])] = { + def outboundControl(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[Send, (OutboundControlIngress, Future[Done])] = { Flow.fromGraph(killSwitch.flow[Send]) .via(new OutboundHandshake(system, outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) .via(new SystemMessageDelivery(outboundContext, system.deadLetters, systemMessageResendInterval, @@ -767,17 +766,17 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // FIXME we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages } - def createEncoder(compression: OutboundCompression, bufferPool: EnvelopeBufferPool): Flow[Send, EnvelopeBuffer, NotUsed] = + def createEncoder(compression: OutboundCompressions, bufferPool: EnvelopeBufferPool): Flow[Send, EnvelopeBuffer, NotUsed] = Flow.fromGraph(new Encoder(localAddress, system, compression, bufferPool)) - private def createInboundCompressionTable(inboundContext: InboundContext): InboundCompression = - if (remoteSettings.ArteryCompressionSettings.enabled) new InboundCompressionImpl(system, inboundContext) - else new NoInboundCompression(system) + private def createInboundCompressions(inboundContext: InboundContext): InboundCompressions = + if (remoteSettings.ArteryCompressionSettings.enabled) new InboundCompressionsImpl(system, inboundContext) + else new NoInboundCompressions(system) - def createEncoder(pool: EnvelopeBufferPool, compression: OutboundCompression): Flow[Send, EnvelopeBuffer, NotUsed] = + def createEncoder(pool: EnvelopeBufferPool, compression: OutboundCompressions): Flow[Send, EnvelopeBuffer, NotUsed] = Flow.fromGraph(new Encoder(localAddress, system, compression, pool)) - def encoder(compression: OutboundCompression): Flow[Send, EnvelopeBuffer, NotUsed] = createEncoder(envelopePool, compression) + def encoder(compression: OutboundCompressions): Flow[Send, EnvelopeBuffer, NotUsed] = createEncoder(envelopePool, compression) def aeronSource(streamId: Int, pool: EnvelopeBufferPool): Source[EnvelopeBuffer, NotUsed] = Source.fromGraph(new AeronSource(inboundChannel, streamId, aeron, taskRunner, pool, @@ -788,14 +787,14 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R inboundEnvelopePool.release(m) } - def createDecoder(compression: InboundCompression, bufferPool: EnvelopeBufferPool): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { + def createDecoder(compression: InboundCompressions, bufferPool: EnvelopeBufferPool): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { val resolveActorRefWithLocalAddress: String ⇒ InternalActorRef = recipient ⇒ provider.resolveActorRefWithLocalAddress(recipient, localAddress.address) Flow.fromGraph(new Decoder(this, system, resolveActorRefWithLocalAddress, compression, bufferPool, inboundEnvelopePool)) } - def decoder(compression: InboundCompression): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = + def decoder(compression: InboundCompressions): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = createDecoder(compression, envelopePool) def inboundSink: Sink[InboundEnvelope, Future[Done]] = @@ -804,13 +803,13 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .via(new InboundQuarantineCheck(this)) .toMat(messageDispatcherSink)(Keep.right) - def inboundFlow(compression: InboundCompression): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { + def inboundFlow(compression: InboundCompressions): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { Flow[EnvelopeBuffer] .via(killSwitch.flow) .via(decoder(compression)) } - def inboundLargeFlow(compression: InboundCompression): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { + def inboundLargeFlow(compression: InboundCompressions): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { Flow[EnvelopeBuffer] .via(killSwitch.flow) .via(createDecoder(compression, largeEnvelopePool)) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 4a73807ba8..523da7ff8f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -9,8 +9,7 @@ import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.CountDownLatch import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicReference -import akka.remote.artery.compress.{ OutboundCompressionImpl, CompressionProtocol } - +import akka.remote.artery.compress.{ CompressionProtocol, CompressionTable, OutboundCompressionsImpl } import scala.annotation.tailrec import scala.concurrent.Future import scala.concurrent.Promise @@ -40,6 +39,7 @@ import org.agrona.concurrent.ManyToOneConcurrentArrayQueue import akka.util.OptionVal import akka.remote.QuarantinedEvent import akka.remote.DaemonMsgCreate +import akka.remote.artery.compress.CompressionProtocol._ /** * INTERNAL API @@ -82,7 +82,7 @@ private[remote] class Association( // start sending (enqueuing) to the Association immediate after construction. /** Accesses the currently active outbound compression. */ - def compression: OutboundCompression = associationState.outboundCompression + def compression: OutboundCompressions = associationState.outboundCompression def createQueue(capacity: Int): Queue[Send] = new ManyToOneConcurrentArrayQueue[Send](capacity) @@ -283,19 +283,19 @@ private[remote] class Association( private def runOutboundStreams(): Unit = { // TODO no compression for control / large streams currently - val disableCompression = NoOutboundCompression + val disableCompression = NoOutboundCompressions // it's important to materialize the outboundControl stream first, // so that outboundControlIngress is ready when stages for all streams start runOutboundControlStream(disableCompression) - runOutboundOrdinaryMessagesStream(CurrentAssociationStateOutboundCompressionProxy) + runOutboundOrdinaryMessagesStream(CurrentAssociationStateOutboundCompressionsProxy) if (largeMessageChannelEnabled) { runOutboundLargeMessagesStream(disableCompression) } } - private def runOutboundControlStream(compression: OutboundCompression): Unit = { + private def runOutboundControlStream(compression: OutboundCompressions): Unit = { // stage in the control stream may access the outboundControlIngress before returned here // using CountDownLatch to make sure that materialization is completed before accessing outboundControlIngress materializing = new CountDownLatch(1) @@ -340,7 +340,7 @@ private[remote] class Association( QueueWrapper(createQueue(capacity)) } - private def runOutboundOrdinaryMessagesStream(compression: OutboundCompression): Unit = { + private def runOutboundOrdinaryMessagesStream(compression: OutboundCompressions): Unit = { val wrapper = getOrCreateQueueWrapper(queue, queueSize) queue = wrapper // use new underlying queue immediately for restarts @@ -365,7 +365,7 @@ private[remote] class Association( attachStreamRestart("Outbound message stream", completed, _ ⇒ runOutboundOrdinaryMessagesStream(compression)) } - private def runOutboundLargeMessagesStream(compression: OutboundCompression): Unit = { + private def runOutboundLargeMessagesStream(compression: OutboundCompressions): Unit = { val wrapper = getOrCreateQueueWrapper(queue, largeQueueSize) largeQueue = wrapper // use new underlying queue immediately for restarts @@ -411,25 +411,25 @@ private[remote] class Association( } // TODO: Make sure that once other channels use Compression, each gets it's own - private def createOutboundCompressionTable(remoteAddress: Address): OutboundCompression = { + private def createOutboundCompressionTable(remoteAddress: Address): OutboundCompressions = { if (transport.provider.remoteSettings.ArteryCompressionSettings.enabled) { - val compression = new OutboundCompressionImpl(transport.system, remoteAddress) + val compression = new OutboundCompressionsImpl(transport.system, remoteAddress) // FIXME should use verion number of table instead of hashCode log.info("Creating Outbound compression table ({}) to [{}]", compression.hashCode, remoteAddress) compression - } else NoOutboundCompression + } else NoOutboundCompressions } - /* + /** * This proxy uses the current associationStates compression table, which is reset for a new incarnation. * This way the same outgoing stream will switch to using the new table without the need of restarting it. */ - object CurrentAssociationStateOutboundCompressionProxy extends OutboundCompression { - override final def allocateActorRefCompressionId(ref: ActorRef, id: Int): Unit = - associationState.outboundCompression.allocateActorRefCompressionId(ref, id) + private object CurrentAssociationStateOutboundCompressionsProxy extends OutboundCompressions { + override def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit = + associationState.outboundCompression.applyActorRefCompressionTable(table) - override final def allocateClassManifestCompressionId(manifest: String, id: Int): Unit = - associationState.outboundCompression.allocateClassManifestCompressionId(manifest, id) + override def applyClassManifestCompressionTable(table: CompressionTable[String]): Unit = + associationState.outboundCompression.applyClassManifestCompressionTable(table) override final def compressActorRef(ref: ActorRef): Int = associationState.outboundCompression.compressActorRef(ref) diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index ee32b9a187..d2c2f1883e 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -7,8 +7,9 @@ package akka.remote.artery import java.nio.charset.Charset import java.nio.{ ByteBuffer, ByteOrder } -import akka.actor.{ Address, ActorRef } -import akka.remote.artery.compress.{ NoopOutboundCompression, NoopInboundCompression } +import akka.actor.{ ActorRef, Address } +import akka.remote.artery.compress.CompressionProtocol._ +import akka.remote.artery.compress.{ CompressionTable, NoopInboundCompressions, NoopOutboundCompressions } import akka.serialization.Serialization import org.agrona.concurrent.{ ManyToManyConcurrentArrayQueue, UnsafeBuffer } import akka.util.{ OptionVal, Unsafe } @@ -56,8 +57,10 @@ private[remote] object EnvelopeBuffer { val SenderActorRefTagOffset = 16 // Int val RecipientActorRefTagOffset = 20 // Int val ClassManifestTagOffset = 24 // Int + val ActorRefCompressionTableVersionTagOffset = 28 // Int + val ClassManifestCompressionTableVersionTagOffset = 32 // Int - val LiteralsSectionOffset = 28 + val LiteralsSectionOffset = 36 val UsAscii = Charset.forName("US-ASCII") @@ -70,51 +73,70 @@ private[remote] object EnvelopeBuffer { /** * 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 InboundCompression { - def hitActorRef(remote: Address, ref: ActorRef): Unit - def decompressActorRef(idx: Int): OptionVal[ActorRef] +private[remote] trait InboundCompressions { + def hitActorRef(originUid: Long, tableVersion: Int, remote: Address, ref: ActorRef): Unit + def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] - def hitClassManifest(remote: Address, manifest: String): Unit - def decompressClassManifest(idx: Int): OptionVal[String] + def hitClassManifest(originUid: Long, tableVersion: Int, remote: Address, manifest: String): Unit + def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] } /** * INTERNAL API * Compress outgoing data and handle compression advertisements to fill compression table. + * + * One per outgoing message stream. */ -private[remote] trait OutboundCompression { - def allocateActorRefCompressionId(ref: ActorRef, id: Int): Unit +private[remote] trait OutboundCompressions { + def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit def compressActorRef(ref: ActorRef): Int - def allocateClassManifestCompressionId(manifest: String, id: Int): Unit + def applyClassManifestCompressionTable(table: CompressionTable[String]): Unit def compressClassManifest(manifest: String): Int } -object HeaderBuilder { +/** 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: InboundCompression): HeaderBuilder = new HeaderBuilderImpl(compression, NoopOutboundCompression) - def out(compression: OutboundCompression): HeaderBuilder = new HeaderBuilderImpl(NoopInboundCompression, compression) + def in(compression: InboundCompressions): HeaderBuilder = new HeaderBuilderImpl(compression, NoopOutboundCompressions) + def out(compression: OutboundCompressions): HeaderBuilder = new HeaderBuilderImpl(NoopInboundCompressions, compression) /** INTERNAL API, FOR TESTING ONLY */ - private[remote] def bothWays(in: InboundCompression, out: OutboundCompression): HeaderBuilder = new HeaderBuilderImpl(in, out) + private[remote] def bothWays(in: InboundCompressions, out: OutboundCompressions): HeaderBuilder = new HeaderBuilderImpl(in, out) } /** * INTERNAL API */ sealed trait HeaderBuilder { - def version_=(v: Int): Unit + def setVersion(v: Int): Unit def version: Int - def uid_=(u: Long): Unit + def setActorRefCompressionTableVersion(v: Int): Unit + def actorRefCompressionTableVersion: Int + + def setClassManifestCompressionTableVersion(v: Int): Unit + def classManifestCompressionTableVersion: Int + + def setUid(u: Long): Unit def uid: Long - def senderActorRef_=(ref: ActorRef): Unit - def senderActorRef: OptionVal[ActorRef] - def senderActorRefPath: String + 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 @@ -122,25 +144,35 @@ sealed trait HeaderBuilder { def setNoRecipient(): Unit def isNoRecipient: Boolean - def recipientActorRef_=(ref: ActorRef): Unit - def recipientActorRef: OptionVal[ActorRef] - def recipientActorRefPath: String + 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 serializer_=(serializer: Int): Unit + def setSerializer(serializer: Int): Unit def serializer: Int - def manifest_=(manifest: String): Unit - def manifest: String + def setManifest(manifest: String): Unit + def manifest(originUid: Long): String } /** * INTERNAL API */ -private[remote] final class HeaderBuilderImpl(inboundCompression: InboundCompression, outboundCompression: OutboundCompression) extends HeaderBuilder { - var version: Int = _ - var uid: Long = _ - +private[remote] final class HeaderBuilderImpl(inboundCompression: InboundCompressions, outboundCompression: OutboundCompressions) extends HeaderBuilder { // Fields only available for EnvelopeBuffer + var _version: Int = _ + var _uid: Long = _ + var _actorRefCompressionTableVersion: Int = -1 + var _classManifestCompressionTableVersion: Int = -1 + var _senderActorRef: String = null var _senderActorRefIdx: Int = -1 var _recipientActorRef: String = null @@ -150,25 +182,33 @@ private[remote] final class HeaderBuilderImpl(inboundCompression: InboundCompres var _manifest: String = null var _manifestIdx: Int = -1 - def senderActorRef_=(ref: ActorRef): Unit = { + override def setVersion(v: Int) = _version = v + override def version = _version + + override def setUid(uid: Long) = _uid = uid + override def uid: Long = _uid + + override def setActorRefCompressionTableVersion(v: Int): Unit = _actorRefCompressionTableVersion = v + override def actorRefCompressionTableVersion: Int = _actorRefCompressionTableVersion + + override def setClassManifestCompressionTableVersion(v: Int): Unit = _classManifestCompressionTableVersion = v + override def classManifestCompressionTableVersion: Int = _classManifestCompressionTableVersion + + override def setSenderActorRef(ref: ActorRef): Unit = { _senderActorRefIdx = outboundCompression.compressActorRef(ref) if (_senderActorRefIdx == -1) _senderActorRef = Serialization.serializedActorPath(ref) // includes local address from `currentTransportInformation` } - def setNoSender(): Unit = { + override def setNoSender(): Unit = { _senderActorRef = null _senderActorRefIdx = EnvelopeBuffer.DeadLettersCode } - def isNoSender: Boolean = + override def isNoSender: Boolean = (_senderActorRef eq null) && _senderActorRefIdx == EnvelopeBuffer.DeadLettersCode - def senderActorRef: OptionVal[ActorRef] = - if (_senderActorRef eq null) inboundCompression.decompressActorRef(_senderActorRefIdx) + override def senderActorRef(originUid: Long): OptionVal[ActorRef] = + if (_senderActorRef eq null) inboundCompression.decompressActorRef(originUid, actorRefCompressionTableVersion, _senderActorRefIdx) else OptionVal.None - def senderActorRefPath: String = - if (_senderActorRef ne null) _senderActorRef - else { - _senderActorRef = inboundCompression.decompressActorRef(_senderActorRefIdx).get.path.toSerializationFormat - _senderActorRef - } + def senderActorRefPath: OptionVal[String] = + OptionVal(_senderActorRef) def setNoRecipient(): Unit = { _recipientActorRef = null @@ -177,39 +217,48 @@ private[remote] final class HeaderBuilderImpl(inboundCompression: InboundCompres def isNoRecipient: Boolean = (_recipientActorRef eq null) && _recipientActorRefIdx == EnvelopeBuffer.DeadLettersCode - def recipientActorRef_=(ref: ActorRef): Unit = { + def setRecipientActorRef(ref: ActorRef): Unit = { _recipientActorRefIdx = outboundCompression.compressActorRef(ref) if (_recipientActorRefIdx == -1) _recipientActorRef = ref.path.toSerializationFormat } - def recipientActorRef: OptionVal[ActorRef] = - if (_recipientActorRef eq null) inboundCompression.decompressActorRef(_recipientActorRefIdx) + def recipientActorRef(originUid: Long): OptionVal[ActorRef] = + if (_recipientActorRef eq null) inboundCompression.decompressActorRef(originUid, actorRefCompressionTableVersion, _recipientActorRefIdx) else OptionVal.None - def recipientActorRefPath: String = - if (_recipientActorRef ne null) _recipientActorRef - else { - _recipientActorRef = inboundCompression.decompressActorRef(_recipientActorRefIdx).get.path.toSerializationFormat - _recipientActorRef - } + def recipientActorRefPath: OptionVal[String] = + OptionVal(_recipientActorRef) - override def serializer_=(serializer: Int): Unit = { + override def setSerializer(serializer: Int): Unit = { _serializer = serializer } override def serializer: Int = _serializer - override def manifest_=(manifest: String): Unit = { + override def setManifest(manifest: String): Unit = { _manifestIdx = outboundCompression.compressClassManifest(manifest) if (_manifestIdx == -1) _manifest = manifest } - override def manifest: String = { + override def manifest(originUid: Long): String = { if (_manifest ne null) _manifest else { - _manifest = inboundCompression.decompressClassManifest(_manifestIdx).get + _manifest = inboundCompression.decompressClassManifest(originUid, classManifestCompressionTableVersion, _manifestIdx).get _manifest } } - override def toString = s"HeaderBuilderImpl($version, $uid, ${_senderActorRef}, ${_senderActorRefIdx}, ${_recipientActorRef}, ${_recipientActorRefIdx}, ${_serializer}, ${_manifest}, ${_manifestIdx})" + override def toString = + "HeaderBuilderImpl(" + + version + ", " + + actorRefCompressionTableVersion + ", " + + classManifestCompressionTableVersion + ", " + + uid + ", " + + _senderActorRef + ", " + + _senderActorRefIdx + ", " + + _recipientActorRef + ", " + + _recipientActorRefIdx + ", " + + _serializer + ", " + + _manifest + ", " + + _manifestIdx + ")" + } /** @@ -231,6 +280,10 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { byteBuffer.putLong(header.uid) byteBuffer.putInt(header.serializer) + // compression table version numbers + byteBuffer.putInt(ActorRefCompressionTableVersionTagOffset, header._actorRefCompressionTableVersion | TagTypeMask) + byteBuffer.putInt(ClassManifestCompressionTableVersionTagOffset, header._classManifestCompressionTableVersion | TagTypeMask) + // Write compressable, variable-length parts always to the actual position of the buffer // Write tag values explicitly in their proper offset byteBuffer.position(LiteralsSectionOffset) @@ -258,9 +311,19 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { val header = h.asInstanceOf[HeaderBuilderImpl] // Read fixed length parts - header.version = byteBuffer.getInt - header.uid = byteBuffer.getLong - header.serializer = byteBuffer.getInt + header setVersion byteBuffer.getInt + header setUid byteBuffer.getLong + header setSerializer byteBuffer.getInt + + // compression table versions (stored in the Tag) + val refCompressionVersionTag = byteBuffer.getInt(ActorRefCompressionTableVersionTagOffset) + if ((refCompressionVersionTag & TagTypeMask) != 0) { + header setActorRefCompressionTableVersion refCompressionVersionTag & TagValueMask + } + val manifestCompressionVersionTag = byteBuffer.getInt(ClassManifestCompressionTableVersionTagOffset) + if ((manifestCompressionVersionTag & TagTypeMask) != 0) { + header setClassManifestCompressionTableVersion manifestCompressionVersionTag & TagValueMask + } // Read compressable, variable-length parts always from the actual position of the buffer // Read tag values explicitly from their proper offset diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 61ae1368df..49d4c6bdd8 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -23,7 +23,7 @@ import akka.stream.stage.TimerGraphStageLogic private[remote] class Encoder( uniqueLocalAddress: UniqueAddress, system: ActorSystem, - compression: OutboundCompression, + compression: OutboundCompressions, bufferPool: EnvelopeBufferPool) extends GraphStage[FlowShape[Send, EnvelopeBuffer]] { @@ -35,8 +35,8 @@ private[remote] class Encoder( new GraphStageLogic(shape) with InHandler with OutHandler with StageLogging { private val headerBuilder = HeaderBuilder.out(compression) - headerBuilder.version = ArteryTransport.Version - headerBuilder.uid = uniqueLocalAddress.uid + 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) @@ -48,7 +48,7 @@ private[remote] class Encoder( val envelope = bufferPool.acquire() // internally compression is applied by the builder: - headerBuilder.recipientActorRef = send.recipient + headerBuilder setRecipientActorRef send.recipient try { // avoiding currentTransportInformation.withValue due to thunk allocation @@ -58,7 +58,7 @@ private[remote] class Encoder( send.senderOption match { case OptionVal.None ⇒ headerBuilder.setNoSender() - case OptionVal.Some(s) ⇒ headerBuilder.senderActorRef = s + case OptionVal.Some(s) ⇒ headerBuilder setSenderActorRef s } MessageSerializer.serializeForArtery(serialization, send.message.asInstanceOf[AnyRef], headerBuilder, envelope) @@ -109,7 +109,7 @@ private[remote] class Decoder( inboundContext: InboundContext, system: ExtendedActorSystem, resolveActorRefWithLocalAddress: String ⇒ InternalActorRef, - compression: InboundCompression, + compression: InboundCompressions, // TODO has to do demuxing on remote address It would seem, as decoder does not yet know bufferPool: EnvelopeBufferPool, inEnvelopePool: ObjectPool[InboundEnvelope]) extends GraphStage[FlowShape[EnvelopeBuffer, InboundEnvelope]] { val in: Inlet[EnvelopeBuffer] = Inlet("Artery.Decoder.in") @@ -135,23 +135,29 @@ private[remote] class Decoder( val originUid = headerBuilder.uid val association = inboundContext.association(originUid) - val recipient: OptionVal[InternalActorRef] = headerBuilder.recipientActorRef match { - case OptionVal.Some(ref) ⇒ OptionVal(ref.asInstanceOf[InternalActorRef]) - case OptionVal.None ⇒ resolveRecipient(headerBuilder.recipientActorRefPath) + val recipient: OptionVal[InternalActorRef] = headerBuilder.recipientActorRef(originUid) match { + case OptionVal.Some(ref) ⇒ + OptionVal(ref.asInstanceOf[InternalActorRef]) + case OptionVal.None ⇒ + // `get` on Path is safe because it surely is not a compressed value here + resolveRecipient(headerBuilder.recipientActorRefPath.get) } - val sender: InternalActorRef = headerBuilder.senderActorRef match { - case OptionVal.Some(ref) ⇒ ref.asInstanceOf[InternalActorRef] - case OptionVal.None ⇒ resolveActorRefWithLocalAddress(headerBuilder.senderActorRefPath) + val sender: InternalActorRef = headerBuilder.senderActorRef(originUid) match { + case OptionVal.Some(ref) ⇒ + ref.asInstanceOf[InternalActorRef] + case OptionVal.None ⇒ + // `get` on Path is safe because it surely is not a compressed value here + resolveActorRefWithLocalAddress(headerBuilder.senderActorRefPath.get) } // --- hit refs and manifests for heavy-hitter counting association match { case OptionVal.Some(assoc) ⇒ val remoteAddress = assoc.remoteAddress - compression.hitActorRef(remoteAddress, sender) - if (recipient.isDefined) compression.hitActorRef(remoteAddress, recipient.get) - compression.hitClassManifest(remoteAddress, headerBuilder.manifest) + compression.hitActorRef(originUid, headerBuilder.actorRefCompressionTableVersion, remoteAddress, sender) + if (recipient.isDefined) compression.hitActorRef(originUid, headerBuilder.actorRefCompressionTableVersion, remoteAddress, recipient.get) + compression.hitClassManifest(originUid, headerBuilder.classManifestCompressionTableVersion, remoteAddress, headerBuilder.manifest(originUid)) 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?") @@ -160,7 +166,7 @@ private[remote] class Decoder( try { val deserializedMessage = MessageSerializer.deserializeForArtery( - system, serialization, headerBuilder, envelope) + system, originUid, serialization, headerBuilder, envelope) val decoded = inEnvelopePool.acquire() decoded.asInstanceOf[ReusableInboundEnvelope].init( @@ -176,14 +182,14 @@ private[remote] class Decoder( // recipient for the first message that is sent to it, best effort retry scheduleOnce(RetryResolveRemoteDeployedRecipient( retryResolveRemoteDeployedRecipientAttempts, - headerBuilder.recipientActorRefPath, decoded), retryResolveRemoteDeployedRecipientInterval) + headerBuilder.recipientActorRefPath.get, decoded), retryResolveRemoteDeployedRecipientInterval) // FIXME IS THIS SAFE? } else push(out, decoded) } catch { case NonFatal(e) ⇒ log.warning( "Failed to deserialize message with serializer id [{}] and manifest [{}]. {}", - headerBuilder.serializer, headerBuilder.manifest, e.getMessage) + headerBuilder.serializer, headerBuilder.manifest(originUid), e.getMessage) pull(in) } finally { bufferPool.release(envelope) diff --git a/akka-remote/src/main/scala/akka/remote/artery/NoLiteralCompression.scala b/akka-remote/src/main/scala/akka/remote/artery/NoLiteralCompression.scala index be70489b85..fe7502ea5e 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/NoLiteralCompression.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/NoLiteralCompression.scala @@ -3,7 +3,8 @@ */ package akka.remote.artery -import akka.actor.{ Address, InternalActorRef, ActorSystem, ActorRef } +import akka.actor.{ ActorRef, ActorSystem, Address, InternalActorRef } +import akka.remote.artery.compress.CompressionTable import akka.util.OptionVal /** @@ -11,15 +12,15 @@ import akka.util.OptionVal * * Literarily, no compression! */ -final class NoInboundCompression(system: ActorSystem) extends InboundCompression { - override def hitActorRef(address: Address, ref: ActorRef): Unit = () - override def decompressActorRef(idx: Int): OptionVal[ActorRef] = +final class NoInboundCompressions(system: ActorSystem) extends InboundCompressions { + override def hitActorRef(originUid: Long, tableVersion: Int, remote: Address, ref: ActorRef): Unit = () + override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = if (idx == -1) throw new IllegalArgumentException("Attemted decompression of illegal compression id: -1") else if (idx == 0) OptionVal.Some(system.deadLetters) // special case deadLetters else OptionVal.None - override def hitClassManifest(address: Address, manifest: String): Unit = () - override def decompressClassManifest(idx: Int): OptionVal[String] = + override def hitClassManifest(originUid: Long, tableVersion: Int, remote: Address, manifest: String): Unit = () + override def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] = if (idx == -1) throw new IllegalArgumentException("Attemted decompression of illegal compression id: -1") else OptionVal.None } @@ -29,10 +30,10 @@ final class NoInboundCompression(system: ActorSystem) extends InboundCompression * * Literarily, no compression! */ -object NoOutboundCompression extends OutboundCompression { - override def allocateActorRefCompressionId(ref: ActorRef, id: Int): Unit = () +object NoOutboundCompressions extends OutboundCompressions { + override def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit = () override def compressActorRef(ref: ActorRef): Int = -1 - override def allocateClassManifestCompressionId(manifest: String, id: Int): Unit = () + override def applyClassManifestCompressionTable(table: CompressionTable[String]): Unit = () override def compressClassManifest(manifest: String): Int = -1 } diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressionTables.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressionTables.scala deleted file mode 100644 index 9aebf26f62..0000000000 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressionTables.scala +++ /dev/null @@ -1,102 +0,0 @@ -/** - * Copyright (C) 2016 Lightbend Inc. - */ - -package akka.remote.artery.compress - -import akka.actor.{ Address, ActorRef, ActorSystem } -import akka.remote.artery._ -import akka.remote.artery.compress.CompressionProtocol.Events -import akka.serialization.Serialization -import akka.stream.impl.ConstantFun -import akka.util.OptionVal - -/** INTERNAL API */ -private[remote] final class OutboundCompressionImpl(system: ActorSystem, remoteAddress: Address) extends OutboundCompression { - - private val settings = CompressionSettings(system) - - private val actorRefsOut = new OutboundActorRefCompressionTable(system, remoteAddress) - - private val classManifestsOut = new OutboundCompressionTable[String](system, remoteAddress) - - // actor ref compression --- - - override def allocateActorRefCompressionId(ref: ActorRef, id: Int): Unit = actorRefsOut.register(ref, id) - override def compressActorRef(ref: ActorRef): Int = actorRefsOut.compress(ref) - - // class manifest compression --- - - override def compressClassManifest(manifest: String): Int = classManifestsOut.compress(manifest) - override def allocateClassManifestCompressionId(manifest: String, id: Int): Unit = classManifestsOut.register(manifest, id) -} - -/** INTERNAL API */ -private[remote] final class InboundCompressionImpl( - system: ActorSystem, - inboundContext: InboundContext -) extends InboundCompression { - - private val settings = CompressionSettings(system) - private val log = system.log - - private val localAddress = inboundContext.localAddress - - // TODO maybe use inbound context to get remoteAddress instead? - val advertiseActorRef = new AdvertiseCompressionId[ActorRef] { - override def apply(remoteAddress: Address, ref: ActorRef, id: Int): Unit = { - - log.debug(s"Advertise ActorRef compression [$ref => $id] to [$remoteAddress]") - // TODO could use remote address via association lookup??? could be more lookups though - inboundContext.sendControl(remoteAddress, CompressionProtocol.ActorRefCompressionAdvertisement(inboundContext.localAddress, ref, id)) - } - } - val advertiseManifest = new AdvertiseCompressionId[String] { - override def apply(remoteAddress: Address, man: String, id: Int): Unit = { - log.error(s"Advertise ClassManifest compression [$man => $id] to [$remoteAddress]") - inboundContext.sendControl(remoteAddress, CompressionProtocol.ClassManifestCompressionAdvertisement(localAddress, man, id)) - } - } - - private val actorRefHitters = new TopHeavyHitters[ActorRef](settings.actorRefs.max) - private val actorRefsIn = new InboundActorRefCompressionTable(system, actorRefHitters, advertiseActorRef) - - private val manifestHitters = new TopHeavyHitters[String](settings.manifests.max) - private val classManifestsIn = new InboundCompressionTable[String](system, manifestHitters, ConstantFun.scalaIdentityFunction, advertiseManifest) - - // actor ref compression --- - - override def decompressActorRef(idx: Int): OptionVal[ActorRef] = { - val value = actorRefsIn.decompress(idx) - OptionVal.Some(value) - } - override def hitActorRef(address: Address, ref: ActorRef): Unit = { - actorRefsIn.increment(address, ref, 1L) - } - - // class manifest compression --- - - override def decompressClassManifest(idx: Int): OptionVal[String] = { - val value = classManifestsIn.decompress(idx) - OptionVal.Some(value) - } - override def hitClassManifest(address: Address, manifest: String): Unit = { - classManifestsIn.increment(address, manifest, 1L) - } -} - -object NoopInboundCompression extends InboundCompression { - override def hitActorRef(remote: Address, ref: ActorRef): Unit = () - override def decompressActorRef(idx: Int): OptionVal[ActorRef] = OptionVal.None - - override def hitClassManifest(remote: Address, manifest: String): Unit = () - override def decompressClassManifest(idx: Int): OptionVal[String] = OptionVal.None -} - -object NoopOutboundCompression extends OutboundCompression { - override def compressActorRef(ref: ActorRef): Int = -1 - override def allocateActorRefCompressionId(ref: ActorRef, id: Int): Unit = () - - override def compressClassManifest(manifest: String): Int = -1 - override def allocateClassManifestCompressionId(manifest: String, id: Int): Unit = () -} diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressions.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressions.scala new file mode 100644 index 0000000000..1580709a1b --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressions.scala @@ -0,0 +1,99 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import java.util.function.LongFunction + +import akka.actor.{ ActorRef, ActorSystem, Address } +import akka.remote.artery._ +import akka.util.OptionVal +import akka.remote.artery.OutboundCompressions +import org.agrona.collections.Long2ObjectHashMap + +/** INTERNAL API */ +private[remote] final class OutboundCompressionsImpl(system: ActorSystem, remoteAddress: Address) extends OutboundCompressions { + + private val actorRefsOut = new OutboundActorRefCompression(system, remoteAddress) + private val classManifestsOut = new OutboundCompressionTable[String](system, remoteAddress) + + // actor ref compression --- + + override def compressActorRef(ref: ActorRef): Int = actorRefsOut.compress(ref) + override def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit = + actorRefsOut.flipTable(table) + + // class manifest compression --- + + override def compressClassManifest(manifest: String): Int = classManifestsOut.compress(manifest) + override def applyClassManifestCompressionTable(table: CompressionTable[String]): Unit = + classManifestsOut.flipTable(table) +} + +/** + * 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 +) extends InboundCompressions { + + private val settings = CompressionSettings(system) + private val localAddress = inboundContext.localAddress + + // FIXME we also must remove the ones that won't be used anymore - when quarantine triggers + private[this] val _actorRefsIn = new Long2ObjectHashMap[InboundActorRefCompression]() + private val createInboundActorRefsForOrigin = new LongFunction[InboundActorRefCompression] { + override def apply(originUid: Long): InboundActorRefCompression = { + val actorRefHitters = new TopHeavyHitters[ActorRef](settings.actorRefs.max) + new InboundActorRefCompression(system, settings, originUid, inboundContext, actorRefHitters) + } + } + private def actorRefsIn(originUid: Long): InboundActorRefCompression = + _actorRefsIn.computeIfAbsent(originUid, createInboundActorRefsForOrigin) + + private[this] val _classManifestsIn = new Long2ObjectHashMap[InboundManifestCompression]() + private val createInboundManifestsForOrigin = new LongFunction[InboundManifestCompression] { + override def apply(originUid: Long): InboundManifestCompression = { + val manifestHitters = new TopHeavyHitters[String](settings.manifests.max) + new InboundManifestCompression(system, settings, originUid, inboundContext, manifestHitters) + } + } + private def classManifestsIn(originUid: Long): InboundManifestCompression = + _classManifestsIn.computeIfAbsent(originUid, createInboundManifestsForOrigin) + + // actor ref compression --- + + override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = + actorRefsIn(originUid).decompress(tableVersion, idx) + override def hitActorRef(originUid: Long, tableVersion: Int, address: Address, ref: ActorRef): Unit = { + actorRefsIn(originUid).increment(address, ref, 1L) + } + + // class manifest compression --- + + override def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] = + classManifestsIn(originUid).decompress(tableVersion, idx) + override def hitClassManifest(originUid: Long, tableVersion: Int, address: Address, manifest: String): Unit = { + classManifestsIn(originUid).increment(address, manifest, 1L) + } +} + +object NoopInboundCompressions extends InboundCompressions { + override def hitActorRef(originUid: Long, tableVersion: Int, remote: Address, ref: ActorRef): Unit = () + override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = OptionVal.None + + override def hitClassManifest(originUid: Long, tableVersion: Int, remote: Address, manifest: String): Unit = () + override def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] = OptionVal.None +} + +object NoopOutboundCompressions extends OutboundCompressions { + override def compressActorRef(ref: ActorRef): Int = -1 + override def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit = () + + override def compressClassManifest(manifest: String): Int = -1 + override def applyClassManifestCompressionTable(table: CompressionTable[String]): Unit = () +} 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 index 4dfe2763ce..5cee5f77f3 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionProtocol.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionProtocol.scala @@ -4,6 +4,7 @@ package akka.remote.artery.compress +import scala.language.existentials import akka.actor.{ ActorRef, Address } import akka.remote.UniqueAddress import akka.remote.artery.ControlMessage @@ -19,14 +20,14 @@ object CompressionProtocol { * 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, ref: ActorRef, id: Int) + private[remote] final case class ActorRefCompressionAdvertisement(from: UniqueAddress, table: CompressionTable[ActorRef]) 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, manifest: String, id: Int) + private[remote] final case class ClassManifestCompressionAdvertisement(from: UniqueAddress, table: CompressionTable[String]) extends ControlMessage with CompressionMessage /** INTERNAL API */ @@ -38,7 +39,7 @@ object CompressionProtocol { final case class HeavyHitterDetected(key: Any, id: Int, count: Long) extends Event /** INTERNAL API */ - final case class ReceivedCompressionAdvertisement(from: UniqueAddress, key: Any, id: Int) extends Event + final case class ReceivedCompressionTable[T](from: UniqueAddress, table: CompressionTable[T]) extends Event } diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionSettings.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionSettings.scala index 2e27d244d5..1b7e1ca6ef 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionSettings.scala @@ -3,9 +3,13 @@ */ package akka.remote.artery.compress +import java.util.concurrent.TimeUnit + import akka.actor.ActorSystem import com.typesafe.config.Config +import scala.concurrent.duration._ + /** INTERNAL API */ private[akka] class CompressionSettings(_config: Config) { val enabled = _config.getBoolean("enabled") @@ -17,12 +21,14 @@ private[akka] class CompressionSettings(_config: Config) { private val c = _config.getConfig("actor-refs") val enabled = globalEnabled && c.getBoolean("enabled") + val advertisementInterval = c.getDuration("advertisement-interval", TimeUnit.MILLISECONDS).millis val max = c.getInt("max") } object manifests { private val c = _config.getConfig("manifests") val enabled = globalEnabled && c.getBoolean("enabled") + val advertisementInterval = c.getDuration("advertisement-interval", TimeUnit.MILLISECONDS).millis val max = c.getInt("max") } } 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..6125a2b3ea --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala @@ -0,0 +1,30 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +/** INTERNAL API: Versioned compression table to be advertised between systems */ +private[akka] final case class CompressionTable[T](version: Long, map: Map[T, Int]) { + + def invert: DecompressionTable[T] = + if (map.isEmpty) DecompressionTable.empty[T].copy(version = version) + else { + // TODO: these are some expensive sanity checks, about the numbers being consequitive, without gaps + // TODO: we can remove them, make them re-map (not needed I believe though) + val expectedGaplessSum = Integer.valueOf((map.size * (map.size + 1)) / 2) /* Dirichlet */ + require(map.values.min == 0, "Compression table should start allocating from 0, yet lowest allocated id was " + map.values.min) + require(map.values.sum + map.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: " + map) + + val vals = map.toList.sortBy(_._2).iterator.map(_._1) + val dtab = Array.ofDim[Object](map.size).asInstanceOf[Array[T]] + vals.copyToArray(dtab) // TODO HEAVY, AVOID COPYING AND THE MAP ETC!!! + DecompressionTable[T](version, dtab) + } +} +/** INTERNAL API */ +private[remote] object CompressionTable { + private[this] val _empty = new CompressionTable[Any](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..2434112982 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala @@ -0,0 +1,28 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +/** INTERNAL API */ +private[remote] final case class DecompressionTable[T](version: Long, table: Array[T]) { + def get(idx: Int): T = table(idx) + + def invert: CompressionTable[T] = + CompressionTable(version, Map(table.zipWithIndex: _*)) + + /** Writes complete table as String (heavy operation) */ + def toDebugString = + getClass.getName + + s"(version: $version, " + + ( + if (table.length == 0) "[empty]" + else s"table: [${table.zipWithIndex.map({ case (t, i) ⇒ s"$i -> $t" }).mkString(",")}" + ) + "])" +} + +/** INTERNAL API */ +private[remote] object DecompressionTable { + private[this] val _empty = DecompressionTable(0, Array.empty) + def empty[T] = _empty.asInstanceOf[DecompressionTable[T]] +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressionTable.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressionTable.scala deleted file mode 100644 index bae6264695..0000000000 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressionTable.scala +++ /dev/null @@ -1,161 +0,0 @@ -/* - * Copyright (C) 2016 Lightbend Inc. - */ - -package akka.remote.artery.compress - -import akka.actor.{ Address, ActorRef, ActorSystem } -import akka.event.Logging - -final class InboundActorRefCompressionTable( - system: ActorSystem, - heavyHitters: TopHeavyHitters[ActorRef], - onNewHeavyHitterDetected: AdvertiseCompressionId[ActorRef] -) extends InboundCompressionTable[ActorRef](system, heavyHitters, _.path.toSerializationFormat, onNewHeavyHitterDetected) { - - preAllocate( - system.deadLetters - ) - - /* Since the table is empty here, anything we increment here becomes a heavy hitter immediately. */ - def preAllocate(allocations: ActorRef*): Unit = { - allocations foreach { case ref ⇒ increment(null, ref, 100000) } - } - - override def shouldAdvertiseCompressionId(idx: Int): Boolean = - idx > 0 // 0 is special => deadLetters - - override def decompress(idx: Int): ActorRef = - if (idx == 0) system.deadLetters - else super.decompress(idx) -} - -/** - * Handles counting and detecting of heavy-hitters and compressing them via a table lookup. - * Mutable and not thread-safe. - * - * Compression flow goes like: - * [1] on each message we add the actor path here - * [2] if it becomes a heavy hitter, we allocate an identifier for it and invoke the callback - * [3]> the callback for example then triggers an CompressionAdvertisement to the receiving side - */ -// TODO should the onHeavyHitter be inside HeavyHitters? -class InboundCompressionTable[T]( - system: ActorSystem, - heavyHitters: TopHeavyHitters[T], - convertKeyToString: T ⇒ String, - onNewHeavyHitterDetected: AdvertiseCompressionId[T]) { - require(heavyHitters != null, "heavyHitters must not be null") - - private val settings = CompressionSettings(system) - val log = Logging(system, "InboundCompressionTable") - - // TODO calibrate properly (h/w have direct relation to preciseness and max capacity) - private[this] val cms = new CountMinSketch(100, 100, System.currentTimeMillis().toInt) - - @volatile private[this] var compressionAllocations = Map.empty[Int, T] // TODO replace with a specialized LongMap - private[this] var currentCompressionId = InboundCompressionTable.CompressionAllocationCounterStart - - /** - * Decompress given identifier into original String representation. - * - * @throws UnknownCompressedIdException if given id is not known, this may indicate a bug – such situation should not happen. - */ - def decompress(idx: Int): T = { - if (settings.debug) log.debug(s"Decompress [{}] => {}", idx, compressionAllocations.get(idx)) - compressionAllocations.get(idx) match { - case Some(value) ⇒ value - case None ⇒ throw new UnknownCompressedIdException(idx) - } - } - - /** - * Add `n` occurance for the given key and call `heavyHittedDetected` if element has become a heavy hitter. - * Empty keys are omitted. - */ - // TODO not so happy about passing around address here, but in incoming there's no other earlier place to get it? - def increment(remoteAddress: Address, value: T, n: Long): Unit = { - val key = convertKeyToString(value) - if (shouldIgnore(key)) { - // ignore... - } else { - // val countBefore = cms.estimateCount(key) - val count = cms.addAndEstimateCount(key, n) - // log.warning(s"HIT: increment $key + $n => ($countBefore->) $count; (addAndCheckIfheavyHitterDetected(value, count) = ${addAndCheckIfheavyHitterDetected(value, count)}); (!wasCompressedPreviously(key) = ${!wasCompressedPreviously(key)})") - - // TODO optimise order of these, what is more expensive? (now the `previous` is, but if aprox datatype there it would be faster)... Needs pondering. - if (addAndCheckIfheavyHitterDetected(value, count) && !wasCompressedPreviously(key)) { - val idx = allocateCompressedId(value) - log.debug("Allocated compression id [" + idx + "] for [" + value + "], in association with [" + remoteAddress + "]") - if (shouldAdvertiseCompressionId(idx)) { // TODO change to "time based accumulate new table => advertise it" - // TODO guard with if - log.debug(s"Inbound: Heavy hitter detected: [{} => $idx], {} hits recorded for it (confidence: {}, relative error (eps) {}).\n" + - s"All allocations: ${compressionAllocations}", key, count, cms.getConfidence, cms.getRelativeError) - onNewHeavyHitterDetected(remoteAddress, value, idx) // would be used to signal via side-channel to OutboundCompression that we want to send a ActorRefCompressionAdvertisement - } - } - } - } - - /** Some compression IDs are special and known upfront by both sides, thus need not be advertised (e.g. deadLetters => 0) */ - def shouldAdvertiseCompressionId(idx: Int): Boolean = - true // TODO this will be different in the "advertise entire table mode", it will be "once table is big enough or much time passed" - - private def shouldIgnore(key: String) = { // TODO this is hacky, if we'd do this we trigger compression too early (before association exists, so control messages fail) - key match { - case null ⇒ true - case "" ⇒ true // empty class manifest for example - case _ ⇒ key.endsWith("/system/dummy") || key.endsWith("/") // TODO dummy likely shouldn't exist? can we remove it? - } - } - - // TODO this must be optimised, we really don't want to scan the entire key-set each time to make sure - private def wasCompressedPreviously(key: String): Boolean = - compressionAllocations.values.exists(_ == key) // TODO expensive, aprox or something sneakier? - - /** Mutates heavy hitters */ - private def addAndCheckIfheavyHitterDetected(value: T, count: Long): Boolean = { - heavyHitters.update(value, count) - } - - private def allocateCompressedId(value: T): Int = { - val idx = nextCompressionId() - compressionAllocations.get(idx) match { - case Some(previouslyCompressedValue) ⇒ - // should never really happen, but let's not assume that - throw new ExistingcompressedIdReuseAttemptException(idx, previouslyCompressedValue) - - case None ⇒ - // good, the idx is not used so we can allocate it - compressionAllocations = compressionAllocations.updated(idx, value) - idx - } - } - - private def nextCompressionId(): Int = { - val id = currentCompressionId - currentCompressionId += 1 - id - } - - override def toString = - s"""${getClass.getSimpleName}(countMinSketch: $cms, heavyHitters: $heavyHitters)""" - -} - -object InboundCompressionTable { - val CompressionAllocationCounterStart = 0 - // val CompressionAllocationCounterStart = 64L // we leave 64 slots (0 counts too) for pre-allocated Akka compressions -} - -final class ExistingcompressedIdReuseAttemptException(id: Long, value: Any) - extends RuntimeException( - s"Attempted to re-allocate compressedId [$id] which is still in use for compressing [$value]! " + - s"This should never happen and is likely an implementation bug.") - -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.") 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..5464d71f72 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala @@ -0,0 +1,276 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import java.util.concurrent.atomic.AtomicReference + +import akka.actor.{ ActorRef, ActorSystem, Address } +import akka.event.Logging +import akka.remote.artery.{ InboundContext, OutboundContext } +import akka.stream.impl.ConstantFun +import akka.util.{ OptionVal, PrettyDuration } + +import scala.concurrent.duration.{ Duration, FiniteDuration } + +/** + * 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: CompressionSettings, + originUid: Long, + inboundContext: InboundContext, + heavyHitters: TopHeavyHitters[ActorRef] +) extends InboundCompression[ActorRef](system, settings, originUid, inboundContext, heavyHitters, _.path.toSerializationFormat) { + + preAllocate(system.deadLetters) + + /* Since the table is empty here, anything we increment here becomes a heavy hitter immediately. */ + def preAllocate(allocations: ActorRef*): Unit = { + allocations foreach { case ref ⇒ increment(null, ref, 100000) } + } + + override def decompress(tableId: Long, idx: Int): OptionVal[ActorRef] = + if (idx == 0) OptionVal.Some(system.deadLetters) + else super.decompress(tableId, idx) + + scheduleNextTableAdvertisement() + override protected def tableAdvertisementInterval = settings.actorRefs.advertisementInterval + + def advertiseCompressionTable(association: OutboundContext, table: CompressionTable[ActorRef]): Unit = { + log.debug(s"Advertise ActorRef compression [$table] to [${association.remoteAddress}]") + association.sendControl(CompressionProtocol.ActorRefCompressionAdvertisement(inboundContext.localAddress, table)) + } +} + +final class InboundManifestCompression( + system: ActorSystem, + settings: CompressionSettings, + originUid: Long, + inboundContext: InboundContext, + heavyHitters: TopHeavyHitters[String] +) extends InboundCompression[String](system, settings, originUid, inboundContext, heavyHitters, ConstantFun.scalaIdentityFunction) { + + scheduleNextTableAdvertisement() + override protected def tableAdvertisementInterval = settings.manifests.advertisementInterval + + override def advertiseCompressionTable(association: OutboundContext, table: CompressionTable[String]): Unit = { + log.debug(s"Advertise ClassManifest compression [$table] to [${association.remoteAddress}]") + association.sendControl(CompressionProtocol.ClassManifestCompressionAdvertisement(inboundContext.localAddress, table)) + } +} + +/** + * 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: CompressionSettings, + originUid: Long, + inboundContext: InboundContext, + val heavyHitters: TopHeavyHitters[T], + convertKeyToString: T ⇒ String) { // TODO avoid converting to string, in order to use the ActorRef.hashCode! + + val log = Logging(system, "InboundCompressionTable") + + // TODO atomic / state machine? the InbouncCompression could even extend ActomicReference[State]! + + // TODO NOTE: there exist edge cases around, we advertise table 1, accumulate table 2, the remote system has not used 2 yet, + // yet we technically could already prepare table 3, then it starts using table 1 suddenly. Edge cases like that. + // SOLUTION 1: We don't start building new tables until we've seen the previous one be used (move from new to active) + // This is nice as it practically disables all the "build the table" work when the other side is not interested in using it. + // SOLUTION 2: We end up dropping messages when old table comes in (we do that anyway) + + // TODO have a marker that "advertised table XXX", so we don't generate a new-new one until the new one is in use? + + // 2 tables are used, one is "still in use", and the + @volatile private[this] var activeTable = DecompressionTable.empty[T] + @volatile private[this] var nextTable = DecompressionTable.empty[T] + + // TODO calibrate properly (h/w have direct relation to preciseness and max capacity) + private[this] val cms = new CountMinSketch(100, 100, System.currentTimeMillis().toInt) + + /* ==== COMPRESSION ==== */ + + /** + * 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. + */ + // not tailrec because we allow special casing in sub-class, however recursion is always at most 1 level deep + def decompress(tableVersion: Long, idx: Int): OptionVal[T] = { + val activeVersion = activeTable.version // TODO move into state + + if (tableVersion == -1) OptionVal.None // no compression, bail out early + else if (tableVersion == activeVersion) { + val value: T = activeTable.get(idx) + if (settings.debug) log.debug(s"Decompress [{}] => {}", idx, value) + if (value != null) OptionVal.Some[T](value) + else throw new UnknownCompressedIdException(idx) + } else if (tableVersion < activeVersion) { + log.warning("Received value compressed with old table: [{}], current table version is: [{}]", tableVersion, activeVersion) + OptionVal.None + } else if (tableVersion == nextTable.version) { + flipTables() + decompress(tableVersion, idx) // recurse, activeTable will not be able to handle this + } else { + // which means that incoming version was > nextTable.version, which likely is a bug + log.error("Inbound message is using compression table version higher than the highest allocated table on this node. " + + "This should not happen! State: activeTable: {}, nextTable, incoming tableVersion: {}", activeVersion, nextTable, tableVersion) + OptionVal.None + } + + } + + /** + * Add `n` occurance for the given key and call `heavyHittedDetected` if element has become a heavy hitter. + * Empty keys are omitted. + */ + // TODO not so happy about passing around address here, but in incoming there's no other earlier place to get it? + def increment(remoteAddress: Address, value: T, n: Long): Unit = { + val key = convertKeyToString(value) + if (shouldIgnore(key)) { + // ignore... + } else { + val count = cms.addAndEstimateCount(key, n) + + // TODO optimise order of these, what is more expensive? + // TODO (now the `previous` is, but if aprox datatype there it would be faster)... Needs pondering. + val wasHeavyHitter = addAndCheckIfheavyHitterDetected(value, count) + if (wasHeavyHitter) + log.debug(s"Heavy hitter detected: {} [count: {}]", value, count) + // if (wasHeavyHitter && !wasCompressedPreviously(key)) { + // val idx = prepareCompressionAdvertisement() + // log.debug("Allocated compression id [" + idx + "] for [" + value + "], in association with [" + remoteAddress + "]") + // } + } + } + + private def shouldIgnore(key: String) = { // TODO this is hacky, if we'd do this we trigger compression too early (before association exists, so control messages fail) + key match { + case null ⇒ true + case "" ⇒ true // empty class manifest for example + case _ ⇒ key.endsWith("/system/dummy") || key.endsWith("/") // TODO dummy likely shouldn't exist? can we remove it? + } + } + + // TODO this must be optimised, we really don't want to scan the entire key-set each time to make sure + private def wasCompressedPreviously(key: String): Boolean = { + var i = 0 + val len = activeTable.table.length + while (i < len) { + if (activeTable.table(i) == key) return true + i += 1 + } + false + } + + /** 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. + * + * TODO: Technically this would be solvable by a "triggerable" scheduler. + */ + private[remote] def triggerNextTableAdvertisement(): Unit = // TODO expose and use in tests + runNextTableAdvertisement() + + def scheduleNextTableAdvertisement(): Unit = + tableAdvertisementInterval match { + case d: FiniteDuration ⇒ + try { + system.scheduler.scheduleOnce(d, ScheduledTableAdvertisementRunnable)(system.dispatcher) + log.debug("Scheduled {} advertisement in [{}] from now...", getClass.getSimpleName, PrettyDuration.format(tableAdvertisementInterval, includeNanos = false, 1)) + } catch { + case ex: IllegalStateException ⇒ + log.warning("Unable to schedule {} advertisement, " + + "likely system is shutting down. " + + "Reason: {}", getClass.getName, ex.getMessage) + } + case _ ⇒ // ignore... + } + + private val ScheduledTableAdvertisementRunnable = new Runnable { + override def run(): Unit = + try runNextTableAdvertisement() + finally scheduleNextTableAdvertisement() + } + + /** + * 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. + */ + def runNextTableAdvertisement() = { // TODO guard against re-entrancy? + inboundContext.association(originUid) match { + case OptionVal.Some(association) ⇒ + val table = prepareCompressionAdvertisement() + nextTable = table.invert // TODO expensive, check if building the other way wouldn't be faster? + advertiseCompressionTable(association, table) + + 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.warning("No Association for originUid [{}] yet, unable to advertise compression table.", originUid) + } + } + + /** + * Must be implementeed by extending classes in order to send a [[akka.remote.artery.ControlMessage]] + * of apropriate type to the remote system in order to advertise the compression table to it. + */ + protected def advertiseCompressionTable(association: OutboundContext, table: CompressionTable[T]): Unit + + /** Drop `activeTable` and start using the `nextTable` in its place. */ + private def flipTables(): Unit = { + log.debug("Swaping active decompression table to version {}.", nextTable.version) + activeTable = nextTable + nextTable = DecompressionTable.empty + // TODO we want to keep the currentTableVersion in State too, update here as well then + } + + private def prepareCompressionAdvertisement(): CompressionTable[T] = { + // TODO surely we can do better than that, optimise + CompressionTable(activeTable.version + 1, Map(heavyHitters.snapshot.filterNot(_ == null).zipWithIndex: _*)) + } + + override def toString = + s"""${getClass.getSimpleName}(countMinSketch: $cms, heavyHitters: $heavyHitters)""" + +} + +final class ExistingcompressedIdReuseAttemptException(id: Long, value: Any) + extends RuntimeException( + s"Attempted to re-allocate compressedId [$id] which is still in use for compressing [$value]! " + + s"This should never happen and is likely an implementation bug.") + +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.") diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompression.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompression.scala new file mode 100644 index 0000000000..03224b491c --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompression.scala @@ -0,0 +1,106 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import java.util.concurrent.atomic.AtomicReference +import java.{ util ⇒ ju } + +import akka.actor.{ ActorRef, ActorSystem, Address } +import akka.event.Logging +import akka.remote.artery.compress.OutboundCompression.OutboundCompressionState + +import scala.annotation.tailrec + +/** INTERNAL API */ +private[remote] final class OutboundActorRefCompression(system: ActorSystem, remoteAddress: Address) + extends OutboundCompressionTable[ActorRef](system, remoteAddress) { + + flipTable(CompressionTable( + version = 0, + map = Map( + system.deadLetters → 0 + ) + )) +} + +/** + * INTERNAL API + * Base class for all outgoing compression. + * Encapsulates the compressedId registration and lookup. + */ +private[remote] class OutboundCompressionTable[T](system: ActorSystem, remoteAddress: Address) + extends AtomicReference[OutboundCompressionState[T]](OutboundCompressionState.initial) { // TODO could be instead via Unsafe + import OutboundCompression._ + + // TODO: The compression map may benefit from padding if we want multiple compressions to be running in parallel + + private[this] val log = Logging(system, "OutboundCompressionTable") + + /** + * Flips the currently used compression table to the new one (iff the new one has a version number higher than the currently used one). + */ + // (╯°□°)╯︵ ┻━┻ + @tailrec final def flipTable(activate: CompressionTable[T]): Unit = { + val state = get() + if (state.version < activate.version) // TODO or we could demand it to be strictly `currentVersion + 1` + if (compareAndSet(state, prepareState(activate))) + log.debug("Successfully flipped compression table to version {}, for ourgoing connection to {}", activate.version, remoteAddress) + else + flipTable(activate) // retry + else if (state.version == activate.version) + log.warning("Received duplicate compression table (version: {})! Ignoring it.", state.version) + else + log.error("Received unexpected compression table with version nr [{}]! " + + "Current version number is []") + + } + + // TODO this is crazy hot-path; optimised FastUtil-like Object->int hash map would perform better here (and avoid Integer) allocs + final def compress(value: T): Int = + get().table.getOrDefault(value, NotCompressedId) + + private final def prepareState(activate: CompressionTable[T]): OutboundCompressionState[T] = { + val size = activate.map.size + // load factor is `1` since we will never grow this table beyond the initial size, + // this way we can avoid any rehashing from happening. + val m = new ju.HashMap[T, Integer](size, 1.0f) // TODO could be replaced with primitive `int` specialized version + val it = activate.map.keysIterator + var i = 0 + while (it.hasNext) { + m.put(it.next(), i) // TODO boxing :< + i += 1 + } + OutboundCompressionState(activate.version, m) + } + + def toDebugString: String = { + s"""${Logging.simpleName(getClass)}( + | version: ${get.version} to [$remoteAddress] + | ${get.table} + |)""".stripMargin + } + + override def toString = { + val s = get + s"""${Logging.simpleName(getClass)}(to: $remoteAddress, version: ${s.version}, compressedEntries: ${s.table.size})""" + } + +} + +/** INTERNAL API */ +private[remote] object OutboundCompression { + // format: OFF + final val DeadLettersId = 0 + final val NotCompressedId = -1 + + // format: ON + + /** INTERNAL API */ + private[remote] final case class OutboundCompressionState[T](version: Long, table: ju.Map[T, Integer]) + private[remote] object OutboundCompressionState { + def initial[T] = OutboundCompressionState[T](-1, ju.Collections.emptyMap()) + } + +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompressionTable.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompressionTable.scala deleted file mode 100644 index 0b22effa2b..0000000000 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompressionTable.scala +++ /dev/null @@ -1,125 +0,0 @@ -/* - * Copyright (C) 2016 Lightbend Inc. - */ - -package akka.remote.artery.compress - -import akka.actor.{ Address, ActorRef, ActorSystem } -import akka.event.Logging - -import scala.annotation.tailrec - -final class OutboundActorRefCompressionTable(system: ActorSystem, remoteAddress: Address) - extends OutboundCompressionTable[ActorRef](system, remoteAddress) { - - preAllocate( - system.deadLetters → 0 - ) - - // if (system.toString.contains("systemB")) - // system.log.error(new Throwable, "new OutboundActorRefCompressionTable = " + this.hashCode()) - - def preAllocate(allocations: (ActorRef, Int)*): Unit = - allocations foreach { case (ref, id) ⇒ register(ref, id) } -} - -/** - * Base class for all outgoing compression. - * Encapsulates the compressedId registration and lookup. - * - * Not thread safe. - */ -class OutboundCompressionTable[T](system: ActorSystem, remoteAddress: Address) { - import OutboundCompressionTable._ - - private val settings = CompressionSettings(system) - - private val log = system.log - - // TODO can we specialize this? (tuning due here) - @volatile private[this] var backing = Map.empty[T, Int] // TODO could use unsafe to swap the map instead of volatile - - // mapping guarding - private[this] var compressionIdsAllocated = -1 - private[this] var aheadAllocatedCompressionIds = Set.empty[Int] - - def register(value: T, id: Int): Unit = { - backing.get(value) match { - case None if isNextCompressionId(id) ⇒ - log.debug("Outbound: Registering new compression from [{}] to [{}].", value, id) // TODO should be debug - addFastForwardCompressionIdsAllocatedCounter() - backing = backing.updated(value, id) - - if (settings.debug) log.debug("Outgoing: Updated compression table state: \n{}", toDebugString) // TODO debug - - case None ⇒ - // TODO could be wrong? since we can not guarantee alocations come in sequence? - if (compressionIdAlreadyAllocated(id)) - throw new AllocatedSameIdMultipleTimesException(id, backing.find(_._2 == id).get._1, value) - - aheadAllocatedCompressionIds += id - backing = backing.updated(value, id) - - case Some(existingId) ⇒ - throw new ConflictingCompressionException(value, id, existingId) - } - } - - def compressionIdAlreadyAllocated(id: Int): Boolean = - id <= compressionIdsAllocated || aheadAllocatedCompressionIds.contains(id) - - def compress(value: T): Int = { - backing.get(value) match { // TODO possibly optimise avoid the Option? Depends on used Map - case None ⇒ NotCompressedId - case Some(id) ⇒ id - } - } - - private def isNextCompressionId(id: Int): Boolean = - id == compressionIdsAllocated + 1 - - private def addFastForwardCompressionIdsAllocatedCounter(): Unit = { - @tailrec def fastForwardConsume(): Unit = { - val nextId = compressionIdsAllocated + 1 - if (aheadAllocatedCompressionIds.contains(nextId)) { - aheadAllocatedCompressionIds = aheadAllocatedCompressionIds.filterNot(_ == nextId) - compressionIdsAllocated += 1 - fastForwardConsume() - } else () - } - - compressionIdsAllocated += 1 - fastForwardConsume() - } - - def toDebugString: String = { - val pad = backing.keys.iterator.map(_.toString.length).max - s"""${Logging.simpleName(getClass)}( - | hashCode: ${this.hashCode()} to [$remoteAddress] - | compressionIdsAllocated: ${compressionIdsAllocated + 1}, - | aheadAllocatedCompressionIds: $aheadAllocatedCompressionIds) - | - | ${backing.map { case (k, v) ⇒ k.toString.padTo(pad, " ").mkString("") + " => " + v }.mkString("\n ")} - |)""".stripMargin - } - - override def toString = - s"""${Logging.simpleName(getClass)}(compressionIdsAllocated: ${compressionIdsAllocated + 1}, aheadAllocatedCompressionIds: $aheadAllocatedCompressionIds)""" -} -object OutboundCompressionTable { - // format: OFF - final val DeadLettersId = 0 - final val NotCompressedId = -1 - // format: ON -} - -final class ConflictingCompressionException(value: Any, id: Int, existingId: Int) - extends IllegalStateException( - s"Value [$value] was already given a compression id [$id], " + - s"yet new compressionId for it was given: $existingId. This could lead to inconsistencies!") - -final class AllocatedSameIdMultipleTimesException(id: Int, previousValue: Any, conflictingValue: Any) - extends IllegalStateException( - s"Attempted to allocate compression id [$id] second time, " + - s"was already bound to value [$previousValue], " + - s"tried to bind to [$conflictingValue]!") 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 index 70fb236702..4f6a9e0a15 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/TopHeavyHitters.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/TopHeavyHitters.scala @@ -30,8 +30,13 @@ private[remote] final class TopHeavyHitters[T](val max: Int) { private[this] val items: Array[T] = Array.ofDim[Object](max).asInstanceOf[Array[T]] private[this] val weights: Array[Long] = Array.ofDim(max) - /** Slow operation, mostly exposed for testing and debugging purposes, avoid using in hot paths. */ - def itemsSnapshot: immutable.Seq[T] = Util.immutableSeq(items).filter(_ != null) + // TODO think if we could get away without copy + /** Returns copy(!) of items which are currently considered to be heavy hitters. */ + def snapshot: Array[T] = { + val snap = Array.ofDim(max).asInstanceOf[Array[T]] + System.arraycopy(items, 0, snap, 0, items.length) + snap + } def toDebugString = s"""TopHeavyHitters( diff --git a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala index 1fea6b4629..3aa3c64773 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala @@ -3,14 +3,14 @@ package akka.remote.artery import java.nio.{ ByteBuffer, ByteOrder } import akka.actor._ -import akka.remote.artery.compress.CompressionTestUtils +import akka.remote.artery.compress.{ CompressionTable, CompressionTestUtils } import akka.testkit.AkkaSpec -import akka.util.{ OptionVal, ByteString } +import akka.util.{ ByteString, OptionVal } class EnvelopeBufferSpec extends AkkaSpec { import CompressionTestUtils._ - object TestCompressor extends InboundCompression with OutboundCompression { + object TestCompressor extends InboundCompressions with OutboundCompressions { val refToIdx: Map[ActorRef, Int] = Map( minimalRef("compressable0") → 0, minimalRef("compressable1") → 1, @@ -27,15 +27,15 @@ class EnvelopeBufferSpec extends AkkaSpec { "manifest1" → 1) val idxToManifest = manifestToIdx.map(_.swap) - override def allocateActorRefCompressionId(ref: ActorRef, id: Int): Unit = ??? // dynamic allocating not implemented here + override def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit = ??? // dynamic allocating not needed in these tests override def compressActorRef(ref: ActorRef): Int = refToIdx.getOrElse(ref, -1) - override def hitActorRef(address: Address, ref: ActorRef): Unit = () - override def decompressActorRef(idx: Int): OptionVal[ActorRef] = OptionVal.Some(idxToRef(idx)) + override def hitActorRef(originUid: Long, tableVersion: Int, remote: Address, ref: ActorRef): Unit = () + override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = OptionVal(idxToRef(idx)) - override def allocateClassManifestCompressionId(manifest: String, id: Int): Unit = ??? // dynamic allocating not implemented here + override def applyClassManifestCompressionTable(table: CompressionTable[String]): Unit = ??? // dynamic allocating not needed in these tests override def compressClassManifest(manifest: String): Int = manifestToIdx.getOrElse(manifest, -1) - override def hitClassManifest(address: Address, manifest: String): Unit = () - override def decompressClassManifest(idx: Int) = OptionVal.Some(idxToManifest(idx)) + override def hitClassManifest(originUid: Long, tableVersion: Int, remote: Address, manifest: String): Unit = () + override def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] = OptionVal(idxToManifest(idx)) } "EnvelopeBuffer" must { @@ -45,13 +45,18 @@ class EnvelopeBufferSpec extends AkkaSpec { 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.version = 1 - headerIn.uid = 42 - headerIn.serializer = 4 - headerIn.senderActorRef = minimalRef("compressable0") - headerIn.recipientActorRef = minimalRef("compressable1") - headerIn.manifest = "manifest1" + headerIn setVersion 1 + headerIn setUid 42 + headerIn setSerializer 4 + headerIn setActorRefCompressionTableVersion 0xCAFE + headerIn setClassManifestCompressionTableVersion 0xBABE + headerIn setRecipientActorRef minimalRef("compressable1") + headerIn setSenderActorRef minimalRef("compressable0") + + headerIn setManifest "manifest1" envelope.writeHeader(headerIn) envelope.byteBuffer.position() should ===(EnvelopeBuffer.LiteralsSectionOffset) // Fully compressed header @@ -61,25 +66,29 @@ class EnvelopeBufferSpec extends AkkaSpec { headerOut.version should ===(1) headerOut.uid should ===(42) + headerOut.actorRefCompressionTableVersion should ===(0xCAFE) + headerOut.classManifestCompressionTableVersion should ===(0xBABE) headerOut.serializer should ===(4) - headerOut.senderActorRefPath should ===("akka://EnvelopeBufferSpec/compressable0") - headerOut.recipientActorRefPath should ===("akka://EnvelopeBufferSpec/compressable1") - headerOut.manifest should ===("manifest1") + 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) should ===("manifest1") } "be able to encode and decode headers with uncompressed literals" in { - headerIn.version = 1 - headerIn.uid = 42 - headerIn.serializer = 4 - headerIn.senderActorRef = minimalRef("uncompressable0") - headerIn.recipientActorRef = minimalRef("uncompressable11") - headerIn.manifest = "uncompressable3333" + headerIn setVersion 1 + headerIn setUid 42 + headerIn setSerializer 4 + headerIn setSenderActorRef minimalRef("uncompressable0") + headerIn setRecipientActorRef minimalRef("uncompressable11") + headerIn setManifest "uncompressable3333" val expectedHeaderLength = EnvelopeBuffer.LiteralsSectionOffset + // Constant header part - 2 + headerIn.senderActorRefPath.length + // Length field + literal - 2 + headerIn.recipientActorRefPath.length + // Length field + literal - 2 + headerIn.manifest.length // Length field + literal + 2 + headerIn.senderActorRefPath.get.length + // Length field + literal + 2 + headerIn.recipientActorRefPath.get.length + // Length field + literal + 2 + headerIn.manifest(originUid).length // Length field + literal envelope.writeHeader(headerIn) envelope.byteBuffer.position() should ===(expectedHeaderLength) @@ -90,23 +99,25 @@ class EnvelopeBufferSpec extends AkkaSpec { headerOut.version should ===(1) headerOut.uid should ===(42) headerOut.serializer should ===(4) - headerOut.senderActorRefPath should ===("akka://EnvelopeBufferSpec/uncompressable0") - headerOut.recipientActorRefPath should ===("akka://EnvelopeBufferSpec/uncompressable11") - headerOut.manifest should ===("uncompressable3333") + 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) should ===("uncompressable3333") } "be able to encode and decode headers with mixed literals" in { - headerIn.version = 1 - headerIn.uid = 42 - headerIn.serializer = 4 - headerIn.senderActorRef = minimalRef("reallylongcompressablestring") - headerIn.recipientActorRef = minimalRef("uncompressable1") - headerIn.manifest = "manifest1" + 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.position() should ===( EnvelopeBuffer.LiteralsSectionOffset + - 2 + headerIn.recipientActorRefPath.length) + 2 + headerIn.recipientActorRefPath.get.length) envelope.byteBuffer.flip() envelope.parseHeader(headerOut) @@ -114,22 +125,24 @@ class EnvelopeBufferSpec extends AkkaSpec { headerOut.version should ===(1) headerOut.uid should ===(42) headerOut.serializer should ===(4) - headerOut.senderActorRefPath should ===("akka://EnvelopeBufferSpec/reallylongcompressablestring") - headerOut.recipientActorRefPath should ===("akka://EnvelopeBufferSpec/uncompressable1") - headerOut.manifest should ===("manifest1") + 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) should ===("manifest1") - headerIn.version = 3 - headerIn.uid = Long.MinValue - headerIn.serializer = -1 - headerIn.senderActorRef = minimalRef("uncompressable0") - headerIn.recipientActorRef = minimalRef("reallylongcompressablestring") - headerIn.manifest = "longlonglongliteralmanifest" + headerIn setVersion 3 + headerIn setUid Long.MinValue + headerIn setSerializer -1 + headerIn setSenderActorRef minimalRef("uncompressable0") + headerIn setRecipientActorRef minimalRef("reallylongcompressablestring") + headerIn setManifest "longlonglongliteralmanifest" envelope.writeHeader(headerIn) envelope.byteBuffer.position() should ===( EnvelopeBuffer.LiteralsSectionOffset + - 2 + headerIn.senderActorRefPath.length + - 2 + headerIn.manifest.length) + 2 + headerIn.senderActorRefPath.get.length + + 2 + headerIn.manifest(originUid).length) envelope.byteBuffer.flip() envelope.parseHeader(headerOut) @@ -137,20 +150,22 @@ class EnvelopeBufferSpec extends AkkaSpec { headerOut.version should ===(3) headerOut.uid should ===(Long.MinValue) headerOut.serializer should ===(-1) - headerOut.senderActorRefPath should ===("akka://EnvelopeBufferSpec/uncompressable0") - headerOut.recipientActorRefPath should ===("akka://EnvelopeBufferSpec/reallylongcompressablestring") - headerOut.manifest should ===("longlonglongliteralmanifest") + 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) should ===("longlonglongliteralmanifest") } "be able to encode and decode headers with mixed literals and payload" in { val payload = ByteString("Hello Artery!") - headerIn.version = 1 - headerIn.uid = 42 - headerIn.serializer = 4 - headerIn.senderActorRef = minimalRef("reallylongcompressablestring") - headerIn.recipientActorRef = minimalRef("uncompressable1") - headerIn.manifest = "manifest1" + 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) @@ -161,9 +176,11 @@ class EnvelopeBufferSpec extends AkkaSpec { headerOut.version should ===(1) headerOut.uid should ===(42) headerOut.serializer should ===(4) - headerOut.senderActorRefPath should ===("akka://EnvelopeBufferSpec/reallylongcompressablestring") - headerOut.recipientActorRefPath should ===("akka://EnvelopeBufferSpec/uncompressable1") - headerOut.manifest should ===("manifest1") + 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) should ===("manifest1") ByteString.fromByteBuffer(envelope.byteBuffer) should ===(payload) } diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala index 0035043f87..1d378a6ea9 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -75,7 +75,7 @@ private[akka] class TestOutboundContext( _associationState.uniqueRemoteAddress.value match { case Some(Success(`peer`)) ⇒ // our value case _ ⇒ - _associationState = _associationState.newIncarnation(Promise.successful(peer), NoOutboundCompression) + _associationState = _associationState.newIncarnation(Promise.successful(peer), NoOutboundCompressions) } } 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 index 8d2c230ec8..f312f118d1 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala @@ -33,6 +33,14 @@ object CompressionIntegrationSpec { remote.artery.port = 0 remote.handshake-timeout = 10s + remote.artery.advanced.compression { + enabled = on + actor-refs { + enabled = on + advertisement-interval = 3 seconds + } + } + } """) @@ -67,13 +75,19 @@ class CompressionIntegrationSpec extends AkkaSpec(CompressionIntegrationSpec.com // cause testActor-1 to become a heavy hitter (1 to messagesToExchange).foreach { i ⇒ voidSel ! "hello" } // does not reply, but a hot receiver should be advertised - val a1 = aProbe.expectMsgType[Events.ReceivedCompressionAdvertisement](10.seconds) + val a1 = aProbe.expectMsgType[Events.ReceivedCompressionTable[ActorRef]](10.seconds) info("System [A] received: " + a1) - a1.id should ===(1) - a1.key.toString should include(testActor.path.name) + assertCompression[ActorRef](a1.table, 0, _ should ===(system.deadLetters)) + assertCompression[ActorRef](a1.table, 1, _ should ===(testActor)) } } + def assertCompression[T](table: CompressionTable[T], id: Int, assertion: T ⇒ Unit): Unit = { + table.map.find(_._2 == id) + .orElse { throw new AssertionError(s"No key was compressed to the id [$id]! Table was: $table") } + .foreach(i ⇒ assertion(i._1)) + } + def identify(_system: String, port: Int, name: String) = { val selection = system.actorSelection(s"artery://${_system}@localhost:$port/user/$name") 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..27b61c03d2 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTableSpec.scala @@ -0,0 +1,36 @@ +/* + * 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(1, Map("0" → 0, "1" → 1, "2" → 2, "3" → 3)).invert + decomp.table should ===(Array("0", "1", "2", "3")) + } + + "enforce to start allocating from 0th index" in { + val compressionTable = CompressionTable(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(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/CompressionTestKit.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTestKit.scala new file mode 100644 index 0000000000..9872374252 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTestKit.scala @@ -0,0 +1,17 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +/* INTERNAL API */ +private[remote] trait CompressionTestKit { + def assertCompression[T](table: CompressionTable[T], id: Int, assertion: T ⇒ Unit): Unit = { + table.map.find(_._2 == id) + .orElse { throw new AssertionError(s"No key was compressed to the id [$id]! Table was: $table") } + .foreach(i ⇒ assertion(i._1)) + } +} + +/* INTERNAL API */ +private[remote] object CompressionTestKit extends CompressionTestKit 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 index 9549072906..433c2c5acc 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala @@ -4,7 +4,7 @@ package akka.remote.artery.compress -import akka.actor.{ ActorIdentity, ActorSystem, Identify } +import akka.actor.{ ActorIdentity, ActorRef, ActorSystem, Identify } import akka.remote.artery.compress.CompressionProtocol.Events import akka.testkit._ import akka.util.Timeout @@ -25,14 +25,17 @@ object HandshakeShouldDropCompressionTableSpec { actor.provider = "akka.remote.RemoteActorRefProvider" remote.artery.enabled = on - remote.artery.advanced { - compression.enabled = on - compression.debug = on - } remote.artery.hostname = localhost remote.artery.port = 0 remote.handshake-timeout = 10s - + + remote.artery.advanced.compression { + enabled = on + actor-refs { + enabled = on + advertisement-interval = 3 seconds + } + } } """) @@ -42,7 +45,8 @@ object HandshakeShouldDropCompressionTableSpec { } class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDropCompressionTableSpec.commonConfig) - with ImplicitSender with BeforeAndAfter { + with ImplicitSender with BeforeAndAfter + with CompressionTestKit { import HandshakeShouldDropCompressionTableSpec._ implicit val t = Timeout(3.seconds) @@ -70,18 +74,16 @@ class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDr // cause testActor-1 to become a heavy hitter (1 to messagesToExchange).foreach { i ⇒ voidSel ! "hello" } // does not reply, but a hot receiver should be advertised // give it enough time to advertise first table - val a0 = aProbe.expectMsgType[Events.ReceivedCompressionAdvertisement](10.seconds) + val a0 = aProbe.expectMsgType[Events.ReceivedCompressionTable[ActorRef]](10.seconds) info("System [A] received: " + a0) - a0.id should ===(1) - a0.key.toString should include(testActor.path.name) + assertCompression[ActorRef](a0.table, 1, _.toString should include(testActor.path.name)) // cause a1Probe to become a heavy hitter (we want to not have it in the 2nd compression table later) (1 to messagesToExchange).foreach { i ⇒ voidSel.tell("hello", a1Probe.ref) } // does not reply, but a hot receiver should be advertised // give it enough time to advertise first table - val a1 = aProbe.expectMsgType[Events.ReceivedCompressionAdvertisement](10.seconds) + val a1 = aProbe.expectMsgType[Events.ReceivedCompressionTable[ActorRef]](10.seconds) info("System [A] received: " + a1) - a1.id should ===(2) - a1.key.toString should include(a1Probe.ref.path.name) + assertCompression[ActorRef](a1.table, 2, _.toString should include(a1Probe.ref.path.name)) log.warning("SHUTTING DOWN system {}...", systemB) shutdown(systemB) @@ -92,17 +94,15 @@ class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDr systemB.actorOf(TestActors.blackholeProps, "void") // start it again (1 to messagesToExchange).foreach { i ⇒ voidSel ! "hello" } // does not reply, but a hot receiver should be advertised // compression triggered again - val a2 = aProbe.expectMsgType[Events.ReceivedCompressionAdvertisement](10.seconds) + val a2 = aProbe.expectMsgType[Events.ReceivedCompressionTable[ActorRef]](10.seconds) info("System [A] received: " + a2) - a2.id should ===(1) - a2.key.toString should include(testActor.path.name) + assertCompression[ActorRef](a2.table, 1, _.toString should include(testActor.path.name)) (1 to messagesToExchange).foreach { i ⇒ voidSel.tell("hello", aNew2Probe.ref) } // does not reply, but a hot receiver should be advertised // compression triggered again - val a3 = aProbe.expectMsgType[Events.ReceivedCompressionAdvertisement](10.seconds) + val a3 = aProbe.expectMsgType[Events.ReceivedCompressionTable[ActorRef]](10.seconds) info("Received second compression: " + a3) - a3.id should ===(2) - a3.key.toString should include(aNew2Probe.ref.path.name) + assertCompression[ActorRef](a3.table, 2, _.toString should include(aNew2Probe.ref.path.name)) } } 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 index 3120140b93..647d81f318 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/HeavyHittersSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/HeavyHittersSpec.scala @@ -12,57 +12,57 @@ class HeavyHittersSpec extends WordSpecLike with Matchers { "should work" in { val hitters = new TopHeavyHitters[String](3) hitters.update("A", 10) shouldBe true - hitters.itemsSnapshot.toSet should ===(Set("A")) + hitters.snapshot.filter(_ ne null).toSet should ===(Set("A")) hitters.update("B", 20) shouldBe true - hitters.itemsSnapshot.toSet should ===(Set("A", "B")) + hitters.snapshot.filter(_ ne null).toSet should ===(Set("A", "B")) hitters.update("C", 1) shouldBe true - hitters.itemsSnapshot.toSet should ===(Set("A", "B", "C")) + hitters.snapshot.filter(_ ne null).toSet should ===(Set("A", "B", "C")) hitters.update("D", 100) shouldBe true - hitters.itemsSnapshot.toSet should ===(Set("A", "B", "D")) + hitters.snapshot.filter(_ ne null).toSet should ===(Set("A", "B", "D")) hitters.update("E", 200) shouldBe true - hitters.itemsSnapshot.toSet should ===(Set("B", "D", "E")) + hitters.snapshot.filter(_ ne null).toSet should ===(Set("B", "D", "E")) hitters.update("BB", 22) shouldBe true - hitters.itemsSnapshot.toSet should ===(Set("BB", "D", "E")) + hitters.snapshot.filter(_ ne null).toSet should ===(Set("BB", "D", "E")) hitters.update("a", 1) shouldBe false - hitters.itemsSnapshot.toSet should ===(Set("BB", "D", "E")) + hitters.snapshot.filter(_ ne null).toSet should ===(Set("BB", "D", "E")) } "correctly replace a hitter" in { val hitters = new TopHeavyHitters[String](3) hitters.update("A", 10) shouldBe true - hitters.itemsSnapshot.toSet should ===(Set("A")) + hitters.snapshot.filter(_ ne null).toSet should ===(Set("A")) hitters.update("A", 12) shouldBe false hitters.update("A", 22) shouldBe false - hitters.itemsSnapshot.toSet should ===(Set("A")) + 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](3) hitters.update("A", 1) shouldBe true - hitters.itemsSnapshot.toSet should ===(Set("A")) + hitters.snapshot.filter(_ ne null).toSet should ===(Set("A")) hitters.update("B", 22) shouldBe true - hitters.itemsSnapshot.toSet should ===(Set("A", "B")) + hitters.snapshot.filter(_ ne null).toSet should ===(Set("A", "B")) hitters.update("C", 33) shouldBe true - hitters.itemsSnapshot.toSet should ===(Set("A", "B", "C")) + hitters.snapshot.filter(_ ne null).toSet should ===(Set("A", "B", "C")) hitters.lowestHitterWeight should ===(1) // first item which forces dropping least heavy hitter hitters.update("D", 100) shouldBe true - hitters.itemsSnapshot.toSet should ===(Set("B", "C", "D")) + hitters.snapshot.filter(_ ne null).toSet should ===(Set("B", "C", "D")) // second item which forces dropping least heavy hitter hitters.update("X", 999) shouldBe true - hitters.itemsSnapshot.toSet should ===(Set("X", "C", "D")) + hitters.snapshot.filter(_ ne null).toSet should ===(Set("X", "C", "D")) } "replace the right item even when hashCodes collide" in { @@ -73,19 +73,19 @@ class HeavyHittersSpec extends WordSpecLike with Matchers { val b1 = MockHashCode("B", 1) hitters.update(a1, 1) - hitters.itemsSnapshot.toSet should ===(Set(a1)) + hitters.snapshot.filter(_ ne null).toSet should ===(Set(a1)) hitters.lowestHitterWeight should ===(0) hitters.update(b1, 2) - hitters.itemsSnapshot.toSet should ===(Set(a1, b1)) + hitters.snapshot.filter(_ ne null).toSet should ===(Set(a1, b1)) hitters.lowestHitterWeight should ===(1) hitters.update(a1, 10) - hitters.itemsSnapshot.toSet should ===(Set(a1, b1)) + hitters.snapshot.filter(_ ne null).toSet should ===(Set(a1, b1)) hitters.lowestHitterWeight should ===(2) hitters.update(b1, 100) - hitters.itemsSnapshot.toSet should ===(Set(a1, b1)) + hitters.snapshot.filter(_ ne null).toSet should ===(Set(a1, b1)) hitters.lowestHitterWeight should ===(10) } diff --git a/akka-remote/src/test/scala/akka/remote/artery/compress/InboundCompressionTableSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/InboundCompressionTableSpec.scala deleted file mode 100644 index 7b7d4688eb..0000000000 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/InboundCompressionTableSpec.scala +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Copyright (C) 2009-2016 Lightbend Inc. - */ - -package akka.remote.artery.compress - -import akka.actor.Address -import akka.stream.impl.ConstantFun -import akka.testkit.AkkaSpec - -class InboundCompressionTableSpec extends AkkaSpec { - - "InboundCompressionTable" must { - val NoChange: (String, Int) = null - - "invoke callback when compression triggered" in { - var p: (String, Int) = NoChange - val heavyHitters = new TopHeavyHitters[String](2) - val advertiseCompressionId = new AdvertiseCompressionId[String] { - override def apply(remoteAddress: Address, ref: String, id: Int): Unit = - p = ref → id - } - val table = new InboundCompressionTable[String](system, heavyHitters, ConstantFun.scalaIdentityFunction, advertiseCompressionId) - - table.increment(null, "A", 1L) - p should ===("A" → 0) - - table.increment(null, "B", 1L) - p should ===("B" → 1) - - p = NoChange - table.increment(null, "A", 1L) // again, yet was already compressed (A count == 2), thus no need to compress (call callback) again - p should ===(NoChange) // no change - - table.increment(null, "B", 1L) // again, yet was already compressed (B count == 2), thus no need to compress (call callback) again - p should ===(NoChange) // no change - - table.increment(null, "C", 1L) // max hitters = 2; [A=2, B=2] C=1 - p should ===(NoChange) // no change - - table.increment(null, "C", 1L) // max hitters = 2; [A=2, B=2] C=2 – causes compression of C! - p should ===(NoChange) // no change - table.increment(null, "C", 1L) // max hitters = 2; [..., C=3] – causes compression of C! - p should ===("C" → 2) // allocated - - p = NoChange - table.increment(null, "A", 1L) // again! - p should ===(NoChange) - - p = NoChange - table.increment(null, "B", 1L) // again! - p should ===(NoChange) - - // and again and again... won't be signalled again since already compressed - table.increment(null, "A", 1L) - table.increment(null, "A", 1L) - table.increment(null, "A", 1L) - p should ===(NoChange) - } - } - -} 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..723a4c9a13 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/OutboundCompressionSpec.scala @@ -0,0 +1,50 @@ +/* + * Copyright (C) 2009-2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import akka.actor._ +import akka.testkit.AkkaSpec + +class OutboundCompressionSpec extends AkkaSpec { + import CompressionTestUtils._ + + val remoteAddress = Address("artery", "example", "localhost", 0) + + "OutboundCompression" must { + "not compress not-known values" in { + val table = new OutboundActorRefCompression(system, remoteAddress) + table.compress(minimalRef("banana")) should ===(-1) + } + } + + "OutboundActorRefCompression" must { + val alice = minimalRef("alice") + val bob = minimalRef("bob") + + "always compress /deadLetters" in { + val table = new OutboundActorRefCompression(system, remoteAddress) + table.compress(system.deadLetters) should ===(0) + } + + "not compress unknown actor ref" in { + val table = new OutboundActorRefCompression(system, remoteAddress) + table.compress(alice) should ===(-1) // not compressed + } + + "compress previously registered actor ref" in { + val compression = new OutboundActorRefCompression(system, remoteAddress) + val table = CompressionTable(1, Map(system.deadLetters → 0, alice → 1)) + compression.flipTable(table) + compression.compress(alice) should ===(1) // compressed + compression.compress(bob) should ===(-1) // not compressed + + val table2 = table.copy(2, map = table.map.updated(bob, 2)) + compression.flipTable(table2) + compression.compress(alice) should ===(1) // compressed + compression.compress(bob) should ===(2) // compressed + } + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/artery/compress/OutboundCompressionTableSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/OutboundCompressionTableSpec.scala deleted file mode 100644 index 5af8be2fbe..0000000000 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/OutboundCompressionTableSpec.scala +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Copyright (C) 2009-2016 Lightbend Inc. - */ - -package akka.remote.artery.compress - -import akka.actor._ -import akka.testkit.AkkaSpec - -class OutboundCompressionTableSpec extends AkkaSpec { - import CompressionTestUtils._ - - val remoteAddress = Address("artery", "example", "localhost", 0) - - "OutboundCompressionTable" must { - "not compress not-known values" in { - val table = new OutboundActorRefCompressionTable(system, remoteAddress) - table.compress(minimalRef("banana")) should ===(-1) - } - } - - "OutboundActorRefCompressionTable" must { - val alice = minimalRef("alice") - val bob = minimalRef("bob") - - "always compress /deadLetters" in { - val table = new OutboundActorRefCompressionTable(system, remoteAddress) - table.compress(system.deadLetters) should ===(0) - } - - "not compress unknown actor ref" in { - val table = new OutboundActorRefCompressionTable(system, remoteAddress) - table.compress(alice) should ===(-1) // not compressed - } - - "compress previously registered actor ref" in { - val table = new OutboundActorRefCompressionTable(system, remoteAddress) - table.register(alice, 1) - table.compress(alice) should ===(1) // compressed - - table.compress(bob) should ===(-1) // not compressed - } - - "fail if same id attempted to be registered twice" in { - val table = new OutboundActorRefCompressionTable(system, remoteAddress) - table.register(alice, 1) - val ex = intercept[AllocatedSameIdMultipleTimesException] { - table.register(bob, 1) - } - - ex.getMessage should include("Attempted to allocate compression id [1] second time, " + - "was already bound to value [Actor[akka://OutboundCompressionTableSpec/alice]], " + - "tried to bind to [Actor[akka://OutboundCompressionTableSpec/bob]]!") - } - - "survive compression ahead-allocation, and then fast forward allocated Ids counter when able to (compact storage)" in { - val table = new OutboundActorRefCompressionTable(system, remoteAddress) - table.register(alice, 1) - table.compressionIdAlreadyAllocated(1) should ===(true) - - table.register(bob, 3) // ahead allocated - table.compressionIdAlreadyAllocated(2) should ===(false) - table.compressionIdAlreadyAllocated(3) should ===(true) - - table.register(minimalRef("oogie-boogie"), 4) // ahead allocated (we're able to survive re-delivery of allocation messages) - table.compressionIdAlreadyAllocated(2) should ===(false) - table.compressionIdAlreadyAllocated(4) should ===(true) - - table.register(minimalRef("jack-skellington"), 2) // missing allocation was re-delivered, cause fast-forward - - table.compressionIdAlreadyAllocated(2) should ===(true) - - table.register(minimalRef("jack-sparrow"), 5) // immediate next, after fast-forward - } - - // FIXME "fast forward" concept will not exist once we use "advertise entire table", possibly remove mentions of that - // TODO cover more cases of holes in the redeliveries of advertisements - // TODO ^ to cover the fast forward logic a bit more - } - -} From a021eb5ff40170760655b139c93424875ad2ff7d Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 23 Jun 2016 18:11:56 +0200 Subject: [PATCH 081/186] flush messages on shutdown, #20811 * StreamSupervisor as system actor so that it is stopped after ordinary actors * when transport is shutdown send flush message to all outbound associations (over control stream) and wait for ack or timeout --- .../remote/routing/RemoteRandomSpec.scala | 3 - .../akka/remote/artery/ArteryTransport.scala | 155 +++++++++++++----- .../akka/remote/artery/Association.scala | 2 +- .../scala/akka/remote/artery/Control.scala | 10 ++ .../remote/artery/SystemMessageDelivery.scala | 8 + .../artery/SystemMessageDeliverySpec.scala | 27 ++- ...dshakeShouldDropCompressionTableSpec.scala | 13 +- .../scala/akka/stream/ActorMaterializer.scala | 16 ++ 8 files changed, 181 insertions(+), 53 deletions(-) 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 0b9331b8e4..318b71a4ad 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 @@ -99,9 +99,6 @@ class RemoteRandomSpec(multiNodeConfig: RemoteRandomConfig) extends MultiNodeSpe // "Terminate" to a shut down node system.stop(actor) enterBarrier("done") - - // FIXME this test has problems shutting down actor system when running with Artery - // [akka.actor.ActorSystemImpl(RemoteRandomSpec)] Failed to stop [RemoteRandomSpec] within [5 seconds] } } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 10417fdc2f..4e1c158f69 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -69,6 +69,8 @@ import io.aeron.driver.ThreadingMode import org.agrona.concurrent.BackoffIdleStrategy import org.agrona.concurrent.BusySpinIdleStrategy import scala.util.control.NonFatal +import akka.actor.Props +import akka.actor.Actor /** * INTERNAL API @@ -313,6 +315,50 @@ private[akka] trait OutboundContext { def dummyRecipient: RemoteActorRef } +/** + * 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 = associations.flatMap(_.associationState.uniqueRemoteAddressValue) + + val timeoutTask = context.system.scheduler.scheduleOnce(timeout, self, FlushOnShutdown.Timeout)(context.dispatcher) + + override def preStart(): Unit = { + val msg = ActorSystemTerminating(inboundContext.localAddress) + associations.foreach { a ⇒ a.send(msg, OptionVal.Some(self), a.dummyRecipient) } + } + + override def postStop(): Unit = + timeoutTask.cancel() + + def receive = { + case ActorSystemTerminatingAck(from) ⇒ + remaining -= from + if (remaining.isEmpty) { + done.trySuccess(Done) + context.stop(self) + } + case FlushOnShutdown.Timeout ⇒ + done.trySuccess(Done) + context.stop(self) + } +} + /** * INTERNAL API */ @@ -352,6 +398,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R "handshake-timeout must be > 0") private val injectHandshakeInterval: FiniteDuration = 1.second private val giveUpSendAfter: FiniteDuration = 60.seconds + private val shutdownFlushTimeout = 1.second + + private val remoteDispatcher = system.dispatchers.lookup(remoteSettings.Dispatcher) private val largeMessageDestinations = system.settings.config.getStringList("akka.remote.artery.large-message-destinations").asScala.foldLeft(WildcardTree[NotUsed]()) { (tree, entry) ⇒ @@ -380,8 +429,15 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R val (afrFileChannel, afrFlie, flightRecorder) = initializeFlightRecorder() + def createFlightRecorderEventSink(): EventSink = { + // FIXME there is some concurrency issue with the FlightRecorder, when shutting down. + // It crashes the JVM. + // flightRecorder.createEventSink() + IgnoreEventSink + } + // !!! WARNING !!! This is *NOT* thread safe, - private val topLevelFREvents = flightRecorder.createEventSink() + private val topLevelFREvents = createFlightRecorderEventSink() private val associationRegistry = new AssociationRegistry( remoteAddress ⇒ new Association(this, materializer, remoteAddress, controlSubject, largeMessageDestinations)) @@ -413,7 +469,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R val materializerSettings = ActorMaterializerSettings( remoteSettings.config.getConfig("akka.remote.artery.advanced.materializer")) - materializer = ActorMaterializer(materializerSettings)(system) + materializer = ActorMaterializer.systemMaterializer(materializerSettings, "remote", system) messageDispatcher = new MessageDispatcher(system, provider) topLevelFREvents.loFreq(Transport_MaterializerStarted, NoMetaData) @@ -563,23 +619,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R controlSubject = ctrl - // ordinary messages stream controlSubject.attach(new ControlMessageObserver { override def notify(inboundEnvelope: InboundEnvelope): Unit = { - inboundEnvelope.message match { - case Quarantined(from, to) if to == localAddress ⇒ - val lifecycleEvent = ThisActorSystemQuarantinedEvent(localAddress.address, from.address) - publishLifecycleEvent(lifecycleEvent) - // quarantine the other system from here - association(from.address).quarantine(lifecycleEvent.toString, Some(from.uid)) - case _ ⇒ // not interesting - } - } - }) - - // compression messages - controlSubject.attach(new ControlMessageObserver { - override def notify(inboundEnvelope: InboundEnvelope): Unit = inboundEnvelope.message match { case m: CompressionMessage ⇒ m match { @@ -593,8 +634,22 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R association(from.address).compression.applyClassManifestCompressionTable(table) system.eventStream.publish(CompressionProtocol.Events.ReceivedCompressionTable(from, table)) } - case _ ⇒ // not interested in non CompressionMessages + + case Quarantined(from, to) if to == localAddress ⇒ + val lifecycleEvent = ThisActorSystemQuarantinedEvent(localAddress.address, from.address) + publishLifecycleEvent(lifecycleEvent) + // quarantine the other system from here + association(from.address).quarantine(lifecycleEvent.toString, Some(from.uid)) + + case _: ActorSystemTerminating ⇒ + inboundEnvelope.sender match { + case OptionVal.Some(snd) ⇒ snd.tell(ActorSystemTerminatingAck(localAddress), ActorRef.noSender) + case OptionVal.None ⇒ log.error("Expected sender for ActorSystemTerminating message") + } + + case _ ⇒ // not interesting } + } }) attachStreamRestart("Inbound control stream", completed, () ⇒ runInboundControlStream(compression)) @@ -661,28 +716,42 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R override def shutdown(): Future[Done] = { _shutdown = true - killSwitch.shutdown() - topLevelFREvents.loFreq(Transport_KillSwitchPulled, NoMetaData) - if (taskRunner != null) { - taskRunner.stop() - topLevelFREvents.loFreq(Transport_Stopped, NoMetaData) + val allAssociations = associationRegistry.allAssociations + val flushing: Future[Done] = + if (allAssociations.isEmpty) Future.successful(Done) + else { + val flushingPromise = Promise[Done]() + system.systemActorOf(FlushOnShutdown.props(flushingPromise, shutdownFlushTimeout, + this, allAssociations).withDispatcher(remoteSettings.Dispatcher), "remoteFlushOnShutdown") + flushingPromise.future + } + implicit val ec = remoteDispatcher + flushing.recover { case _ ⇒ Done }.map { _ ⇒ + killSwitch.shutdown() + + topLevelFREvents.loFreq(Transport_KillSwitchPulled, NoMetaData) + if (taskRunner != null) { + taskRunner.stop() + topLevelFREvents.loFreq(Transport_Stopped, NoMetaData) + } + if (aeronErrorLogTask != null) { + aeronErrorLogTask.cancel() + topLevelFREvents.loFreq(Transport_AeronErrorLogTaskStopped, NoMetaData) + } + if (aeron != null) aeron.close() + if (mediaDriver.isDefined) { + stopMediaDriver() + topLevelFREvents.loFreq(Transport_MediaFileDeleted, NoMetaData) + } + topLevelFREvents.loFreq(Transport_FlightRecorderClose, NoMetaData) + + flightRecorder.close() + afrFileChannel.force(true) + afrFileChannel.close() + // TODO: Be smarter about this in tests and make it always-on-for prod + afrFlie.delete() + Done } - if (aeronErrorLogTask != null) { - aeronErrorLogTask.cancel() - topLevelFREvents.loFreq(Transport_AeronErrorLogTaskStopped, NoMetaData) - } - if (aeron != null) aeron.close() - if (mediaDriver.isDefined) { - stopMediaDriver() - topLevelFREvents.loFreq(Transport_MediaFileDeleted, NoMetaData) - } - topLevelFREvents.loFreq(Transport_FlightRecorderClose, NoMetaData) - flightRecorder.close() - afrFileChannel.force(true) - afrFileChannel.close() - // TODO: Be smarter about this in tests and make it always-on-for prod - afrFlie.delete() - Future.successful(Done) } private[remote] def isShutdown: Boolean = _shutdown @@ -742,7 +811,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .via(new OutboundHandshake(system, outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) .via(encoder(compression)) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), ordinaryStreamId, aeron, taskRunner, - envelopePool, giveUpSendAfter, flightRecorder.createEventSink()))(Keep.right) + envelopePool, giveUpSendAfter, createFlightRecorderEventSink()))(Keep.right) } def outboundLarge(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[Send, Future[Done]] = { @@ -750,7 +819,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .via(new OutboundHandshake(system, outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) .via(createEncoder(largeEnvelopePool, compression)) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), largeStreamId, aeron, taskRunner, - envelopePool, giveUpSendAfter, flightRecorder.createEventSink()))(Keep.right) + envelopePool, giveUpSendAfter, createFlightRecorderEventSink()))(Keep.right) } def outboundControl(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[Send, (OutboundControlIngress, Future[Done])] = { @@ -761,7 +830,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .viaMat(new OutboundControlJunction(outboundContext))(Keep.right) .via(encoder(compression)) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), controlStreamId, aeron, taskRunner, - envelopePool, Duration.Inf, flightRecorder.createEventSink()))(Keep.both) + envelopePool, Duration.Inf, createFlightRecorderEventSink()))(Keep.both) // FIXME we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages } @@ -780,7 +849,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R def aeronSource(streamId: Int, pool: EnvelopeBufferPool): Source[EnvelopeBuffer, NotUsed] = Source.fromGraph(new AeronSource(inboundChannel, streamId, aeron, taskRunner, pool, - flightRecorder.createEventSink())) + createFlightRecorderEventSink())) val messageDispatcherSink: Sink[InboundEnvelope, Future[Done]] = Sink.foreach[InboundEnvelope] { m ⇒ messageDispatcher.dispatch(m.recipient.get, m.recipientAddress, m.message, m.sender) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 523da7ff8f..af5660fd21 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -171,7 +171,7 @@ private[remote] class Association( if (message.isInstanceOf[ActorSelectionMessage] || !associationState.isQuarantined() || message == ClearSystemMessageDelivery) { // FIXME: Use a different envelope than the old Send, but make sure the new is handled by deadLetters properly message match { - case _: SystemMessage | ClearSystemMessageDelivery ⇒ + case _: SystemMessage | ClearSystemMessageDelivery | _: ControlMessage ⇒ val send = Send(message, sender, recipient, None) if (!controlQueue.offer(send)) { quarantine(reason = s"Due to overflow of control queue, size [$controlQueueSize]") diff --git a/akka-remote/src/main/scala/akka/remote/artery/Control.scala b/akka-remote/src/main/scala/akka/remote/artery/Control.scala index c88d63a43a..f22a7a5d3c 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Control.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Control.scala @@ -37,6 +37,16 @@ private[akka] trait ControlMessage */ private[akka] final case class Quarantined(from: UniqueAddress, to: UniqueAddress) extends ControlMessage // FIXME serialization +/** + * INTERNAL API + */ +private[akka] case class ActorSystemTerminating(from: UniqueAddress) extends ControlMessage // FIXME serialization + +/** + * INTERNAL API + */ +private[akka] case class ActorSystemTerminatingAck(from: UniqueAddress) // FIXME serialization + /** * INTERNAL API */ diff --git a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala index 12bb70cec0..320c00603b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -169,6 +169,14 @@ private[akka] class SystemMessageDelivery( case s @ Send(ClearSystemMessageDelivery, _, _, _) ⇒ clear() pull(in) + case s @ Send(msg: ControlMessage, _, _, _) ⇒ + // e.g. ActorSystemTerminating, no need for acked delivery + if (resending.isEmpty && isAvailable(out)) + push(out, s) + else { + resending.offer(s) + tryResend() + } case s @ Send(msg: AnyRef, _, _, _) ⇒ if (unacknowledged.size < maxBufferSize) { seqNo += 1 diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index 5df50600d1..2ebedb00c3 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -35,6 +35,7 @@ import akka.util.OptionVal object SystemMessageDeliverySpec { val config = ConfigFactory.parseString(s""" + akka.loglevel=INFO akka { actor.provider = remote remote.artery.enabled = on @@ -108,7 +109,7 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi "System messages" must { "be delivered with real actors" in { - val actorOnSystemB = systemB.actorOf(TestActors.echoActorProps, "echo") + systemB.actorOf(TestActors.echoActorProps, "echo") val remoteRef = { system.actorSelection(rootB / "user" / "echo") ! Identify(None) @@ -120,6 +121,30 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi 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 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 index 433c2c5acc..673f70cbcc 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala @@ -22,7 +22,7 @@ object HandshakeShouldDropCompressionTableSpec { val commonConfig = ConfigFactory.parseString(s""" akka { loglevel = INFO - + actor.provider = "akka.remote.RemoteActorRefProvider" remote.artery.enabled = on remote.artery.hostname = localhost @@ -63,8 +63,7 @@ class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDr // listen for compression table events val aProbe = TestProbe() val a1Probe = TestProbe() - val aNew2Probe = TestProbe() - val b1Probe = TestProbe() + val b1Probe = TestProbe()(systemB) system.eventStream.subscribe(aProbe.ref, classOf[CompressionProtocol.Events.Event]) systemB.eventStream.subscribe(b1Probe.ref, classOf[CompressionProtocol.Events.Event]) @@ -91,16 +90,20 @@ class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDr Thread.sleep(5000) log.warning("SYSTEM READY {}...", systemB) + val aNewProbe = TestProbe() + system.eventStream.subscribe(aNewProbe.ref, classOf[CompressionProtocol.Events.Event]) + systemB.actorOf(TestActors.blackholeProps, "void") // start it again (1 to messagesToExchange).foreach { i ⇒ voidSel ! "hello" } // does not reply, but a hot receiver should be advertised // compression triggered again - val a2 = aProbe.expectMsgType[Events.ReceivedCompressionTable[ActorRef]](10.seconds) + val a2 = aNewProbe.expectMsgType[Events.ReceivedCompressionTable[ActorRef]](10.seconds) info("System [A] received: " + a2) assertCompression[ActorRef](a2.table, 1, _.toString should include(testActor.path.name)) + val aNew2Probe = TestProbe() (1 to messagesToExchange).foreach { i ⇒ voidSel.tell("hello", aNew2Probe.ref) } // does not reply, but a hot receiver should be advertised // compression triggered again - val a3 = aProbe.expectMsgType[Events.ReceivedCompressionTable[ActorRef]](10.seconds) + val a3 = aNewProbe.expectMsgType[Events.ReceivedCompressionTable[ActorRef]](10.seconds) info("Received second compression: " + a3) assertCompression[ActorRef](a3.table, 2, _.toString should include(aNew2Probe.ref.path.name)) } diff --git a/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala b/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala index 1e7ede7b60..0c2d7cebaa 100644 --- a/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala +++ b/akka-stream/src/main/scala/akka/stream/ActorMaterializer.scala @@ -79,6 +79,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]] From 9ffe211546a7060a02bc8e46641bce3ab9b3cff7 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 23 Jun 2016 19:19:59 +0200 Subject: [PATCH 082/186] adjust oversized error message --- akka-remote/src/main/scala/akka/remote/artery/Codecs.scala | 2 +- .../akka/remote/artery/RemoteMessageSerializationSpec.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 49d4c6bdd8..03577e289d 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -76,7 +76,7 @@ private[remote] class Encoder( throw e case _ if e.isInstanceOf[java.nio.BufferOverflowException] ⇒ val reason = new OversizedPayloadException(s"Discarding oversized payload sent to ${send.recipient}: max allowed size ${envelope.byteBuffer.limit()} bytes. Message type [${send.message.getClass.getName}].") - log.error(reason, "Transient association error (association remains live)") + log.error(reason, "Failed to serialize oversized message [{}].", send.message.getClass.getName) pull(in) case _ ⇒ log.error(e, "Failed to serialize message [{}].", send.message.getClass.getName) diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala index d0ed9e5a93..77fdca9792 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala @@ -55,7 +55,7 @@ class RemoteMessageSerializationSpec extends ArteryMultiNodeSpec(""" "drop sent messages over payload size" in { val oversized = byteStringOfSize(maxPayloadBytes + 1) - EventFilter[OversizedPayloadException](pattern = ".*Discarding oversized payload sent.*", occurrences = 1).intercept { + EventFilter[OversizedPayloadException](start = "Failed to serialize oversized message", occurrences = 1).intercept { verifySend(oversized) { expectNoMsg(1.second) // No AssocitionErrorEvent should be published } From 4865afe5abc6bbe7f08e420e8d036f7ba414264a Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 23 Jun 2016 19:37:44 +0200 Subject: [PATCH 083/186] remove invalid actorFor test * it's actually possible to send msg to new incarnation with ref aquired with actorFor, the uid has special 0 value when using actorFor --- .../remote/artery/RemoteActorForSpec.scala | 30 ------------------- .../artery/RemoteSendConsistencySpec.scala | 22 ++++++++++++++ 2 files changed, 22 insertions(+), 30 deletions(-) diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorForSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorForSpec.scala index 80536c106e..69c4640c7c 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorForSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorForSpec.scala @@ -51,36 +51,6 @@ class RemoteActorForSpec extends ArteryMultiNodeSpec("akka.loglevel=INFO") with }(remoteSystem) } - // FIXME can't communicate with new ref looked up after starting a new instance (!?!) - "not send to remote re-created actor with same name" ignore { - - def lookItUp() = localSystem.actorFor(s"artery://${remoteSystem.name}@localhost:$remotePort/user/re-created") - - val echo1 = remoteSystem.actorOf(TestActors.echoActorProps, "re-created") - val remoteRef1 = lookItUp() - remoteRef1 ! 2 - expectMsg(2) - - // now stop and start a new actor with the same name - watch(echo1) - remoteSystem.stop(echo1) - expectTerminated(echo1) - - val echo2 = remoteSystem.actorOf(TestActors.echoActorProps, "re-created") - val remoteRef2 = lookItUp() - remoteRef2 ! 2 - expectMsg(2) - - // the old ref should not interact with the - // new actor instance at the same path - remoteRef1 ! 3 - expectNoMsg(1.second) - - // and additionally, but it would have failed already - // if this wasn't true - remoteRef1.path.uid should !==(remoteRef2.path.uid) - } - // FIXME needs remote deployment section "look-up actors across node boundaries" ignore { val l = localSystem.actorOf(Props(new Actor { diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala index a800d86dbc..8149919ee6 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala @@ -9,6 +9,9 @@ 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 object RemoteSendConsistencySpec { @@ -52,7 +55,26 @@ class RemoteSendConsistencySpec extends AkkaSpec(RemoteSendConsistencySpec.confi 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 { From 5f4b265c6b604a563366904a00c111290d27c3db Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 29 Jun 2016 12:36:17 +0200 Subject: [PATCH 084/186] enable flight recorder by config * SyncronizedEventSink for the one used concurrently in ArteryTransport * jvm is still crashing so disabled it for now --- akka-remote/src/main/resources/reference.conf | 5 ++ .../scala/akka/remote/RemoteSettings.scala | 2 + .../akka/remote/artery/ArteryTransport.scala | 48 +++++++++++-------- .../scala/akka/remote/artery/BufferPool.scala | 2 +- .../akka/remote/artery/FlightRecorder.scala | 21 ++++++++ 5 files changed, 58 insertions(+), 20 deletions(-) diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 21f1eae9a7..f4e466d7df 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -134,6 +134,11 @@ akka { # Level 1 strongly prefer low CPU consumption over low latency. # Level 10 strongly prefer low latency over low CPU consumption. idle-cpu-level = 5 + + flight-recorder { + // FIXME it should be enabled by default, but there is some concurrency issue that crashes the JVM + enabled = off + } # compression of common strings in remoting messages, like actor destinations, serializers etc compression { diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index 77fd6b18cb..a4b11f41bb 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -34,6 +34,8 @@ final class RemoteSettings(val config: Config) { val IdleCpuLevel: Int = getInt("akka.remote.artery.advanced.idle-cpu-level").requiring(level ⇒ 1 <= level && level <= 10, "idle-cpu-level must be between 1 and 10") + val FlightRecorderEnabled: Boolean = getBoolean("akka.remote.artery.advanced.flight-recorder.enabled") + val ArteryCompressionSettings = CompressionSettings(getConfig("akka.remote.artery.advanced.compression")) val LogReceive: Boolean = getBoolean("akka.remote.log-received-messages") diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 4e1c158f69..800d1fd99d 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -427,17 +427,24 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R 16, create = () ⇒ new ReusableInboundEnvelope, clear = inEnvelope ⇒ inEnvelope.asInstanceOf[ReusableInboundEnvelope].clear()) - val (afrFileChannel, afrFlie, flightRecorder) = initializeFlightRecorder() - - def createFlightRecorderEventSink(): EventSink = { - // FIXME there is some concurrency issue with the FlightRecorder, when shutting down. - // It crashes the JVM. - // flightRecorder.createEventSink() - IgnoreEventSink + val (afrFileChannel, afrFlie, flightRecorder) = initializeFlightRecorder() match { + case None ⇒ (None, None, None) + case Some((c, f, r)) ⇒ (Some(c), Some(f), Some(r)) } - // !!! WARNING !!! This is *NOT* thread safe, - private val topLevelFREvents = createFlightRecorderEventSink() + 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, remoteAddress, controlSubject, largeMessageDestinations)) @@ -745,11 +752,11 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } topLevelFREvents.loFreq(Transport_FlightRecorderClose, NoMetaData) - flightRecorder.close() - afrFileChannel.force(true) - afrFileChannel.close() + flightRecorder.foreach(_.close()) + afrFileChannel.foreach(_.force(true)) + afrFileChannel.foreach(_.close()) // TODO: Be smarter about this in tests and make it always-on-for prod - afrFlie.delete() + afrFlie.foreach(_.delete()) Done } } @@ -893,13 +900,16 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .toMat(messageDispatcherSink)(Keep.both) } - private def initializeFlightRecorder(): (FileChannel, File, FlightRecorder) = { - // TODO: Figure out where to put it, currently using temporary files - val afrFile = File.createTempFile("artery", ".afr") - afrFile.deleteOnExit() + private def initializeFlightRecorder(): Option[(FileChannel, File, FlightRecorder)] = { + if (remoteSettings.FlightRecorderEnabled) { + // TODO: Figure out where to put it, currently using temporary files + val afrFile = File.createTempFile("artery", ".afr") + afrFile.deleteOnExit() - val fileChannel = FlightRecorder.prepareFileForFlightRecorder(afrFile) - (fileChannel, afrFile, new FlightRecorder(fileChannel)) + val fileChannel = FlightRecorder.prepareFileForFlightRecorder(afrFile) + Some((fileChannel, afrFile, new FlightRecorder(fileChannel))) + } else + None } def inboundTestFlow: Flow[InboundEnvelope, InboundEnvelope, TestManagementApi] = diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index d2c2f1883e..77b865c4b4 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -205,7 +205,7 @@ private[remote] final class HeaderBuilderImpl(inboundCompression: InboundCompres override def isNoSender: Boolean = (_senderActorRef eq null) && _senderActorRefIdx == EnvelopeBuffer.DeadLettersCode override def senderActorRef(originUid: Long): OptionVal[ActorRef] = - if (_senderActorRef eq null) inboundCompression.decompressActorRef(originUid, actorRefCompressionTableVersion, _senderActorRefIdx) + if (_senderActorRef eq null) inboundCompression.decompressActorRef(originUid, actorRefCompressionTableVersion, _senderActorRefIdx) else OptionVal.None def senderActorRefPath: OptionVal[String] = OptionVal(_senderActorRef) diff --git a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala index 820125998b..f86248cb08 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala @@ -36,6 +36,27 @@ private[remote] object IgnoreEventSink extends EventSink { 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 * From b2089d06a738f7140e5b44a6e32af7a1227798e6 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 29 Jun 2016 17:09:33 +0200 Subject: [PATCH 085/186] new OutboundEnvelope * instead of the old Send * optional recipient, remove of dummy * pool of OutboundEnvelope --- .../akka/remote/artery/CodecBenchmark.scala | 33 ++-- .../remote/testkit/PerfFlamesSupport.scala | 5 +- .../remote/artery/MaxThroughputSpec.scala | 2 +- .../akka/remote/RemoteActorRefProvider.scala | 5 + .../akka/remote/artery/ArteryTransport.scala | 147 ++++-------------- .../akka/remote/artery/Association.scala | 84 +++++----- .../scala/akka/remote/artery/Codecs.scala | 48 +++--- .../scala/akka/remote/artery/Control.scala | 19 +-- .../scala/akka/remote/artery/Handshake.scala | 23 +-- .../akka/remote/artery/InboundEnvelope.scala | 108 +++++++++++++ .../scala/akka/remote/artery/ObjectPool.scala | 2 +- .../akka/remote/artery/OutboundEnvelope.scala | 86 ++++++++++ .../remote/artery/SystemMessageDelivery.scala | 47 +++--- .../scala/akka/remote/artery/TestStage.scala | 9 +- .../artery/compress/InboundCompressions.scala | 10 +- .../remote/artery/FlightRecorderSpec.scala | 2 +- .../artery/OutboundControlJunctionSpec.scala | 9 +- .../remote/artery/OutboundHandshakeSpec.scala | 9 +- .../artery/SystemMessageDeliverySpec.scala | 40 ++--- .../akka/remote/artery/TestContext.scala | 3 - 20 files changed, 407 insertions(+), 284 deletions(-) create mode 100644 akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/OutboundEnvelope.scala 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 index 0454c0e1c1..e7e4233284 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -19,7 +19,6 @@ import scala.concurrent.duration._ import akka.NotUsed import akka.actor._ import akka.remote.AddressUidExtension -import akka.remote.EndpointManager.Send import akka.remote.RARP import akka.remote.RemoteActorRef import akka.remote.UniqueAddress @@ -54,11 +53,9 @@ class CodecBenchmark { implicit val system = ActorSystem("CodecBenchmark", config) val systemB = ActorSystem("systemB", system.settings.config) - val envelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) - val inboundEnvelopePool = new ObjectPool[InboundEnvelope]( - 16, - create = () ⇒ new ReusableInboundEnvelope, clear = inEnvelope ⇒ inEnvelope.asInstanceOf[ReusableInboundEnvelope].clear() - ) + private val envelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) + private val inboundEnvelopePool = ReusableInboundEnvelope.createObjectPool(capacity = 16) + private val outboundEnvelopePool = ReusableOutboundEnvelope.createObjectPool(capacity = 16) val compressionOut = NoOutboundCompressions val headerIn = HeaderBuilder.in(NoopInboundCompressions) @@ -139,11 +136,11 @@ class CodecBenchmark { val latch = new CountDownLatch(1) val N = 100000 - val encoder: Flow[Send, EnvelopeBuffer, NotUsed] = - Flow.fromGraph(new Encoder(uniqueLocalAddress, system, compressionOut, envelopePool)) + val encoder: Flow[OutboundEnvelope, EnvelopeBuffer, NotUsed] = + Flow.fromGraph(new Encoder(uniqueLocalAddress, system, compressionOut, outboundEnvelopePool, envelopePool)) Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) - .map(_ ⇒ Send(payload, OptionVal.None, remoteRefB, None)) + .map(msg ⇒ outboundEnvelopePool.acquire().init(OptionVal.None, payload, OptionVal.Some(remoteRefB))) .via(encoder) .map(envelope => envelopePool.release(envelope)) .runWith(new LatchSink(N, latch))(materializer) @@ -180,9 +177,9 @@ class CodecBenchmark { envelope } .via(decoder) - .map { env => - inboundEnvelopePool.release(env) - () + .map { + case env: ReusableInboundEnvelope => inboundEnvelopePool.release(env) + case _ => } .runWith(new LatchSink(N, latch))(materializer) @@ -196,8 +193,8 @@ class CodecBenchmark { val latch = new CountDownLatch(1) val N = 100000 - val encoder: Flow[Send, EnvelopeBuffer, NotUsed] = - Flow.fromGraph(new Encoder(uniqueLocalAddress, system, compressionOut, envelopePool)) + val encoder: Flow[OutboundEnvelope, EnvelopeBuffer, NotUsed] = + Flow.fromGraph(new Encoder(uniqueLocalAddress, system, compressionOut, outboundEnvelopePool, envelopePool)) val localRecipient = resolvedRef.path.toSerializationFormatWithAddress(uniqueLocalAddress.address) val provider = RARP(system).provider @@ -213,12 +210,12 @@ class CodecBenchmark { resolveActorRefWithLocalAddress, NoopInboundCompressions, envelopePool, inboundEnvelopePool)) Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) - .map(_ ⇒ Send(payload, OptionVal.None, remoteRefB, None)) + .map(msg ⇒ outboundEnvelopePool.acquire().init(OptionVal.None, payload, OptionVal.Some(remoteRefB))) .via(encoder) .via(decoder) - .map { env => - inboundEnvelopePool.release(env) - () + .map { + case env: ReusableInboundEnvelope => inboundEnvelopePool.release(env) + case _ => } .runWith(new LatchSink(N, latch))(materializer) 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 index 34311bd1b1..3ca461388c 100644 --- 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 @@ -16,10 +16,10 @@ import scala.concurrent.duration._ */ 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 = { @@ -53,6 +53,5 @@ private[akka] trait PerfFlamesSupport { _: MultiNodeSpec ⇒ if (!isIt) println(s"WARN: perf-java-flames not available under [$perfJavaFlamesPath]! Skipping perf profiling.") isIt } - } 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 index 121a166dc6..2bdedd2681 100644 --- 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 @@ -205,7 +205,7 @@ class MaxThroughputSpecMultiJvmNode2 extends MaxThroughputSpec abstract class MaxThroughputSpec extends MultiNodeSpec(MaxThroughputSpec) - with STMultiNodeSpec with ImplicitSender + with STMultiNodeSpec with ImplicitSender with PerfFlamesSupport { import MaxThroughputSpec._ diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 9561ac1d55..ac83fc7d54 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -19,6 +19,7 @@ import akka.ConfigurationException import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics } import akka.remote.artery.ArteryTransport import akka.util.OptionVal +import akka.remote.artery.OutboundEnvelope /** * INTERNAL API @@ -96,6 +97,10 @@ private[akka] object RemoteActorRefProvider { // 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) + case env: OutboundEnvelope ⇒ + super.!(env.message)(env.sender.orNull) + case DeadLetter(env: OutboundEnvelope, _, _) ⇒ + super.!(env.message)(env.sender.orNull) case _ ⇒ super.!(message)(sender) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 800d1fd99d..5daec306a9 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -21,7 +21,6 @@ import akka.actor.Cancellable import akka.event.Logging import akka.event.LoggingAdapter import akka.remote.AddressUidExtension -import akka.remote.EndpointManager.Send import akka.remote.EventPublisher import akka.remote.RemoteActorRef import akka.remote.RemoteActorRefProvider @@ -72,96 +71,6 @@ import scala.util.control.NonFatal import akka.actor.Props import akka.actor.Actor -/** - * INTERNAL API - */ -private[akka] object InboundEnvelope { - def apply( - recipient: OptionVal[InternalActorRef], - recipientAddress: Address, - message: AnyRef, - sender: OptionVal[ActorRef], - originUid: Long, - association: OptionVal[OutboundContext]): InboundEnvelope = { - val env = new ReusableInboundEnvelope - env.init(recipient, recipientAddress, message, sender, originUid, association) - env - } - -} - -/** - * INTERNAL API - */ -private[akka] trait InboundEnvelope { - def recipient: OptionVal[InternalActorRef] - def recipientAddress: Address - def message: AnyRef - def sender: OptionVal[ActorRef] - def originUid: Long - def association: OptionVal[OutboundContext] - - def withMessage(message: AnyRef): InboundEnvelope - - def withRecipient(ref: InternalActorRef): InboundEnvelope -} - -/** - * INTERNAL API - */ -private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { - private var _recipient: OptionVal[InternalActorRef] = OptionVal.None - private var _recipientAddress: Address = null - private var _message: AnyRef = null - private var _sender: OptionVal[ActorRef] = OptionVal.None - private var _originUid: Long = 0L - private var _association: OptionVal[OutboundContext] = OptionVal.None - - override def recipient: OptionVal[InternalActorRef] = _recipient - override def recipientAddress: Address = _recipientAddress - override def message: AnyRef = _message - override def sender: OptionVal[ActorRef] = _sender - override def originUid: Long = _originUid - override def association: OptionVal[OutboundContext] = _association - - override def withMessage(message: AnyRef): InboundEnvelope = { - _message = message - this - } - - def withRecipient(ref: InternalActorRef): InboundEnvelope = { - _recipient = OptionVal(ref) - this - } - - def clear(): Unit = { - _recipient = OptionVal.None - _recipientAddress = null - _message = null - _sender = OptionVal.None - _originUid = 0L - _association = OptionVal.None - } - - def init( - recipient: OptionVal[InternalActorRef], - recipientAddress: Address, - message: AnyRef, - sender: OptionVal[ActorRef], - originUid: Long, - association: OptionVal[OutboundContext]): Unit = { - _recipient = recipient - _recipientAddress = recipientAddress - _message = message - _sender = sender - _originUid = originUid - _association = association - } - - override def toString: String = - s"InboundEnvelope($recipient, $recipientAddress, $message, $sender, $originUid, $association)" -} - /** * INTERNAL API * Inbound API that is used by the stream stages. @@ -311,8 +220,6 @@ private[akka] trait OutboundContext { */ def controlSubject: ControlMessageSubject - // FIXME we should be able to Send without a recipient ActorRef - def dummyRecipient: RemoteActorRef } /** @@ -340,7 +247,7 @@ private[remote] class FlushOnShutdown(done: Promise[Done], timeout: FiniteDurati override def preStart(): Unit = { val msg = ActorSystemTerminating(inboundContext.localAddress) - associations.foreach { a ⇒ a.send(msg, OptionVal.Some(self), a.dummyRecipient) } + associations.foreach { a ⇒ a.send(msg, OptionVal.Some(self), OptionVal.None) } } override def postStop(): Unit = @@ -423,9 +330,10 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val envelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) private val largeEnvelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumLargeFrameSize, ArteryTransport.MaximumPooledBuffers) - private val inboundEnvelopePool = new ObjectPool[InboundEnvelope]( - 16, - create = () ⇒ new ReusableInboundEnvelope, clear = inEnvelope ⇒ inEnvelope.asInstanceOf[ReusableInboundEnvelope].clear()) + private val inboundEnvelopePool = ReusableInboundEnvelope.createObjectPool(capacity = 16) + // FIXME capacity of outboundEnvelopePool should probably be derived from the sendQueue capacity + // times a factor (for reasonable number of outbound streams) + private val outboundEnvelopePool = ReusableOutboundEnvelope.createObjectPool(capacity = 3072 * 2) val (afrFileChannel, afrFlie, flightRecorder) = initializeFlightRecorder() match { case None ⇒ (None, None, None) @@ -447,7 +355,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R createFlightRecorderEventSink(synchr = true) private val associationRegistry = new AssociationRegistry( - remoteAddress ⇒ new Association(this, materializer, remoteAddress, controlSubject, largeMessageDestinations)) + remoteAddress ⇒ new Association(this, materializer, remoteAddress, controlSubject, largeMessageDestinations, + outboundEnvelopePool)) def remoteSettings: RemoteSettings = provider.remoteSettings @@ -789,7 +698,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R a2 } - a.send(message, sender, recipient) + a.send(message, sender, OptionVal.Some(recipient)) } override def association(remoteAddress: Address): Association = { @@ -813,28 +722,31 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R association(remoteAddress).quarantine(reason = "", uid.map(_.toLong)) } - def outbound(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[Send, Future[Done]] = { - Flow.fromGraph(killSwitch.flow[Send]) - .via(new OutboundHandshake(system, outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) + def outbound(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[OutboundEnvelope, Future[Done]] = { + Flow.fromGraph(killSwitch.flow[OutboundEnvelope]) + .via(new OutboundHandshake(system, outboundContext, outboundEnvelopePool, handshakeTimeout, + handshakeRetryInterval, injectHandshakeInterval)) .via(encoder(compression)) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), ordinaryStreamId, aeron, taskRunner, envelopePool, giveUpSendAfter, createFlightRecorderEventSink()))(Keep.right) } - def outboundLarge(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[Send, Future[Done]] = { - Flow.fromGraph(killSwitch.flow[Send]) - .via(new OutboundHandshake(system, outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) + def outboundLarge(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[OutboundEnvelope, Future[Done]] = { + Flow.fromGraph(killSwitch.flow[OutboundEnvelope]) + .via(new OutboundHandshake(system, outboundContext, outboundEnvelopePool, handshakeTimeout, + handshakeRetryInterval, injectHandshakeInterval)) .via(createEncoder(largeEnvelopePool, compression)) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), largeStreamId, aeron, taskRunner, envelopePool, giveUpSendAfter, createFlightRecorderEventSink()))(Keep.right) } - def outboundControl(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[Send, (OutboundControlIngress, Future[Done])] = { - Flow.fromGraph(killSwitch.flow[Send]) - .via(new OutboundHandshake(system, outboundContext, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) + def outboundControl(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[OutboundEnvelope, (OutboundControlIngress, Future[Done])] = { + Flow.fromGraph(killSwitch.flow[OutboundEnvelope]) + .via(new OutboundHandshake(system, outboundContext, outboundEnvelopePool, handshakeTimeout, + handshakeRetryInterval, injectHandshakeInterval)) .via(new SystemMessageDelivery(outboundContext, system.deadLetters, systemMessageResendInterval, remoteSettings.SysMsgBufferSize)) - .viaMat(new OutboundControlJunction(outboundContext))(Keep.right) + .viaMat(new OutboundControlJunction(outboundContext, outboundEnvelopePool))(Keep.right) .via(encoder(compression)) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), controlStreamId, aeron, taskRunner, envelopePool, Duration.Inf, createFlightRecorderEventSink()))(Keep.both) @@ -842,17 +754,17 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // FIXME we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages } - def createEncoder(compression: OutboundCompressions, bufferPool: EnvelopeBufferPool): Flow[Send, EnvelopeBuffer, NotUsed] = - Flow.fromGraph(new Encoder(localAddress, system, compression, bufferPool)) + def createEncoder(compression: OutboundCompressions, bufferPool: EnvelopeBufferPool): Flow[OutboundEnvelope, EnvelopeBuffer, NotUsed] = + Flow.fromGraph(new Encoder(localAddress, system, compression, outboundEnvelopePool, bufferPool)) private def createInboundCompressions(inboundContext: InboundContext): InboundCompressions = if (remoteSettings.ArteryCompressionSettings.enabled) new InboundCompressionsImpl(system, inboundContext) else new NoInboundCompressions(system) - def createEncoder(pool: EnvelopeBufferPool, compression: OutboundCompressions): Flow[Send, EnvelopeBuffer, NotUsed] = - Flow.fromGraph(new Encoder(localAddress, system, compression, pool)) + def createEncoder(pool: EnvelopeBufferPool, compression: OutboundCompressions): Flow[OutboundEnvelope, EnvelopeBuffer, NotUsed] = + Flow.fromGraph(new Encoder(localAddress, system, compression, outboundEnvelopePool, pool)) - def encoder(compression: OutboundCompressions): Flow[Send, EnvelopeBuffer, NotUsed] = createEncoder(envelopePool, compression) + def encoder(compression: OutboundCompressions): Flow[OutboundEnvelope, EnvelopeBuffer, NotUsed] = createEncoder(envelopePool, compression) def aeronSource(streamId: Int, pool: EnvelopeBufferPool): Source[EnvelopeBuffer, NotUsed] = Source.fromGraph(new AeronSource(inboundChannel, streamId, aeron, taskRunner, pool, @@ -860,7 +772,10 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R val messageDispatcherSink: Sink[InboundEnvelope, Future[Done]] = Sink.foreach[InboundEnvelope] { m ⇒ messageDispatcher.dispatch(m.recipient.get, m.recipientAddress, m.message, m.sender) - inboundEnvelopePool.release(m) + m match { + case r: ReusableInboundEnvelope ⇒ inboundEnvelopePool.release(r) + case _ ⇒ + } } def createDecoder(compression: InboundCompressions, bufferPool: EnvelopeBufferPool): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { @@ -915,7 +830,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R def inboundTestFlow: Flow[InboundEnvelope, InboundEnvelope, TestManagementApi] = Flow.fromGraph(new InboundTestStage(this)) - def outboundTestFlow(association: Association): Flow[Send, Send, TestManagementApi] = + def outboundTestFlow(association: Association): Flow[OutboundEnvelope, OutboundEnvelope, TestManagementApi] = Flow.fromGraph(new OutboundTestStage(association)) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index af5660fd21..affceabd77 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -24,7 +24,6 @@ import akka.actor.RootActorPath import akka.dispatch.sysmsg.SystemMessage import akka.event.Logging import akka.remote.{ LargeDestination, RegularDestination, RemoteActorRef, UniqueAddress } -import akka.remote.EndpointManager.Send import akka.remote.artery.AeronSink.GaveUpSendingException import akka.remote.artery.InboundControlJunction.ControlMessageSubject import akka.remote.artery.OutboundControlJunction.OutboundControlIngress @@ -45,8 +44,8 @@ import akka.remote.artery.compress.CompressionProtocol._ * INTERNAL API */ private[remote] object Association { - final case class QueueWrapper(queue: Queue[Send]) extends SendQueue.ProducerApi[Send] { - override def offer(message: Send): Boolean = queue.offer(message) + final case class QueueWrapper(queue: Queue[OutboundEnvelope]) extends SendQueue.ProducerApi[OutboundEnvelope] { + override def offer(message: OutboundEnvelope): Boolean = queue.offer(message) } } @@ -61,7 +60,8 @@ private[remote] class Association( val materializer: Materializer, override val remoteAddress: Address, override val controlSubject: ControlMessageSubject, - largeMessageDestinations: WildcardTree[NotUsed]) + largeMessageDestinations: WildcardTree[NotUsed], + outboundEnvelopePool: ObjectPool[ReusableOutboundEnvelope]) extends AbstractAssociation with OutboundContext { import Association._ @@ -84,12 +84,12 @@ private[remote] class Association( /** Accesses the currently active outbound compression. */ def compression: OutboundCompressions = associationState.outboundCompression - def createQueue(capacity: Int): Queue[Send] = - new ManyToOneConcurrentArrayQueue[Send](capacity) + def createQueue(capacity: Int): Queue[OutboundEnvelope] = + new ManyToOneConcurrentArrayQueue[OutboundEnvelope](capacity) - @volatile private[this] var queue: SendQueue.ProducerApi[Send] = QueueWrapper(createQueue(queueSize)) - @volatile private[this] var largeQueue: SendQueue.ProducerApi[Send] = QueueWrapper(createQueue(largeQueueSize)) - @volatile private[this] var controlQueue: SendQueue.ProducerApi[Send] = QueueWrapper(createQueue(controlQueueSize)) + @volatile private[this] var queue: SendQueue.ProducerApi[OutboundEnvelope] = QueueWrapper(createQueue(queueSize)) + @volatile private[this] var largeQueue: SendQueue.ProducerApi[OutboundEnvelope] = QueueWrapper(createQueue(largeQueueSize)) + @volatile private[this] var controlQueue: SendQueue.ProducerApi[OutboundEnvelope] = QueueWrapper(createQueue(controlQueueSize)) @volatile private[this] var _outboundControlIngress: OutboundControlIngress = _ @volatile private[this] var materializing = new CountDownLatch(1) @@ -165,48 +165,51 @@ private[remote] class Association( override def sendControl(message: ControlMessage): Unit = outboundControlIngress.sendControlMessage(message) - def send(message: Any, sender: OptionVal[ActorRef], recipient: RemoteActorRef): Unit = { + def send(message: Any, sender: OptionVal[ActorRef], recipient: OptionVal[RemoteActorRef]): Unit = { + def createOutboundEnvelope(): OutboundEnvelope = + outboundEnvelopePool.acquire().init(recipient, message.asInstanceOf[AnyRef], sender) + // allow ActorSelectionMessage to pass through quarantine, to be able to establish interaction with new system // FIXME where is that ActorSelectionMessage check in old remoting? if (message.isInstanceOf[ActorSelectionMessage] || !associationState.isQuarantined() || message == ClearSystemMessageDelivery) { - // FIXME: Use a different envelope than the old Send, but make sure the new is handled by deadLetters properly message match { case _: SystemMessage | ClearSystemMessageDelivery | _: ControlMessage ⇒ - val send = Send(message, sender, recipient, None) - if (!controlQueue.offer(send)) { + val outboundEnvelope = createOutboundEnvelope() + if (!controlQueue.offer(createOutboundEnvelope())) { quarantine(reason = s"Due to overflow of control queue, size [$controlQueueSize]") - transport.system.deadLetters ! send + transport.system.deadLetters ! 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 send1 = Send(message, sender, recipient, None) - if (!controlQueue.offer(send1)) - transport.system.deadLetters ! send1 - val send2 = Send(message, sender, recipient, None) - if (!queue.offer(send2)) - transport.system.deadLetters ! send2 + val outboundEnvelope1 = createOutboundEnvelope() + if (!controlQueue.offer(outboundEnvelope1)) + transport.system.deadLetters ! outboundEnvelope1 + val outboundEnvelope2 = createOutboundEnvelope() + if (!queue.offer(outboundEnvelope2)) + transport.system.deadLetters ! outboundEnvelope2 case _ ⇒ - val send = Send(message, sender, recipient, None) + val outboundEnvelope = createOutboundEnvelope() val offerOk = if (largeMessageChannelEnabled && isLargeMessageDestination(recipient)) - largeQueue.offer(send) + largeQueue.offer(outboundEnvelope) else - queue.offer(send) + queue.offer(outboundEnvelope) if (!offerOk) - transport.system.deadLetters ! send + transport.system.deadLetters ! outboundEnvelope } } else if (log.isDebugEnabled) log.debug("Dropping message to quarantined system {}", remoteAddress) } - private def isLargeMessageDestination(recipient: ActorRef): Boolean = { + private def isLargeMessageDestination(recipient: OptionVal[RemoteActorRef]): Boolean = { recipient match { - case r: RemoteActorRef if r.cachedLargeMessageDestinationFlag ne null ⇒ r.cachedLargeMessageDestinationFlag eq LargeDestination - case r: RemoteActorRef ⇒ - if (largeMessageDestinations.find(r.path.elements.iterator).data.isEmpty) { + case OptionVal.Some(r) ⇒ + if (r.cachedLargeMessageDestinationFlag ne null) + r.cachedLargeMessageDestinationFlag eq LargeDestination + else if (largeMessageDestinations.find(r.path.elements.iterator).data.isEmpty) { r.cachedLargeMessageDestinationFlag = RegularDestination false } else { @@ -214,17 +217,10 @@ private[remote] class Association( r.cachedLargeMessageDestinationFlag = LargeDestination true } - case _ ⇒ false + case OptionVal.None ⇒ false } } - // FIXME we should be able to Send without a recipient ActorRef - override val dummyRecipient: RemoteActorRef = - try transport.provider.resolveActorRef(RootActorPath(remoteAddress) / "system" / "dummy").asInstanceOf[RemoteActorRef] - catch { - case ex: Exception ⇒ throw new Exception("Bad dummy recipient! RemoteAddress: " + remoteAddress, ex) - } - // OutboundContext override def quarantine(reason: String): Unit = { val uid = associationState.uniqueRemoteAddressValue().map(_.uid) @@ -247,7 +243,7 @@ private[remote] class Association( // FIXME when we complete the switch to Long UID we must use Long here also, issue #20644 transport.eventPublisher.notifyListeners(QuarantinedEvent(remoteAddress, u.toInt)) // end delivery of system messages to that incarnation after this point - send(ClearSystemMessageDelivery, OptionVal.None, dummyRecipient) + send(ClearSystemMessageDelivery, OptionVal.None, OptionVal.None) // try to tell the other system that we have quarantined it sendControl(Quarantined(localAddress, peer)) } else @@ -306,14 +302,14 @@ private[remote] class Association( val (queueValue, (control, completed)) = if (transport.remoteSettings.TestMode) { val ((queueValue, mgmt), (control, completed)) = - Source.fromGraph(new SendQueue[Send]) + Source.fromGraph(new SendQueue[OutboundEnvelope]) .viaMat(transport.outboundTestFlow(this))(Keep.both) .toMat(transport.outboundControl(this, compression))(Keep.both) .run()(materializer) _testStages.add(mgmt) (queueValue, (control, completed)) } else { - Source.fromGraph(new SendQueue[Send]) + Source.fromGraph(new SendQueue[OutboundEnvelope]) .toMat(transport.outboundControl(this, compression))(Keep.both) .run()(materializer) } @@ -332,7 +328,7 @@ private[remote] class Association( }) } - private def getOrCreateQueueWrapper(q: SendQueue.ProducerApi[Send], capacity: Int): QueueWrapper = + private def getOrCreateQueueWrapper(q: SendQueue.ProducerApi[OutboundEnvelope], capacity: Int): QueueWrapper = q match { case existing: QueueWrapper ⇒ existing case _ ⇒ @@ -346,14 +342,14 @@ private[remote] class Association( val (queueValue, completed) = if (transport.remoteSettings.TestMode) { - val ((queueValue, mgmt), completed) = Source.fromGraph(new SendQueue[Send]) + val ((queueValue, mgmt), completed) = Source.fromGraph(new SendQueue[OutboundEnvelope]) .viaMat(transport.outboundTestFlow(this))(Keep.both) .toMat(transport.outbound(this, compression))(Keep.both) .run()(materializer) _testStages.add(mgmt) (queueValue, completed) } else { - Source.fromGraph(new SendQueue[Send]) + Source.fromGraph(new SendQueue[OutboundEnvelope]) .toMat(transport.outbound(this, compression))(Keep.both) .run()(materializer) } @@ -371,14 +367,14 @@ private[remote] class Association( val (queueValue, completed) = if (transport.remoteSettings.TestMode) { - val ((queueValue, mgmt), completed) = Source.fromGraph(new SendQueue[Send]) + val ((queueValue, mgmt), completed) = Source.fromGraph(new SendQueue[OutboundEnvelope]) .viaMat(transport.outboundTestFlow(this))(Keep.both) .toMat(transport.outboundLarge(this, compression))(Keep.both) .run()(materializer) _testStages.add(mgmt) (queueValue, completed) } else { - Source.fromGraph(new SendQueue[Send]) + Source.fromGraph(new SendQueue[OutboundEnvelope]) .toMat(transport.outboundLarge(this, compression))(Keep.both) .run()(materializer) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 03577e289d..8e0d5d70ef 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -8,7 +8,6 @@ import scala.concurrent.duration._ import scala.util.control.NonFatal import akka.actor._ import akka.remote.{ MessageSerializer, OversizedPayloadException, UniqueAddress } -import akka.remote.EndpointManager.Send import akka.remote.artery.SystemMessageDelivery.SystemMessageEnvelope import akka.serialization.{ Serialization, SerializationExtension } import akka.stream._ @@ -21,15 +20,16 @@ import akka.stream.stage.TimerGraphStageLogic * INTERNAL API */ private[remote] class Encoder( - uniqueLocalAddress: UniqueAddress, - system: ActorSystem, - compression: OutboundCompressions, - bufferPool: EnvelopeBufferPool) - extends GraphStage[FlowShape[Send, EnvelopeBuffer]] { + uniqueLocalAddress: UniqueAddress, + system: ActorSystem, + compression: OutboundCompressions, + outboundEnvelopePool: ObjectPool[ReusableOutboundEnvelope], + bufferPool: EnvelopeBufferPool) + extends GraphStage[FlowShape[OutboundEnvelope, EnvelopeBuffer]] { - val in: Inlet[Send] = Inlet("Artery.Encoder.in") + val in: Inlet[OutboundEnvelope] = Inlet("Artery.Encoder.in") val out: Outlet[EnvelopeBuffer] = Outlet("Artery.Encoder.out") - val shape: FlowShape[Send, EnvelopeBuffer] = FlowShape(in, out) + val shape: FlowShape[OutboundEnvelope, EnvelopeBuffer] = FlowShape(in, out) override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) with InHandler with OutHandler with StageLogging { @@ -44,11 +44,14 @@ private[remote] class Encoder( override protected def logSource = classOf[Encoder] override def onPush(): Unit = { - val send = grab(in) + val outboundEnvelope = grab(in) val envelope = bufferPool.acquire() // internally compression is applied by the builder: - headerBuilder setRecipientActorRef send.recipient + outboundEnvelope.recipient match { + case OptionVal.Some(r) ⇒ headerBuilder setRecipientActorRef r + case OptionVal.None ⇒ headerBuilder.setNoRecipient() + } try { // avoiding currentTransportInformation.withValue due to thunk allocation @@ -56,12 +59,12 @@ private[remote] class Encoder( try { Serialization.currentTransportInformation.value = serializationInfo - send.senderOption match { + outboundEnvelope.sender match { case OptionVal.None ⇒ headerBuilder.setNoSender() case OptionVal.Some(s) ⇒ headerBuilder setSenderActorRef s } - MessageSerializer.serializeForArtery(serialization, send.message.asInstanceOf[AnyRef], headerBuilder, envelope) + MessageSerializer.serializeForArtery(serialization, outboundEnvelope.message, headerBuilder, envelope) } finally Serialization.currentTransportInformation.value = oldValue envelope.byteBuffer.flip() @@ -70,18 +73,24 @@ private[remote] class Encoder( } catch { case NonFatal(e) ⇒ bufferPool.release(envelope) - send.message match { + outboundEnvelope.message match { case _: SystemMessageEnvelope ⇒ - log.error(e, "Failed to serialize system message [{}].", send.message.getClass.getName) + log.error(e, "Failed to serialize system message [{}].", outboundEnvelope.message.getClass.getName) throw e case _ if e.isInstanceOf[java.nio.BufferOverflowException] ⇒ - val reason = new OversizedPayloadException(s"Discarding oversized payload sent to ${send.recipient}: max allowed size ${envelope.byteBuffer.limit()} bytes. Message type [${send.message.getClass.getName}].") - log.error(reason, "Failed to serialize oversized message [{}].", send.message.getClass.getName) + val reason = new OversizedPayloadException(s"Discarding oversized payload sent to ${outboundEnvelope.recipient}: " + + s"max allowed size ${envelope.byteBuffer.limit()} bytes. Message type [${outboundEnvelope.message.getClass.getName}].") + log.error(reason, "Failed to serialize oversized message [{}].", outboundEnvelope.message.getClass.getName) pull(in) case _ ⇒ - log.error(e, "Failed to serialize message [{}].", send.message.getClass.getName) + log.error(e, "Failed to serialize message [{}].", outboundEnvelope.message.getClass.getName) pull(in) } + } finally { + outboundEnvelope match { + case r: ReusableOutboundEnvelope ⇒ outboundEnvelopePool.release(r) + case _ ⇒ + } } } @@ -111,7 +120,7 @@ private[remote] class Decoder( resolveActorRefWithLocalAddress: String ⇒ InternalActorRef, compression: InboundCompressions, // TODO has to do demuxing on remote address It would seem, as decoder does not yet know bufferPool: EnvelopeBufferPool, - inEnvelopePool: ObjectPool[InboundEnvelope]) extends GraphStage[FlowShape[EnvelopeBuffer, InboundEnvelope]] { + inEnvelopePool: ObjectPool[ReusableInboundEnvelope]) extends GraphStage[FlowShape[EnvelopeBuffer, InboundEnvelope]] { val in: Inlet[EnvelopeBuffer] = Inlet("Artery.Decoder.in") val out: Outlet[InboundEnvelope] = Outlet("Artery.Decoder.out") val shape: FlowShape[EnvelopeBuffer, InboundEnvelope] = FlowShape(in, out) @@ -168,8 +177,7 @@ private[remote] class Decoder( val deserializedMessage = MessageSerializer.deserializeForArtery( system, originUid, serialization, headerBuilder, envelope) - val decoded = inEnvelopePool.acquire() - decoded.asInstanceOf[ReusableInboundEnvelope].init( + val decoded = inEnvelopePool.acquire().init( recipient, localAddress, // FIXME: Is this needed anymore? What should we do here? deserializedMessage, diff --git a/akka-remote/src/main/scala/akka/remote/artery/Control.scala b/akka-remote/src/main/scala/akka/remote/artery/Control.scala index f22a7a5d3c..ceac57e668 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Control.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Control.scala @@ -7,7 +7,6 @@ import java.util.ArrayDeque import scala.concurrent.Future import scala.concurrent.Promise import akka.Done -import akka.remote.EndpointManager.Send import akka.stream.Attributes import akka.stream.FlowShape import akka.stream.Inlet @@ -160,12 +159,13 @@ private[akka] object OutboundControlJunction { /** * INTERNAL API */ -private[akka] class OutboundControlJunction(outboundContext: OutboundContext) - extends GraphStageWithMaterializedValue[FlowShape[Send, Send], OutboundControlJunction.OutboundControlIngress] { +private[akka] class OutboundControlJunction( + outboundContext: OutboundContext, outboundEnvelopePool: ObjectPool[ReusableOutboundEnvelope]) + extends GraphStageWithMaterializedValue[FlowShape[OutboundEnvelope, OutboundEnvelope], OutboundControlJunction.OutboundControlIngress] { import OutboundControlJunction._ - val in: Inlet[Send] = Inlet("OutboundControlJunction.in") - val out: Outlet[Send] = Outlet("OutboundControlJunction.out") - override val shape: FlowShape[Send, Send] = FlowShape(in, out) + 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 @@ -174,7 +174,7 @@ private[akka] class OutboundControlJunction(outboundContext: OutboundContext) private val sendControlMessageCallback = getAsyncCallback[ControlMessage](internalSendControlMessage) private val maxControlMessageBufferSize: Int = 1024 // FIXME config - private val buffer = new ArrayDeque[Send] + private val buffer = new ArrayDeque[OutboundEnvelope] override def preStart(): Unit = { initCallback(sendControlMessageCallback.invoke) @@ -207,8 +207,9 @@ private[akka] class OutboundControlJunction(outboundContext: OutboundContext) } } - private def wrap(message: ControlMessage): Send = - Send(message, OptionVal.None, outboundContext.dummyRecipient, None) + private def wrap(message: ControlMessage): OutboundEnvelope = + outboundEnvelopePool.acquire().init( + recipient = OptionVal.None, message = message, sender = OptionVal.None) setHandlers(in, out, this) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala index 8a66226d6f..a7c8420377 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -7,7 +7,6 @@ import akka.actor.ActorSystem import scala.concurrent.duration._ import scala.util.control.NoStackTrace -import akka.remote.EndpointManager.Send import akka.remote.UniqueAddress import akka.stream.Attributes import akka.stream.FlowShape @@ -50,21 +49,23 @@ private[akka] object OutboundHandshake { * INTERNAL API */ private[akka] class OutboundHandshake( - system: ActorSystem, - outboundContext: OutboundContext, timeout: FiniteDuration, - retryInterval: FiniteDuration, injectHandshakeInterval: FiniteDuration) - extends GraphStage[FlowShape[Send, Send]] { + system: ActorSystem, + outboundContext: OutboundContext, + outboundEnvelopePool: ObjectPool[ReusableOutboundEnvelope], + timeout: FiniteDuration, + retryInterval: FiniteDuration, injectHandshakeInterval: FiniteDuration) + extends GraphStage[FlowShape[OutboundEnvelope, OutboundEnvelope]] { - val in: Inlet[Send] = Inlet("OutboundHandshake.in") - val out: Outlet[Send] = Outlet("OutboundHandshake.out") - override val shape: FlowShape[Send, Send] = FlowShape(in, out) + 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: Send = null + private var pendingMessage: OutboundEnvelope = null private var injectHandshakeTickScheduled = false // InHandler @@ -127,7 +128,9 @@ private[akka] class OutboundHandshake( private def pushHandshakeReq(): Unit = { injectHandshakeTickScheduled = true scheduleOnce(InjectHandshakeTick, injectHandshakeInterval) - push(out, Send(HandshakeReq(outboundContext.localAddress), OptionVal.None, outboundContext.dummyRecipient, None)) + val env: OutboundEnvelope = outboundEnvelopePool.acquire().init( + recipient = OptionVal.None, message = HandshakeReq(outboundContext.localAddress), sender = OptionVal.None) + push(out, env) } private def handshakeCompleted(): Unit = { 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..2c20cb287d --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala @@ -0,0 +1,108 @@ +/** + * 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[akka] object InboundEnvelope { + def apply( + recipient: OptionVal[InternalActorRef], + recipientAddress: Address, + message: AnyRef, + sender: OptionVal[ActorRef], + originUid: Long, + association: OptionVal[OutboundContext]): InboundEnvelope = { + val env = new ReusableInboundEnvelope + env.init(recipient, recipientAddress, message, sender, originUid, association) + } + +} + +/** + * INTERNAL API + */ +private[akka] trait InboundEnvelope { + def recipient: OptionVal[InternalActorRef] + def recipientAddress: Address + def message: AnyRef + def sender: OptionVal[ActorRef] + def originUid: Long + def association: OptionVal[OutboundContext] + + def withMessage(message: AnyRef): InboundEnvelope + + def withRecipient(ref: InternalActorRef): InboundEnvelope +} + +/** + * INTERNAL API + */ +private[akka] object ReusableInboundEnvelope { + def createObjectPool(capacity: Int) = new ObjectPool[ReusableInboundEnvelope]( + capacity, + create = () ⇒ new ReusableInboundEnvelope, clear = inEnvelope ⇒ inEnvelope.asInstanceOf[ReusableInboundEnvelope].clear()) +} + +/** + * INTERNAL API + */ +private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { + private var _recipient: OptionVal[InternalActorRef] = OptionVal.None + private var _recipientAddress: Address = null + private var _message: AnyRef = null + private var _sender: OptionVal[ActorRef] = OptionVal.None + private var _originUid: Long = 0L + private var _association: OptionVal[OutboundContext] = OptionVal.None + + override def recipient: OptionVal[InternalActorRef] = _recipient + override def recipientAddress: Address = _recipientAddress + override def message: AnyRef = _message + override def sender: OptionVal[ActorRef] = _sender + override def originUid: Long = _originUid + override def association: OptionVal[OutboundContext] = _association + + override def withMessage(message: AnyRef): InboundEnvelope = { + _message = message + this + } + + def withRecipient(ref: InternalActorRef): InboundEnvelope = { + _recipient = OptionVal(ref) + this + } + + def clear(): Unit = { + _recipient = OptionVal.None + _recipientAddress = null + _message = null + _sender = OptionVal.None + _originUid = 0L + _association = OptionVal.None + } + + def init( + recipient: OptionVal[InternalActorRef], + recipientAddress: Address, + message: AnyRef, + sender: OptionVal[ActorRef], + originUid: Long, + association: OptionVal[OutboundContext]): InboundEnvelope = { + _recipient = recipient + _recipientAddress = recipientAddress + _message = message + _sender = sender + _originUid = originUid + _association = association + this + } + + override def toString: String = + s"InboundEnvelope($recipient, $recipientAddress, $message, $sender, $originUid, $association)" +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/ObjectPool.scala b/akka-remote/src/main/scala/akka/remote/artery/ObjectPool.scala index 0020d52481..90f6f74d24 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ObjectPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ObjectPool.scala @@ -17,7 +17,7 @@ private[remote] class ObjectPool[A <: AnyRef](capacity: Int, create: () ⇒ A, c else obj } - def release(obj: A) = { + 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..6a1e9ba84c --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/OutboundEnvelope.scala @@ -0,0 +1,86 @@ +/** + * 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 +import akka.remote.RemoteActorRef + +/** + * INTERNAL API + */ +private[akka] 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[akka] trait OutboundEnvelope { + def recipient: OptionVal[RemoteActorRef] + def message: AnyRef + def sender: OptionVal[ActorRef] + + def withMessage(message: AnyRef): OutboundEnvelope + + def copy(): OutboundEnvelope +} + +/** + * INTERNAL API + */ +private[akka] object ReusableOutboundEnvelope { + def createObjectPool(capacity: Int) = new ObjectPool[ReusableOutboundEnvelope]( + capacity, + create = () ⇒ new ReusableOutboundEnvelope, clear = outEnvelope ⇒ outEnvelope.clear()) +} + +/** + * INTERNAL API + */ +private[akka] 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/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala index 320c00603b..a78dc3d18b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -10,7 +10,6 @@ import scala.util.Failure import scala.util.Success import scala.util.Try import akka.Done -import akka.remote.EndpointManager.Send import akka.remote.UniqueAddress import akka.remote.artery.InboundControlJunction.ControlMessageObserver import akka.stream.Attributes @@ -48,21 +47,21 @@ private[akka] class SystemMessageDelivery( deadLetters: ActorRef, resendInterval: FiniteDuration, maxBufferSize: Int) - extends GraphStage[FlowShape[Send, Send]] { + extends GraphStage[FlowShape[OutboundEnvelope, OutboundEnvelope]] { import SystemMessageDelivery._ - val in: Inlet[Send] = Inlet("SystemMessageDelivery.in") - val out: Outlet[Send] = Outlet("SystemMessageDelivery.out") - override val shape: FlowShape[Send, Send] = FlowShape(in, out) + 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[Send] - private var resending = new ArrayDeque[Send] + private val unacknowledged = new ArrayDeque[OutboundEnvelope] + private var resending = new ArrayDeque[OutboundEnvelope] private var resendingFromSeqNo = -1L private var stopping = false @@ -156,43 +155,49 @@ private[akka] class SystemMessageDelivery( private def tryResend(): Unit = { if (isAvailable(out) && !resending.isEmpty) - push(out, resending.poll()) + 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 = { - grab(in) match { - case s @ Send(_: HandshakeReq, _, _, _) ⇒ + val outboundEnvelope = grab(in) + outboundEnvelope.message match { + case _: HandshakeReq ⇒ // pass on HandshakeReq if (isAvailable(out)) - push(out, s) - case s @ Send(ClearSystemMessageDelivery, _, _, _) ⇒ + pushCopy(outboundEnvelope) + case ClearSystemMessageDelivery ⇒ clear() pull(in) - case s @ Send(msg: ControlMessage, _, _, _) ⇒ + case _: ControlMessage ⇒ // e.g. ActorSystemTerminating, no need for acked delivery if (resending.isEmpty && isAvailable(out)) - push(out, s) + pushCopy(outboundEnvelope) else { - resending.offer(s) + resending.offer(outboundEnvelope) tryResend() } - case s @ Send(msg: AnyRef, _, _, _) ⇒ + case msg ⇒ if (unacknowledged.size < maxBufferSize) { seqNo += 1 - val sendMsg = s.copy(message = SystemMessageEnvelope(msg, seqNo, localAddress)) - unacknowledged.offer(sendMsg) + val sendEnvelope = outboundEnvelope.withMessage(SystemMessageEnvelope(msg, seqNo, localAddress)) + unacknowledged.offer(sendEnvelope) scheduleOnce(ResendTick, resendInterval) if (resending.isEmpty && isAvailable(out)) - push(out, sendMsg) + pushCopy(sendEnvelope) else { - resending.offer(sendMsg) + resending.offer(sendEnvelope) tryResend() } } else { // buffer overflow outboundContext.quarantine(reason = s"System message delivery buffer overflow, size [$maxBufferSize]") - deadLetters ! s + deadLetters ! outboundEnvelope pull(in) } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala b/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala index e9fba1ffdd..b354f32bda 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala @@ -9,7 +9,6 @@ import scala.concurrent.Promise import scala.concurrent.duration._ import akka.Done import akka.actor.Address -import akka.remote.EndpointManager.Send import akka.remote.transport.ThrottlerTransportAdapter.Blackhole import akka.remote.transport.ThrottlerTransportAdapter.Direction import akka.remote.transport.ThrottlerTransportAdapter.SetThrottle @@ -59,10 +58,10 @@ private[remote] final case class TestManagementMessage(command: Any, done: Promi * INTERNAL API */ private[remote] class OutboundTestStage(outboundContext: OutboundContext) - extends GraphStageWithMaterializedValue[FlowShape[Send, Send], TestManagementApi] { - val in: Inlet[Send] = Inlet("OutboundTestStage.in") - val out: Outlet[Send] = Outlet("OutboundTestStage.out") - override val shape: FlowShape[Send, Send] = FlowShape(in, out) + extends GraphStageWithMaterializedValue[FlowShape[OutboundEnvelope, OutboundEnvelope], TestManagementApi] { + 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 createLogicAndMaterializedValue(inheritedAttributes: Attributes) = { val stoppedPromise = Promise[Done]() 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 index 5464d71f72..75b7ab0f08 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala @@ -27,8 +27,7 @@ private[remote] final class InboundActorRefCompression( settings: CompressionSettings, originUid: Long, inboundContext: InboundContext, - heavyHitters: TopHeavyHitters[ActorRef] -) extends InboundCompression[ActorRef](system, settings, originUid, inboundContext, heavyHitters, _.path.toSerializationFormat) { + heavyHitters: TopHeavyHitters[ActorRef]) extends InboundCompression[ActorRef](system, settings, originUid, inboundContext, heavyHitters, _.path.toSerializationFormat) { preAllocate(system.deadLetters) @@ -55,8 +54,7 @@ final class InboundManifestCompression( settings: CompressionSettings, originUid: Long, inboundContext: InboundContext, - heavyHitters: TopHeavyHitters[String] -) extends InboundCompression[String](system, settings, originUid, inboundContext, heavyHitters, ConstantFun.scalaIdentityFunction) { + heavyHitters: TopHeavyHitters[String]) extends InboundCompression[String](system, settings, originUid, inboundContext, heavyHitters, ConstantFun.scalaIdentityFunction) { scheduleNextTableAdvertisement() override protected def tableAdvertisementInterval = settings.manifests.advertisementInterval @@ -144,7 +142,7 @@ private[remote] abstract class InboundCompression[T >: Null]( } else { val count = cms.addAndEstimateCount(key, n) - // TODO optimise order of these, what is more expensive? + // TODO optimise order of these, what is more expensive? // TODO (now the `previous` is, but if aprox datatype there it would be faster)... Needs pondering. val wasHeavyHitter = addAndCheckIfheavyHitterDetected(value, count) if (wasHeavyHitter) @@ -160,7 +158,7 @@ private[remote] abstract class InboundCompression[T >: Null]( key match { case null ⇒ true case "" ⇒ true // empty class manifest for example - case _ ⇒ key.endsWith("/system/dummy") || key.endsWith("/") // TODO dummy likely shouldn't exist? can we remove it? + case _ ⇒ key.endsWith("/") } } diff --git a/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala index c6249a383e..e99219bad7 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala @@ -370,7 +370,7 @@ class FlightRecorderSpec extends AkkaSpec { } - def withFlightRecorder(body: (FlightRecorder, FlightRecorderReader, FileChannel) ⇒ Unit): Unit = { + private def withFlightRecorder(body: (FlightRecorder, FlightRecorderReader, FileChannel) ⇒ Unit): Unit = { val file = File.createTempFile("artery", ".afr") file.deleteOnExit() diff --git a/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala index 7b79fdaced..c4b5569a0e 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala @@ -33,17 +33,18 @@ class OutboundControlJunctionSpec extends AkkaSpec with ImplicitSender { val addressA = UniqueAddress(Address("artery", "sysA", "hostA", 1001), 1) val addressB = UniqueAddress(Address("artery", "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 destination = null.asInstanceOf[RemoteActorRef] // not used val ((upstream, controlIngress), downstream) = TestSource.probe[String] - .map(msg ⇒ Send(msg, OptionVal.None, destination, None)) - .viaMat(new OutboundControlJunction(outboundContext))(Keep.both) - .map { case Send(msg, _, _, _) ⇒ msg } + .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() diff --git a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala index 3dd7227cb6..6066d042fc 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala @@ -31,16 +31,17 @@ class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { val addressA = UniqueAddress(Address("artery", "sysA", "hostA", 1001), 1) val addressB = UniqueAddress(Address("artery", "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]) = { - val destination = null.asInstanceOf[RemoteActorRef] // not used TestSource.probe[String] - .map(msg ⇒ Send(msg, OptionVal.None, destination, None)) - .via(new OutboundHandshake(system, outboundContext, timeout, retryInterval, injectHandshakeInterval)) - .map { case Send(msg, _, _, _) ⇒ msg } + .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() } diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index 2ebedb00c3..624acf337a 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -16,7 +16,6 @@ import akka.actor.InternalActorRef import akka.actor.PoisonPill import akka.actor.RootActorPath import akka.remote.{ AddressUidExtension, RARP, RemoteActorRef, UniqueAddress } -import akka.remote.EndpointManager.Send import akka.remote.artery.SystemMessageDelivery._ import akka.stream.ActorMaterializer import akka.stream.ActorMaterializerSettings @@ -62,41 +61,46 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi 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[Send, NotUsed] = { - val remoteRef = null.asInstanceOf[RemoteActorRef] // not used + private def send(sendCount: Int, resendInterval: FiniteDuration, outboundContext: OutboundContext): Source[OutboundEnvelope, NotUsed] = { val deadLetters = TestProbe().ref Source(1 to sendCount) - .map(n ⇒ Send("msg-" + n, OptionVal.None, remoteRef, None)) + .map(n ⇒ outboundEnvelopePool.acquire().init(OptionVal.None, "msg-" + n, OptionVal.None)) .via(new SystemMessageDelivery(outboundContext, deadLetters, resendInterval, maxBufferSize = 1000)) } - private def inbound(inboundContext: InboundContext): Flow[Send, InboundEnvelope, NotUsed] = { + private def inbound(inboundContext: InboundContext): Flow[OutboundEnvelope, InboundEnvelope, NotUsed] = { val recipient = OptionVal.None // not used - Flow[Send] - .map { - case Send(sysEnv: SystemMessageEnvelope, _, _, _) ⇒ + Flow[OutboundEnvelope] + .map(outboundEnvelope ⇒ outboundEnvelope.message match { + case sysEnv: SystemMessageEnvelope ⇒ InboundEnvelope(recipient, addressB.address, sysEnv, OptionVal.None, addressA.uid, inboundContext.association(addressA.uid)) - } + }) .async .via(new SystemMessageAcker(inboundContext)) } - private def drop(dropSeqNumbers: Vector[Long]): Flow[Send, Send, NotUsed] = { - Flow[Send] + private def drop(dropSeqNumbers: Vector[Long]): Flow[OutboundEnvelope, OutboundEnvelope, NotUsed] = { + Flow[OutboundEnvelope] .statefulMapConcat(() ⇒ { var dropping = dropSeqNumbers { - case s @ Send(SystemMessageEnvelope(_, seqNo, _), _, _, _) ⇒ - val i = dropping.indexOf(seqNo) - if (i >= 0) { - dropping = dropping.updated(i, -1L) - Nil - } else - List(s) + 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 + } } }) } diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala index 1d378a6ea9..41c12280f8 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -89,9 +89,6 @@ private[akka] class TestOutboundContext( OptionVal.None)) } - // FIXME we should be able to Send without a recipient ActorRef - override def dummyRecipient: RemoteActorRef = null - } private[akka] class TestControlMessageSubject extends ControlMessageSubject { From d2c3c3ee3b5e622261d61e767ff006fb564c4a93 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 1 Jul 2016 15:38:39 +0200 Subject: [PATCH 086/186] update to Aeron 0.9.9 --- .../artery/compress/InvertCompressionTableBenchmark.scala | 2 +- project/Dependencies.scala | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) 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 index bc6cf2eba1..3f3930232e 100644 --- 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 @@ -17,7 +17,7 @@ class InvertCompressionTableBenchmark { 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(2, Map(Vector.fill[String](256)(randomName).zipWithIndex: _*)) val compTable_1024 = CompressionTable(3, Map(Vector.fill[String](1024)(randomName).zipWithIndex: _*)) diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 8162630707..d17976814d 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -78,8 +78,8 @@ 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" % "0.9.7" // ApacheV2 - val aeronClient = "io.aeron" % "aeron-client" % "0.9.7" // ApacheV2 + val aeronDriver = "io.aeron" % "aeron-driver" % "0.9.9" // ApacheV2 + val aeronClient = "io.aeron" % "aeron-client" % "0.9.9" // ApacheV2 object Docs { val sprayJson = "io.spray" %% "spray-json" % "1.3.2" % "test" @@ -106,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 From d1015c1dc622ba7a3ce43c5ebb43f75a4faf524c Mon Sep 17 00:00:00 2001 From: Konrad Malawski Date: Mon, 4 Jul 2016 16:42:14 +0200 Subject: [PATCH 087/186] Compression tables properly *used* for Outgoing Compression (#20874) * =art now correctly compresses and 2 table mode working * =art AGRESSIVELY optimising hashing, not convienved about correctness yet * fix HandshakeShouldDropCompressionTableSpec --- .../akka/remote/artery/CodecBenchmark.scala | 6 +- .../remote/artery/MaxThroughputSpec.scala | 29 ++-- .../artery/compress/CountMinSketch.java | 106 +++++++------ .../akka/remote/artery/ArteryTransport.scala | 57 +++++-- .../akka/remote/artery/Association.scala | 44 ++++-- .../scala/akka/remote/artery/BufferPool.scala | 60 ++----- .../scala/akka/remote/artery/Codecs.scala | 28 ++-- .../remote/artery/NoLiteralCompression.scala | 11 +- .../artery/compress/ActualCompressions.scala | 47 +++--- .../compress/AdvertiseCompressionId.scala | 11 -- .../artery/compress/AllCompressions.scala | 37 +++++ .../artery/compress/CompressionTable.scala | 2 +- .../artery/compress/DecompressionTable.scala | 14 +- .../artery/compress/InboundCompressions.scala | 148 +++++++----------- .../OutboundActorRefCompression.scala | 38 +++-- .../remote/artery/EnvelopeBufferSpec.scala | 8 +- .../remote/artery/FlightRecorderSpec.scala | 4 + ...dshakeShouldDropCompressionTableSpec.scala | 76 ++++++--- 18 files changed, 407 insertions(+), 319 deletions(-) delete mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/AdvertiseCompressionId.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/AllCompressions.scala 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 index e7e4233284..f08d5f87af 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -58,7 +58,7 @@ class CodecBenchmark { private val outboundEnvelopePool = ReusableOutboundEnvelope.createObjectPool(capacity = 16) val compressionOut = NoOutboundCompressions - val headerIn = HeaderBuilder.in(NoopInboundCompressions) + val headerIn = HeaderBuilder.in(NoInboundCompressions) val envelopeTemplateBuffer = ByteBuffer.allocate(ArteryTransport.MaximumFrameSize).order(ByteOrder.LITTLE_ENDIAN) val uniqueLocalAddress = UniqueAddress( @@ -166,7 +166,7 @@ class CodecBenchmark { val decoder: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = Flow.fromGraph(new Decoder(inboundContext, system.asInstanceOf[ExtendedActorSystem], - resolveActorRefWithLocalAddress, NoopInboundCompressions, envelopePool, inboundEnvelopePool)) + resolveActorRefWithLocalAddress, NoInboundCompressions, envelopePool, inboundEnvelopePool)) Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) .map { _ => @@ -207,7 +207,7 @@ class CodecBenchmark { val decoder: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = Flow.fromGraph(new Decoder(inboundContext, system.asInstanceOf[ExtendedActorSystem], - resolveActorRefWithLocalAddress, NoopInboundCompressions, envelopePool, inboundEnvelopePool)) + resolveActorRefWithLocalAddress, NoInboundCompressions, envelopePool, inboundEnvelopePool)) Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) .map(msg ⇒ outboundEnvelopePool.acquire().init(OptionVal.None, payload, OptionVal.Some(remoteRefB))) 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 index 2bdedd2681..3dfee899bd 100644 --- 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 @@ -6,9 +6,11 @@ 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.RemoteActorRefProvider +import akka.remote.artery.compress.CompressionSettings import akka.remote.testconductor.RoleName import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeSpec @@ -54,12 +56,14 @@ object MaxThroughputSpec extends MultiNodeConfig { # See akka-remote-tests/src/test/resources/aeron.properties #advanced.embedded-media-driver = off #advanced.aeron-dir = "target/aeron" - - #advanced.compression { - # enabled = on - # actor-refs.enabled = on - # manifests.enabled = on - #} + + advanced.compression { + enabled = off + actor-refs { + enabled = on + advertisement-interval = 1 second + } + } } } """))) @@ -137,11 +141,14 @@ object MaxThroughputSpec extends MultiNodeConfig { println( s"=== MaxThroughput ${self.path.name}: " + f"throughput ${throughput * testSettings.senderReceiverPairs}%,.0f msg/s, " + - f"${throughput * payloadSize * testSettings.senderReceiverPairs}%,.0f bytes/s, " + + f"${throughput * payloadSize * testSettings.senderReceiverPairs}%,.0f bytes/s (payload), " + + f"${throughput * totalSize(context.system) * testSettings.senderReceiverPairs}%,.0f bytes/s (total" + + (if (CompressionSettings(context.system).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") plotRef ! PlotResult().add(testName, throughput * payloadSize * testSettings.senderReceiverPairs / 1024 / 1024) context.stop(self) @@ -171,7 +178,10 @@ object MaxThroughputSpec extends MultiNodeConfig { totalMessages: Long, burstSize: Int, payloadSize: Int, - senderReceiverPairs: Int) + senderReceiverPairs: Int) { + // data based on measurement + def totalSize(system: ActorSystem) = payloadSize + (if (CompressionSettings(system).enabled) 38 else 110) + } class TestSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest with ByteBufferSerializer { @@ -270,7 +280,8 @@ abstract class MaxThroughputSpec totalMessages = adjustedTotalMessages(20000), burstSize = 200, // don't exceed the send queue capacity 200*5*3=3000 payloadSize = 100, - senderReceiverPairs = 5)) + senderReceiverPairs = 5) + ) def test(testSettings: TestSettings): Unit = { import testSettings._ 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 index 37b2baffad..6041cb8c78 100644 --- a/akka-remote/src/main/java/akka/remote/artery/compress/CountMinSketch.java +++ b/akka-remote/src/main/java/akka/remote/artery/compress/CountMinSketch.java @@ -4,15 +4,15 @@ package akka.remote.artery.compress; -import akka.actor.Actor; import akka.actor.ActorRef; -import java.io.UnsupportedEncodingException; import java.util.Random; /** * 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 * @@ -31,14 +31,19 @@ public class CountMinSketch { private double eps; private double confidence; + private int[] recyclableCMSHashBuckets; + + public CountMinSketch(int depth, int width, int seed) { 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); } + @SuppressWarnings("unused") public CountMinSketch(double epsOfTotalCount, double confidence, int seed) { // 2/w = eps ; w = 2/eps // 1/2^depth <= 1-confidence ; depth >= -log2 (1-confidence) @@ -46,10 +51,12 @@ public class CountMinSketch { this.confidence = confidence; this.width = (int) Math.ceil(2 / epsOfTotalCount); this.depth = (int) Math.ceil(-Math.log(1 - confidence) / Math.log(2)); + recyclableCMSHashBuckets = preallocateHashBucketsArray(depth); initTablesWith(depth, width, seed); } - CountMinSketch(int depth, int width, int size, long[] hashA, long[][] table) { + @SuppressWarnings("unused") + public CountMinSketch(int depth, int width, int size, long[] hashA, long[][] table) { this.depth = depth; this.width = width; this.eps = 2.0 / width; @@ -57,6 +64,7 @@ public class CountMinSketch { this.hashA = hashA; this.table = table; this.size = size; + recyclableCMSHashBuckets = preallocateHashBucketsArray(depth); } private void initTablesWith(int depth, int width, int seed) { @@ -75,11 +83,11 @@ public class CountMinSketch { } /** Referred to as {@code epsilon} in the whitepaper */ - public double getRelativeError() { + public double relativeError() { return eps; } - public double getConfidence() { + public double confidence() { return confidence; } @@ -108,7 +116,7 @@ public class CountMinSketch { size += count; } - public void add(String item, long count) { + public void addObject(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. @@ -116,19 +124,18 @@ public class CountMinSketch { // parameter to constructor. throw new IllegalArgumentException("Negative increments not implemented"); } - // TODO we could reuse the arrays - final int[] buckets = MurmurHash.hashBuckets(item, depth, width); // TODO replace with Scala's Murmur3, it's much faster + MurmurHash.hashBuckets(item, recyclableCMSHashBuckets, width); for (int i = 0; i < depth; ++i) { - table[i][buckets[i]] += count; + table[i][recyclableCMSHashBuckets[i]] += count; } size += count; } - + /** * 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 addAndEstimateCount(String item, long count) { + 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. @@ -136,14 +143,14 @@ public class CountMinSketch { // parameter to constructor. throw new IllegalArgumentException("Negative increments not implemented"); } - final int[] buckets = MurmurHash.hashBuckets(item, depth, width); + MurmurHash.hashBuckets(item, recyclableCMSHashBuckets, width); for (int i = 0; i < depth; ++i) { - table[i][buckets[i]] += count; + table[i][recyclableCMSHashBuckets[i]] += count; } size += count; - return estimateCount(buckets); + return estimateCount(recyclableCMSHashBuckets); } - + public long size() { return size; } @@ -160,15 +167,6 @@ public class CountMinSketch { return res; } - /** - * The estimate is correct within {@code 'epsilon' * (total item count)}, - * with probability {@code confidence}. - */ - public long estimateCount(String item) { - int[] buckets = MurmurHash.hashBuckets(item, depth, width); - return estimateCount(buckets); - } - /** * The estimate is correct within {@code 'epsilon' * (total item count)}, * with probability {@code confidence}. @@ -198,7 +196,6 @@ public class CountMinSketch { // TODO replace with Scala's Murmur3, it's much faster private static class MurmurHash { - // FIXME: This overload isn't actually ever used public static int hash(Object o) { if (o == null) { return 0; @@ -288,10 +285,14 @@ public class CountMinSketch { } public static int hashLong(long data) { + return hashLong(data, 0); + } + public static int hashLong(long data, int seed) { int m = 0x5bd1e995; int r = 24; - int h = 0; + int h = seed; + // int h = seed ^ length; int k = (int) data * m; k ^= k >>> r; @@ -314,28 +315,41 @@ public class CountMinSketch { // http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf // does prove to work in actual tests, and is obviously faster // than performing further iterations of murmur. - public static int[] hashBuckets(String key, int hashCount, int max) { - byte[] b; - try { - b = key.getBytes("UTF-16");// TODO Use the Unsafe trick @patriknw used to access the backing array directly -- via Endre - } catch (UnsupportedEncodingException e) { - throw new RuntimeException(e); - } - return hashBuckets(b, hashCount, max); - } - - static int[] hashBuckets(byte[] b, int hashCount, int max) { - // TODO we could reuse the arrays - int[] result = new int[hashCount]; - int hash1 = hash(b, b.length, 0); - int hash2 = hash(b, b.length, hash1); - for (int i = 0; i < hashCount; i++) { - result[i] = Math.abs((hash1 + i * hash2) % max); - } - return result; +// public static int[] hashBuckets(String key, int hashCount, int max) { +// byte[] b; +// try { +// b = key.getBytes("UTF-16");// TODO Use the Unsafe trick @patriknw used to access the backing array directly -- via Endre +// } catch (UnsupportedEncodingException e) { +// throw new RuntimeException(e); +// } +// return hashBuckets(b, hashCount, max); +// } + +// static int[] hashBuckets(byte[] b, int hashCount, int max) { +// // TODO we could reuse the arrays +// int[] result = preallocateHashBucketsArray(hashCount); +// int hash1 = hash(b, b.length, 0); +// int hash2 = hash(b, b.length, hash1); +// for (int i = 0; i < hashCount; i++) { +// result[i] = Math.abs((hash1 + i * hash2) % max); +// } +// return result; +// } + + /** Mutates passed in {@code hashBuckets} */ + static void hashBuckets(Object item, int[] hashBuckets, int max) { + int hash1 = hash(item); // specialized hash for ActorRef and Strings + int hash2 = hashLong(hash1, hash1); + final int depth = hashBuckets.length; + for (int i = 0; i < depth; i++) + hashBuckets[i] = Math.abs((hash1 + i * hash2) % max); } } + public int[] preallocateHashBucketsArray(int depth) { + return new int[depth]; + } + @Override public String toString() { return "CountMinSketch{" + diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 5daec306a9..42dcb5bad5 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -5,9 +5,9 @@ package akka.remote.artery import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.TimeUnit + import akka.remote.artery.compress.CompressionProtocol.CompressionMessage -import scala.collection.JavaConverters._ import scala.concurrent.Future import scala.concurrent.Promise import scala.concurrent.duration._ @@ -33,7 +33,7 @@ import akka.remote.artery.InboundControlJunction.ControlMessageObserver import akka.remote.artery.InboundControlJunction.ControlMessageSubject import akka.remote.transport.AkkaPduCodec import akka.remote.transport.AkkaPduProtobufCodec -import akka.remote.artery.compress.{ InboundCompressionsImpl, CompressionProtocol } +import akka.remote.artery.compress._ import akka.stream.AbruptTerminationException import akka.stream.ActorMaterializer import akka.stream.KillSwitches @@ -57,16 +57,20 @@ import org.agrona.IoUtil import java.io.File import java.net.InetSocketAddress import java.nio.channels.{ DatagramChannel, FileChannel } + import akka.remote.artery.OutboundControlJunction.OutboundControlIngress import io.aeron.CncFileDescriptor import java.util.concurrent.atomic.AtomicLong + import scala.collection.JavaConverters._ import akka.stream.ActorMaterializerSettings + import scala.annotation.tailrec import akka.util.OptionVal import io.aeron.driver.ThreadingMode import org.agrona.concurrent.BackoffIdleStrategy import org.agrona.concurrent.BusySpinIdleStrategy + import scala.util.control.NonFatal import akka.actor.Props import akka.actor.Actor @@ -113,7 +117,7 @@ private[akka] object AssociationState { incarnation = 1, uniqueRemoteAddressPromise = Promise(), quarantined = ImmutableLongMap.empty[QuarantinedTimestamp], - outboundCompression = NoOutboundCompressions) + outboundCompressions = NoOutboundCompressions) final case class QuarantinedTimestamp(nanoTime: Long) { override def toString: String = @@ -128,7 +132,7 @@ private[akka] final class AssociationState( val incarnation: Int, val uniqueRemoteAddressPromise: Promise[UniqueAddress], val quarantined: ImmutableLongMap[AssociationState.QuarantinedTimestamp], - val outboundCompression: OutboundCompressions) { + val outboundCompressions: OutboundCompressions) { import AssociationState.QuarantinedTimestamp @@ -154,6 +158,9 @@ private[akka] final class AssociationState( } } + def withCompression(compression: OutboundCompressions) = + new AssociationState(incarnation, uniqueRemoteAddressPromise, quarantined, compression) + def newIncarnation(remoteAddressPromise: Promise[UniqueAddress], compression: OutboundCompressions): AssociationState = new AssociationState(incarnation + 1, remoteAddressPromise, quarantined, compression) @@ -164,7 +171,7 @@ private[akka] final class AssociationState( incarnation, uniqueRemoteAddressPromise, quarantined = quarantined.updated(a.uid, QuarantinedTimestamp(System.nanoTime())), - outboundCompression = NoOutboundCompressions) // after quarantine no compression needed anymore, drop it + outboundCompressions = NoOutboundCompressions) // after quarantine no compression needed anymore, drop it case _ ⇒ this } @@ -283,6 +290,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R @volatile private[this] var aeron: Aeron = _ @volatile private[this] var aeronErrorLogTask: Cancellable = _ + // this is only used to allow triggering compression advertisements or state from tests + @volatile private[this] var activeCompressions = Set.empty[InboundCompressions] + override def localAddress: UniqueAddress = _localAddress override def defaultAddress: Address = localAddress.address override def addresses: Set[Address] = _addresses @@ -416,7 +426,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .threadingMode(ThreadingMode.DEDICATED) .conductorIdleStrategy(new BackoffIdleStrategy(1, 1, 1, 1)) .receiverIdleStrategy(new BusySpinIdleStrategy) - .senderIdleStrategy(new BusySpinIdleStrategy); + .senderIdleStrategy(new BusySpinIdleStrategy) } else if (remoteSettings.IdleCpuLevel == 1) { driverContext .threadingMode(ThreadingMode.SHARED) @@ -505,10 +515,10 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundStreams(): Unit = { - val noCompressions = new NoInboundCompressions(system) // TODO possibly enable on other channels too https://github.com/akka/akka/pull/20546/files#r68074082 + val noCompressions = NoInboundCompressions // TODO possibly enable on other channels too https://github.com/akka/akka/pull/20546/files#r68074082 val compressions = createInboundCompressions(this) - runInboundControlStream(noCompressions) + runInboundControlStream(noCompressions) // TODO should understand compressions too runInboundOrdinaryMessagesStream(compressions) if (largeMessageDestinationsEnabled) { runInboundLargeMessagesStream() @@ -542,12 +552,12 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R m match { case CompressionProtocol.ActorRefCompressionAdvertisement(from, table) ⇒ log.debug("Incoming ActorRef compression advertisement from [{}], table: [{}]", from, table) - association(from.address).compression.applyActorRefCompressionTable(table) + association(from.address).outboundCompression.applyActorRefCompressionTable(table) system.eventStream.publish(CompressionProtocol.Events.ReceivedCompressionTable(from, table)) case CompressionProtocol.ClassManifestCompressionAdvertisement(from, table) ⇒ log.debug("Incoming Class Manifest compression advertisement from [{}], table: [{}]", from, table) - association(from.address).compression.applyClassManifestCompressionTable(table) + association(from.address).outboundCompression.applyClassManifestCompressionTable(table) system.eventStream.publish(CompressionProtocol.Events.ReceivedCompressionTable(from, table)) } @@ -592,12 +602,12 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundLargeMessagesStream(): Unit = { - val compression = new NoInboundCompressions(system) // no compression on large message stream for now + val disableCompression = NoInboundCompressions // no compression on large message stream for now val completed = if (remoteSettings.TestMode) { val (mgmt, c) = aeronSource(largeStreamId, largeEnvelopePool) - .via(inboundLargeFlow(compression)) + .via(inboundLargeFlow(disableCompression)) .viaMat(inboundTestFlow)(Keep.right) .toMat(inboundSink)(Keep.both) .run()(materializer) @@ -605,7 +615,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R c } else { aeronSource(largeStreamId, largeEnvelopePool) - .via(inboundLargeFlow(compression)) + .via(inboundLargeFlow(disableCompression)) .toMat(inboundSink)(Keep.right) .run()(materializer) } @@ -758,8 +768,11 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R Flow.fromGraph(new Encoder(localAddress, system, compression, outboundEnvelopePool, bufferPool)) private def createInboundCompressions(inboundContext: InboundContext): InboundCompressions = - if (remoteSettings.ArteryCompressionSettings.enabled) new InboundCompressionsImpl(system, inboundContext) - else new NoInboundCompressions(system) + if (remoteSettings.ArteryCompressionSettings.enabled) { + val comp = new InboundCompressionsImpl(system, inboundContext) + activeCompressions += comp + comp + } else NoInboundCompressions def createEncoder(pool: EnvelopeBufferPool, compression: OutboundCompressions): Flow[OutboundEnvelope, EnvelopeBuffer, NotUsed] = Flow.fromGraph(new Encoder(localAddress, system, compression, outboundEnvelopePool, pool)) @@ -779,8 +792,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } def createDecoder(compression: InboundCompressions, bufferPool: EnvelopeBufferPool): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { - val resolveActorRefWithLocalAddress: String ⇒ InternalActorRef = + val resolveActorRefWithLocalAddress: String ⇒ InternalActorRef = { recipient ⇒ provider.resolveActorRefWithLocalAddress(recipient, localAddress.address) + } Flow.fromGraph(new Decoder(this, system, resolveActorRefWithLocalAddress, compression, bufferPool, inboundEnvelopePool)) } @@ -833,6 +847,17 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R def outboundTestFlow(association: Association): Flow[OutboundEnvelope, OutboundEnvelope, TestManagementApi] = Flow.fromGraph(new OutboundTestStage(association)) + /** INTERNAL API: for testing only. */ + private[remote] def triggerCompressionAdvertisements(actorRef: Boolean, manifest: Boolean) = { + activeCompressions.foreach { + case c: InboundCompressionsImpl if actorRef || manifest ⇒ + log.info("Triggering compression table advertisement for {}", c) + if (actorRef) c.runNextActorRefAdvertisement() + if (manifest) c.runNextClassManifestAdvertisement() + case _ ⇒ + } + } + } /** diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index affceabd77..fdbfca9747 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -9,7 +9,9 @@ import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.CountDownLatch import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicReference -import akka.remote.artery.compress.{ CompressionProtocol, CompressionTable, OutboundCompressionsImpl } + +import akka.remote.artery.compress.{ CompressionProtocol, CompressionTable, OutboundCompressions, OutboundCompressionsImpl } + import scala.annotation.tailrec import scala.concurrent.Future import scala.concurrent.Promise @@ -82,7 +84,7 @@ private[remote] class Association( // start sending (enqueuing) to the Association immediate after construction. /** Accesses the currently active outbound compression. */ - def compression: OutboundCompressions = associationState.outboundCompression + def outboundCompression: OutboundCompressions = associationState.outboundCompressions def createQueue(capacity: Int): Queue[OutboundEnvelope] = new ManyToOneConcurrentArrayQueue[OutboundEnvelope](capacity) @@ -144,9 +146,14 @@ private[remote] class Association( val current = associationState current.uniqueRemoteAddressPromise.trySuccess(peer) current.uniqueRemoteAddressValue() match { - case Some(`peer`) ⇒ // our value + case Some(`peer`) ⇒ + // our value + if (current.outboundCompressions == NoOutboundCompressions) { + // enable outbound compression (here, since earlier we don't know the remote address) + swapState(current, current.withCompression(createOutboundCompressions(remoteAddress))) + } case _ ⇒ - val newState = current.newIncarnation(Promise.successful(peer), createOutboundCompressionTable(remoteAddress)) + val newState = current.newIncarnation(Promise.successful(peer), createOutboundCompressions(remoteAddress)) if (swapState(current, newState)) { current.uniqueRemoteAddressValue() match { case Some(old) ⇒ @@ -407,11 +414,10 @@ private[remote] class Association( } // TODO: Make sure that once other channels use Compression, each gets it's own - private def createOutboundCompressionTable(remoteAddress: Address): OutboundCompressions = { + private def createOutboundCompressions(remoteAddress: Address): OutboundCompressions = { if (transport.provider.remoteSettings.ArteryCompressionSettings.enabled) { val compression = new OutboundCompressionsImpl(transport.system, remoteAddress) - // FIXME should use verion number of table instead of hashCode - log.info("Creating Outbound compression table ({}) to [{}]", compression.hashCode, remoteAddress) + log.debug("Creating Outbound compression table to [{}]", remoteAddress) compression } else NoOutboundCompressions } @@ -421,17 +427,25 @@ private[remote] class Association( * This way the same outgoing stream will switch to using the new table without the need of restarting it. */ private object CurrentAssociationStateOutboundCompressionsProxy extends OutboundCompressions { - override def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit = - associationState.outboundCompression.applyActorRefCompressionTable(table) + override def actorRefCompressionTableVersion: Int = + associationState.outboundCompressions.actorRefCompressionTableVersion + override def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit = { + associationState.outboundCompressions.applyActorRefCompressionTable(table) + } + override final def compressActorRef(ref: ActorRef): Int = { + associationState.outboundCompressions.compressActorRef(ref) + } + + override def classManifestCompressionTableVersion: Int = + associationState.outboundCompressions.classManifestCompressionTableVersion override def applyClassManifestCompressionTable(table: CompressionTable[String]): Unit = - associationState.outboundCompression.applyClassManifestCompressionTable(table) - - override final def compressActorRef(ref: ActorRef): Int = - associationState.outboundCompression.compressActorRef(ref) - + associationState.outboundCompressions.applyClassManifestCompressionTable(table) override final def compressClassManifest(manifest: String): Int = - associationState.outboundCompression.compressClassManifest(manifest) + associationState.outboundCompressions.compressClassManifest(manifest) + + override def toString = + s"${Logging.simpleName(getClass)}(current delegate: ${associationState.outboundCompressions})" } override def toString: String = diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index 77b865c4b4..d529c1d42e 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -9,7 +9,7 @@ import java.nio.{ ByteBuffer, ByteOrder } import akka.actor.{ ActorRef, Address } import akka.remote.artery.compress.CompressionProtocol._ -import akka.remote.artery.compress.{ CompressionTable, NoopInboundCompressions, NoopOutboundCompressions } +import akka.remote.artery.compress.{ CompressionTable, InboundCompressions, OutboundCompressions } import akka.serialization.Serialization import org.agrona.concurrent.{ ManyToManyConcurrentArrayQueue, UnsafeBuffer } import akka.util.{ OptionVal, Unsafe } @@ -54,6 +54,7 @@ private[remote] object EnvelopeBuffer { val VersionOffset = 0 // Int val UidOffset = 4 // Long val SerializerOffset = 12 // Int + val SenderActorRefTagOffset = 16 // Int val RecipientActorRefTagOffset = 20 // Int val ClassManifestTagOffset = 24 // Int @@ -70,41 +71,14 @@ private[remote] object EnvelopeBuffer { val StringValueFieldOffset = Unsafe.instance.objectFieldOffset(classOf[String].getDeclaredField("value")) } -/** - * 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, tableVersion: Int, remote: Address, ref: ActorRef): Unit - def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] - - def hitClassManifest(originUid: Long, tableVersion: Int, remote: Address, manifest: String): Unit - def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] -} -/** - * INTERNAL API - * Compress outgoing data and handle compression advertisements to fill compression table. - * - * One per outgoing message stream. - */ -private[remote] trait OutboundCompressions { - def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit - def compressActorRef(ref: ActorRef): Int - - def applyClassManifestCompressionTable(table: CompressionTable[String]): Unit - def compressClassManifest(manifest: String): Int -} - /** 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, NoopOutboundCompressions) - def out(compression: OutboundCompressions): HeaderBuilder = new HeaderBuilderImpl(NoopInboundCompressions, compression) + def in(compression: InboundCompressions): HeaderBuilder = new HeaderBuilderImpl(compression, NoOutboundCompressions) + def out(compression: OutboundCompressions): HeaderBuilder = new HeaderBuilderImpl(NoInboundCompressions, compression) /** INTERNAL API, FOR TESTING ONLY */ private[remote] def bothWays(in: InboundCompressions, out: OutboundCompressions): HeaderBuilder = new HeaderBuilderImpl(in, out) @@ -170,8 +144,8 @@ private[remote] final class HeaderBuilderImpl(inboundCompression: InboundCompres // Fields only available for EnvelopeBuffer var _version: Int = _ var _uid: Long = _ - var _actorRefCompressionTableVersion: Int = -1 - var _classManifestCompressionTableVersion: Int = -1 + var _actorRefCompressionTableVersion: Int = 0 + var _classManifestCompressionTableVersion: Int = 0 var _senderActorRef: String = null var _senderActorRefIdx: Int = -1 @@ -247,17 +221,17 @@ private[remote] final class HeaderBuilderImpl(inboundCompression: InboundCompres override def toString = "HeaderBuilderImpl(" + - version + ", " + - actorRefCompressionTableVersion + ", " + - classManifestCompressionTableVersion + ", " + - uid + ", " + - _senderActorRef + ", " + - _senderActorRefIdx + ", " + - _recipientActorRef + ", " + - _recipientActorRefIdx + ", " + - _serializer + ", " + - _manifest + ", " + - _manifestIdx + ")" + "version:" + version + ", " + + "actorRefCompressionTableVersion:" + actorRefCompressionTableVersion + ", " + + "classManifestCompressionTableVersion:" + classManifestCompressionTableVersion + ", " + + "uid:" + uid + ", " + + "_senderActorRef:" + _senderActorRef + ", " + + "_senderActorRefIdx:" + _senderActorRefIdx + ", " + + "_recipientActorRef:" + _recipientActorRef + ", " + + "_recipientActorRefIdx:" + _recipientActorRefIdx + ", " + + "_serializer:" + _serializer + ", " + + "_manifest:" + _manifest + ", " + + "_manifestIdx:" + _manifestIdx + ")" } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 8e0d5d70ef..426107b849 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -1,4 +1,3 @@ - /** * Copyright (C) 2016 Lightbend Inc. */ @@ -12,8 +11,9 @@ 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.util.{ ByteString, OptionVal, PrettyByteString } import akka.actor.EmptyLocalActorRef +import akka.remote.artery.compress.{ InboundCompressions, OutboundCompressions, OutboundCompressionsImpl } import akka.stream.stage.TimerGraphStageLogic /** @@ -47,6 +47,10 @@ private[remote] class Encoder( val outboundEnvelope = grab(in) val envelope = bufferPool.acquire() + // FIXME: OMG race between setting the version, and using the table!!!! + headerBuilder setActorRefCompressionTableVersion compression.actorRefCompressionTableVersion + headerBuilder setClassManifestCompressionTableVersion compression.classManifestCompressionTableVersion + // internally compression is applied by the builder: outboundEnvelope.recipient match { case OptionVal.Some(r) ⇒ headerBuilder setRecipientActorRef r @@ -147,24 +151,26 @@ private[remote] class Decoder( val recipient: OptionVal[InternalActorRef] = headerBuilder.recipientActorRef(originUid) match { case OptionVal.Some(ref) ⇒ OptionVal(ref.asInstanceOf[InternalActorRef]) - case OptionVal.None ⇒ - // `get` on Path is safe because it surely is not a compressed value here + case OptionVal.None if headerBuilder.recipientActorRefPath.isDefined ⇒ resolveRecipient(headerBuilder.recipientActorRefPath.get) + case _ ⇒ + OptionVal.None } - val sender: InternalActorRef = headerBuilder.senderActorRef(originUid) match { + val sender: OptionVal[InternalActorRef] = headerBuilder.senderActorRef(originUid) match { case OptionVal.Some(ref) ⇒ - ref.asInstanceOf[InternalActorRef] - case OptionVal.None ⇒ - // `get` on Path is safe because it surely is not a compressed value here - resolveActorRefWithLocalAddress(headerBuilder.senderActorRefPath.get) + OptionVal(ref.asInstanceOf[InternalActorRef]) + case OptionVal.None if headerBuilder.senderActorRefPath.isDefined ⇒ + OptionVal(resolveActorRefWithLocalAddress(headerBuilder.senderActorRefPath.get)) + case _ ⇒ + OptionVal.None } // --- hit refs and manifests for heavy-hitter counting association match { case OptionVal.Some(assoc) ⇒ val remoteAddress = assoc.remoteAddress - compression.hitActorRef(originUid, headerBuilder.actorRefCompressionTableVersion, remoteAddress, sender) + if (sender.isDefined) compression.hitActorRef(originUid, headerBuilder.actorRefCompressionTableVersion, remoteAddress, sender.get) if (recipient.isDefined) compression.hitActorRef(originUid, headerBuilder.actorRefCompressionTableVersion, remoteAddress, recipient.get) compression.hitClassManifest(originUid, headerBuilder.classManifestCompressionTableVersion, remoteAddress, headerBuilder.manifest(originUid)) case _ ⇒ @@ -181,7 +187,7 @@ private[remote] class Decoder( recipient, localAddress, // FIXME: Is this needed anymore? What should we do here? deserializedMessage, - OptionVal.Some(sender), // FIXME: No need for an option, decode simply to deadLetters instead + sender, // FIXME: No need for an option, decode simply to deadLetters instead originUid, association) diff --git a/akka-remote/src/main/scala/akka/remote/artery/NoLiteralCompression.scala b/akka-remote/src/main/scala/akka/remote/artery/NoLiteralCompression.scala index fe7502ea5e..03b1a4b6a6 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/NoLiteralCompression.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/NoLiteralCompression.scala @@ -3,8 +3,8 @@ */ package akka.remote.artery -import akka.actor.{ ActorRef, ActorSystem, Address, InternalActorRef } -import akka.remote.artery.compress.CompressionTable +import akka.actor.{ ActorRef, Address } +import akka.remote.artery.compress.{ CompressionTable, InboundCompressions, OutboundCompressions } import akka.util.OptionVal /** @@ -12,11 +12,10 @@ import akka.util.OptionVal * * Literarily, no compression! */ -final class NoInboundCompressions(system: ActorSystem) extends InboundCompressions { +case object NoInboundCompressions extends InboundCompressions { override def hitActorRef(originUid: Long, tableVersion: Int, remote: Address, ref: ActorRef): Unit = () override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = if (idx == -1) throw new IllegalArgumentException("Attemted decompression of illegal compression id: -1") - else if (idx == 0) OptionVal.Some(system.deadLetters) // special case deadLetters else OptionVal.None override def hitClassManifest(originUid: Long, tableVersion: Int, remote: Address, manifest: String): Unit = () @@ -30,10 +29,12 @@ final class NoInboundCompressions(system: ActorSystem) extends InboundCompressio * * Literarily, no compression! */ -object NoOutboundCompressions extends OutboundCompressions { +case object NoOutboundCompressions extends OutboundCompressions { override def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit = () + override def actorRefCompressionTableVersion: Int = 0 override def compressActorRef(ref: ActorRef): Int = -1 override def applyClassManifestCompressionTable(table: CompressionTable[String]): Unit = () + override def classManifestCompressionTableVersion: Int = 0 override def compressClassManifest(manifest: String): Int = -1 } diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressions.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressions.scala index 1580709a1b..04eb467c6a 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressions.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressions.scala @@ -9,24 +9,25 @@ import java.util.function.LongFunction import akka.actor.{ ActorRef, ActorSystem, Address } import akka.remote.artery._ import akka.util.OptionVal -import akka.remote.artery.OutboundCompressions import org.agrona.collections.Long2ObjectHashMap /** INTERNAL API */ private[remote] final class OutboundCompressionsImpl(system: ActorSystem, remoteAddress: Address) extends OutboundCompressions { private val actorRefsOut = new OutboundActorRefCompression(system, remoteAddress) - private val classManifestsOut = new OutboundCompressionTable[String](system, remoteAddress) + private val classManifestsOut = new OutboundClassManifestCompression(system, remoteAddress) // actor ref compression --- override def compressActorRef(ref: ActorRef): Int = actorRefsOut.compress(ref) + override def actorRefCompressionTableVersion: Int = actorRefsOut.activeCompressionTableVersion override def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit = actorRefsOut.flipTable(table) // class manifest compression --- override def compressClassManifest(manifest: String): Int = classManifestsOut.compress(manifest) + override def classManifestCompressionTableVersion: Int = classManifestsOut.activeCompressionTableVersion override def applyClassManifestCompressionTable(table: CompressionTable[String]): Unit = classManifestsOut.flipTable(table) } @@ -42,10 +43,9 @@ private[remote] final class InboundCompressionsImpl( ) extends InboundCompressions { private val settings = CompressionSettings(system) - private val localAddress = inboundContext.localAddress // FIXME we also must remove the ones that won't be used anymore - when quarantine triggers - private[this] val _actorRefsIn = new Long2ObjectHashMap[InboundActorRefCompression]() + private[this] val _actorRefsIns = new Long2ObjectHashMap[InboundActorRefCompression]() private val createInboundActorRefsForOrigin = new LongFunction[InboundActorRefCompression] { override def apply(originUid: Long): InboundActorRefCompression = { val actorRefHitters = new TopHeavyHitters[ActorRef](settings.actorRefs.max) @@ -53,9 +53,9 @@ private[remote] final class InboundCompressionsImpl( } } private def actorRefsIn(originUid: Long): InboundActorRefCompression = - _actorRefsIn.computeIfAbsent(originUid, createInboundActorRefsForOrigin) + _actorRefsIns.computeIfAbsent(originUid, createInboundActorRefsForOrigin) - private[this] val _classManifestsIn = new Long2ObjectHashMap[InboundManifestCompression]() + private[this] val _classManifestsIns = new Long2ObjectHashMap[InboundManifestCompression]() private val createInboundManifestsForOrigin = new LongFunction[InboundManifestCompression] { override def apply(originUid: Long): InboundManifestCompression = { val manifestHitters = new TopHeavyHitters[String](settings.manifests.max) @@ -63,12 +63,13 @@ private[remote] final class InboundCompressionsImpl( } } private def classManifestsIn(originUid: Long): InboundManifestCompression = - _classManifestsIn.computeIfAbsent(originUid, createInboundManifestsForOrigin) + _classManifestsIns.computeIfAbsent(originUid, createInboundManifestsForOrigin) // actor ref compression --- - override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = + override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = { actorRefsIn(originUid).decompress(tableVersion, idx) + } override def hitActorRef(originUid: Long, tableVersion: Int, address: Address, ref: ActorRef): Unit = { actorRefsIn(originUid).increment(address, ref, 1L) } @@ -80,20 +81,18 @@ private[remote] final class InboundCompressionsImpl( override def hitClassManifest(originUid: Long, tableVersion: Int, address: Address, manifest: String): Unit = { classManifestsIn(originUid).increment(address, manifest, 1L) } -} - -object NoopInboundCompressions extends InboundCompressions { - override def hitActorRef(originUid: Long, tableVersion: Int, remote: Address, ref: ActorRef): Unit = () - override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = OptionVal.None - - override def hitClassManifest(originUid: Long, tableVersion: Int, remote: Address, manifest: String): Unit = () - override def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] = OptionVal.None -} - -object NoopOutboundCompressions extends OutboundCompressions { - override def compressActorRef(ref: ActorRef): Int = -1 - override def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit = () - - override def compressClassManifest(manifest: String): Int = -1 - override def applyClassManifestCompressionTable(table: CompressionTable[String]): Unit = () + + // testing utilities --- + + /** INTERNAL API: for testing only */ + private[remote] def runNextActorRefAdvertisement() = { + import scala.collection.JavaConverters._ + _actorRefsIns.values().asScala.foreach { inbound ⇒ inbound.runNextTableAdvertisement() } + } + + /** INTERNAL API: for testing only */ + private[remote] def runNextClassManifestAdvertisement() = { + import scala.collection.JavaConverters._ + _classManifestsIns.values().asScala.foreach { inbound ⇒ inbound.runNextTableAdvertisement() } + } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/AdvertiseCompressionId.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/AdvertiseCompressionId.scala deleted file mode 100644 index 92ef0a5840..0000000000 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/AdvertiseCompressionId.scala +++ /dev/null @@ -1,11 +0,0 @@ -/* - * Copyright (C) 2016 Lightbend Inc. - */ -package akka.remote.artery.compress - -import akka.actor.Address - -/** Callback invoked when a compression id allocation should be advertised to the remote actor system. */ -trait AdvertiseCompressionId[T] { - def apply(remoteAddress: Address, ref: T, id: Int): Unit -} diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/AllCompressions.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/AllCompressions.scala new file mode 100644 index 0000000000..c0729636ed --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/AllCompressions.scala @@ -0,0 +1,37 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +import akka.actor.{ ActorRef, Address } +import akka.util.OptionVal + +/** + * 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, tableVersion: Int, remote: Address, ref: ActorRef): Unit + def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] + + def hitClassManifest(originUid: Long, tableVersion: Int, remote: Address, manifest: String): Unit + def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] +} +/** + * INTERNAL API + * Compress outgoing data and handle compression advertisements to fill compression table. + * + * One per outgoing message stream. + */ +private[remote] trait OutboundCompressions { + def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit + def actorRefCompressionTableVersion: Int + def compressActorRef(ref: ActorRef): Int + + def applyClassManifestCompressionTable(table: CompressionTable[String]): Unit + def classManifestCompressionTableVersion: Int + def compressClassManifest(manifest: String): Int +} 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 index 6125a2b3ea..dfc612868b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala @@ -5,7 +5,7 @@ package akka.remote.artery.compress /** INTERNAL API: Versioned compression table to be advertised between systems */ -private[akka] final case class CompressionTable[T](version: Long, map: Map[T, Int]) { +private[akka] final case class CompressionTable[T](version: Int, map: Map[T, Int]) { def invert: DecompressionTable[T] = if (map.isEmpty) DecompressionTable.empty[T].copy(version = version) 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 index 2434112982..4e5b74bf0b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala @@ -5,8 +5,16 @@ package akka.remote.artery.compress /** INTERNAL API */ -private[remote] final case class DecompressionTable[T](version: Long, table: Array[T]) { - def get(idx: Int): T = table(idx) +private[remote] final case class DecompressionTable[T](version: Int, table: Array[T]) { + // TODO version maybe better as Long? // OR implement roll-over + 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].") + table(idx) + } def invert: CompressionTable[T] = CompressionTable(version, Map(table.zipWithIndex: _*)) @@ -16,7 +24,7 @@ private[remote] final case class DecompressionTable[T](version: Long, table: Arr getClass.getName + s"(version: $version, " + ( - if (table.length == 0) "[empty]" + if (length == 0) "[empty]" else s"table: [${table.zipWithIndex.map({ case (t, i) ⇒ s"$i -> $t" }).mkString(",")}" ) + "])" } 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 index 75b7ab0f08..6ec6863585 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala @@ -4,12 +4,9 @@ package akka.remote.artery.compress -import java.util.concurrent.atomic.AtomicReference - import akka.actor.{ ActorRef, ActorSystem, Address } -import akka.event.Logging +import akka.event.{ Logging, NoLogging } import akka.remote.artery.{ InboundContext, OutboundContext } -import akka.stream.impl.ConstantFun import akka.util.{ OptionVal, PrettyDuration } import scala.concurrent.duration.{ Duration, FiniteDuration } @@ -27,7 +24,7 @@ private[remote] final class InboundActorRefCompression( settings: CompressionSettings, originUid: Long, inboundContext: InboundContext, - heavyHitters: TopHeavyHitters[ActorRef]) extends InboundCompression[ActorRef](system, settings, originUid, inboundContext, heavyHitters, _.path.toSerializationFormat) { + heavyHitters: TopHeavyHitters[ActorRef]) extends InboundCompression[ActorRef](system, settings, originUid, inboundContext, heavyHitters) { preAllocate(system.deadLetters) @@ -43,9 +40,9 @@ private[remote] final class InboundActorRefCompression( scheduleNextTableAdvertisement() override protected def tableAdvertisementInterval = settings.actorRefs.advertisementInterval - def advertiseCompressionTable(association: OutboundContext, table: CompressionTable[ActorRef]): Unit = { - log.debug(s"Advertise ActorRef compression [$table] to [${association.remoteAddress}]") - association.sendControl(CompressionProtocol.ActorRefCompressionAdvertisement(inboundContext.localAddress, table)) + override def advertiseCompressionTable(outboundContext: OutboundContext, table: CompressionTable[ActorRef]): Unit = { + log.debug(s"Advertise ActorRef compression [$table], from [${inboundContext.localAddress}] to [${outboundContext.remoteAddress}]") + outboundContext.sendControl(CompressionProtocol.ActorRefCompressionAdvertisement(inboundContext.localAddress, table)) } } @@ -54,14 +51,16 @@ final class InboundManifestCompression( settings: CompressionSettings, originUid: Long, inboundContext: InboundContext, - heavyHitters: TopHeavyHitters[String]) extends InboundCompression[String](system, settings, originUid, inboundContext, heavyHitters, ConstantFun.scalaIdentityFunction) { + heavyHitters: TopHeavyHitters[String]) extends InboundCompression[String](system, settings, originUid, inboundContext, heavyHitters) { scheduleNextTableAdvertisement() override protected def tableAdvertisementInterval = settings.manifests.advertisementInterval - override def advertiseCompressionTable(association: OutboundContext, table: CompressionTable[String]): Unit = { - log.debug(s"Advertise ClassManifest compression [$table] to [${association.remoteAddress}]") - association.sendControl(CompressionProtocol.ClassManifestCompressionAdvertisement(inboundContext.localAddress, table)) + override lazy val log = NoLogging + + override def advertiseCompressionTable(outboundContext: OutboundContext, table: CompressionTable[String]): Unit = { + log.debug(s"Advertise ClassManifest compression [$table] to [${outboundContext.remoteAddress}]") + outboundContext.sendControl(CompressionProtocol.ClassManifestCompressionAdvertisement(inboundContext.localAddress, table)) } } @@ -70,14 +69,13 @@ final class InboundManifestCompression( * 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: CompressionSettings, - originUid: Long, - inboundContext: InboundContext, - val heavyHitters: TopHeavyHitters[T], - convertKeyToString: T ⇒ String) { // TODO avoid converting to string, in order to use the ActorRef.hashCode! + val system: ActorSystem, + val settings: CompressionSettings, + originUid: Long, + inboundContext: InboundContext, + val heavyHitters: TopHeavyHitters[T]) { - val log = Logging(system, "InboundCompressionTable") + lazy val log = Logging(system, getClass.getSimpleName) // TODO atomic / state machine? the InbouncCompression could even extend ActomicReference[State]! @@ -91,7 +89,7 @@ private[remote] abstract class InboundCompression[T >: Null]( // 2 tables are used, one is "still in use", and the @volatile private[this] var activeTable = DecompressionTable.empty[T] - @volatile private[this] var nextTable = DecompressionTable.empty[T] + @volatile private[this] var nextTable = DecompressionTable.empty[T].copy(version = 1) // TODO calibrate properly (h/w have direct relation to preciseness and max capacity) private[this] val cms = new CountMinSketch(100, 100, System.currentTimeMillis().toInt) @@ -106,25 +104,28 @@ private[remote] abstract class InboundCompression[T >: Null]( * @throws UnknownCompressedIdException if given id is not known, this may indicate a bug – such situation should not happen. */ // not tailrec because we allow special casing in sub-class, however recursion is always at most 1 level deep - def decompress(tableVersion: Long, idx: Int): OptionVal[T] = { - val activeVersion = activeTable.version // TODO move into state + def decompress(incomingTableVersion: Long, idx: Int): OptionVal[T] = { + val activeVersion = activeTable.version - if (tableVersion == -1) OptionVal.None // no compression, bail out early - else if (tableVersion == activeVersion) { + if (incomingTableVersion == -1) OptionVal.None // no compression, bail out early + else if (incomingTableVersion == activeVersion) { val value: T = activeTable.get(idx) - if (settings.debug) log.debug(s"Decompress [{}] => {}", idx, value) if (value != null) OptionVal.Some[T](value) else throw new UnknownCompressedIdException(idx) - } else if (tableVersion < activeVersion) { - log.warning("Received value compressed with old table: [{}], current table version is: [{}]", tableVersion, activeVersion) + } else if (incomingTableVersion < activeVersion) { + log.warning("Received value compressed with old table: [{}], current table version is: [{}]", incomingTableVersion, activeVersion) OptionVal.None - } else if (tableVersion == nextTable.version) { - flipTables() - decompress(tableVersion, idx) // recurse, activeTable will not be able to handle this + } else if (incomingTableVersion == nextTable.version) { + advertisementInProgress = false + log.debug("Received first value compressed using the next prepared compression table, flipping to it (version: {})", nextTable.version) + startUsingNextTable() + decompress(incomingTableVersion, idx) // recurse, activeTable will not be able to handle this } else { // which means that incoming version was > nextTable.version, which likely is a bug - log.error("Inbound message is using compression table version higher than the highest allocated table on this node. " + - "This should not happen! State: activeTable: {}, nextTable, incoming tableVersion: {}", activeVersion, nextTable, tableVersion) + log.error( + "Inbound message is using compression table version higher than the highest allocated table on this node. " + + "This should not happen! State: activeTable: {}, nextTable: {}, incoming tableVersion: {}", + activeVersion, nextTable.version, incomingTableVersion) OptionVal.None } @@ -136,41 +137,11 @@ private[remote] abstract class InboundCompression[T >: Null]( */ // TODO not so happy about passing around address here, but in incoming there's no other earlier place to get it? def increment(remoteAddress: Address, value: T, n: Long): Unit = { - val key = convertKeyToString(value) - if (shouldIgnore(key)) { - // ignore... - } else { - val count = cms.addAndEstimateCount(key, n) + val count = cms.addObjectAndEstimateCount(value, n) - // TODO optimise order of these, what is more expensive? - // TODO (now the `previous` is, but if aprox datatype there it would be faster)... Needs pondering. - val wasHeavyHitter = addAndCheckIfheavyHitterDetected(value, count) - if (wasHeavyHitter) - log.debug(s"Heavy hitter detected: {} [count: {}]", value, count) - // if (wasHeavyHitter && !wasCompressedPreviously(key)) { - // val idx = prepareCompressionAdvertisement() - // log.debug("Allocated compression id [" + idx + "] for [" + value + "], in association with [" + remoteAddress + "]") - // } - } - } - - private def shouldIgnore(key: String) = { // TODO this is hacky, if we'd do this we trigger compression too early (before association exists, so control messages fail) - key match { - case null ⇒ true - case "" ⇒ true // empty class manifest for example - case _ ⇒ key.endsWith("/") - } - } - - // TODO this must be optimised, we really don't want to scan the entire key-set each time to make sure - private def wasCompressedPreviously(key: String): Boolean = { - var i = 0 - val len = activeTable.table.length - while (i < len) { - if (activeTable.table(i) == key) return true - i += 1 - } - false + // TODO optimise order of these, what is more expensive? + // TODO (now the `previous` is, but if aprox datatype there it would be faster)... Needs pondering. + addAndCheckIfheavyHitterDetected(value, count) } /** Mutates heavy hitters */ @@ -186,10 +157,8 @@ private[remote] abstract class InboundCompression[T >: Null]( * 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. - * - * TODO: Technically this would be solvable by a "triggerable" scheduler. */ - private[remote] def triggerNextTableAdvertisement(): Unit = // TODO expose and use in tests + private[remote] def triggerNextTableAdvertisement(): Unit = // TODO use this in tests for triggering runNextTableAdvertisement() def scheduleNextTableAdvertisement(): Unit = @@ -200,9 +169,9 @@ private[remote] abstract class InboundCompression[T >: Null]( log.debug("Scheduled {} advertisement in [{}] from now...", getClass.getSimpleName, PrettyDuration.format(tableAdvertisementInterval, includeNanos = false, 1)) } catch { case ex: IllegalStateException ⇒ - log.warning("Unable to schedule {} advertisement, " + - "likely system is shutting down. " + - "Reason: {}", getClass.getName, ex.getMessage) + // this is usually harmless + log.debug("Unable to schedule {} advertisement, " + + "likely system is shutting down. Reason: {}", getClass.getName, ex.getMessage) } case _ ⇒ // ignore... } @@ -213,6 +182,9 @@ private[remote] abstract class InboundCompression[T >: Null]( finally scheduleNextTableAdvertisement() } + // FIXME use AtomicBoolean instead? + @volatile private[this] var advertisementInProgress = false + /** * Entry point to advertising a new compression table. * @@ -223,19 +195,20 @@ private[remote] abstract class InboundCompression[T >: Null]( * 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. */ - def runNextTableAdvertisement() = { // TODO guard against re-entrancy? - inboundContext.association(originUid) match { - case OptionVal.Some(association) ⇒ - val table = prepareCompressionAdvertisement() - nextTable = table.invert // TODO expensive, check if building the other way wouldn't be faster? - advertiseCompressionTable(association, table) + private[remote] def runNextTableAdvertisement() = + if (!advertisementInProgress) + inboundContext.association(originUid) match { + case OptionVal.Some(association) ⇒ + advertisementInProgress = true + val table = prepareCompressionAdvertisement() + nextTable = table.invert // TODO expensive, check if building the other way wouldn't be faster? + advertiseCompressionTable(association, table) - 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.warning("No Association for originUid [{}] yet, unable to advertise compression table.", 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.warning("No Association for originUid [{}] yet, unable to advertise compression table.", originUid) + } /** * Must be implementeed by extending classes in order to send a [[akka.remote.artery.ControlMessage]] @@ -244,7 +217,7 @@ private[remote] abstract class InboundCompression[T >: Null]( protected def advertiseCompressionTable(association: OutboundContext, table: CompressionTable[T]): Unit /** Drop `activeTable` and start using the `nextTable` in its place. */ - private def flipTables(): Unit = { + private def startUsingNextTable(): Unit = { log.debug("Swaping active decompression table to version {}.", nextTable.version) activeTable = nextTable nextTable = DecompressionTable.empty @@ -261,11 +234,6 @@ private[remote] abstract class InboundCompression[T >: Null]( } -final class ExistingcompressedIdReuseAttemptException(id: Long, value: Any) - extends RuntimeException( - s"Attempted to re-allocate compressedId [$id] which is still in use for compressing [$value]! " + - s"This should never happen and is likely an implementation bug.") - final class UnknownCompressedIdException(id: Long) extends RuntimeException( s"Attempted de-compress unknown id [$id]! " + diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompression.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompression.scala index 03224b491c..c70dd95a54 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompression.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompression.scala @@ -8,7 +8,7 @@ import java.util.concurrent.atomic.AtomicReference import java.{ util ⇒ ju } import akka.actor.{ ActorRef, ActorSystem, Address } -import akka.event.Logging +import akka.event.{ Logging, LoggingAdapter } import akka.remote.artery.compress.OutboundCompression.OutboundCompressionState import scala.annotation.tailrec @@ -20,9 +20,14 @@ private[remote] final class OutboundActorRefCompression(system: ActorSystem, rem flipTable(CompressionTable( version = 0, map = Map( - system.deadLetters → 0 - ) - )) + system.deadLetters → 0))) +} + +/** INTERNAL API */ +private[remote] final class OutboundClassManifestCompression(system: ActorSystem, remoteAddress: Address) + extends OutboundCompressionTable[String](system, remoteAddress) { + + flipTable(CompressionTable(version = 0, Map.empty)) } /** @@ -34,9 +39,15 @@ private[remote] class OutboundCompressionTable[T](system: ActorSystem, remoteAdd extends AtomicReference[OutboundCompressionState[T]](OutboundCompressionState.initial) { // TODO could be instead via Unsafe import OutboundCompression._ - // TODO: The compression map may benefit from padding if we want multiple compressions to be running in parallel + // TODO: The compression map may benefit from padding if we want multiple compressions to be running in parallel - private[this] val log = Logging(system, "OutboundCompressionTable") + protected val log: LoggingAdapter = Logging(system, Logging.simpleName(getClass)) + + // TODO this exposes us to a race between setting the Version and USING the table...? + def activeCompressionTableVersion = { + val version = get.version + version + } /** * Flips the currently used compression table to the new one (iff the new one has a version number higher than the currently used one). @@ -44,16 +55,16 @@ private[remote] class OutboundCompressionTable[T](system: ActorSystem, remoteAdd // (╯°□°)╯︵ ┻━┻ @tailrec final def flipTable(activate: CompressionTable[T]): Unit = { val state = get() - if (state.version < activate.version) // TODO or we could demand it to be strictly `currentVersion + 1` + if (activate.version > state.version) // TODO this should handle roll-over as we move to Byte if (compareAndSet(state, prepareState(activate))) - log.debug("Successfully flipped compression table to version {}, for ourgoing connection to {}", activate.version, remoteAddress) + log.debug(s"Successfully flipped compression table versions {}=>{}, for outgoing to [{}]", state.version, activate.version, remoteAddress) else flipTable(activate) // retry else if (state.version == activate.version) log.warning("Received duplicate compression table (version: {})! Ignoring it.", state.version) else log.error("Received unexpected compression table with version nr [{}]! " + - "Current version number is []") + "Current version number is [{}].", activate.version, state.version) } @@ -66,11 +77,8 @@ private[remote] class OutboundCompressionTable[T](system: ActorSystem, remoteAdd // load factor is `1` since we will never grow this table beyond the initial size, // this way we can avoid any rehashing from happening. val m = new ju.HashMap[T, Integer](size, 1.0f) // TODO could be replaced with primitive `int` specialized version - val it = activate.map.keysIterator - var i = 0 - while (it.hasNext) { - m.put(it.next(), i) // TODO boxing :< - i += 1 + activate.map.foreach { + case (key, value) ⇒ m.put(key, value) // TODO boxing :< } OutboundCompressionState(activate.version, m) } @@ -98,7 +106,7 @@ private[remote] object OutboundCompression { // format: ON /** INTERNAL API */ - private[remote] final case class OutboundCompressionState[T](version: Long, table: ju.Map[T, Integer]) + private[remote] final case class OutboundCompressionState[T](version: Int, table: ju.Map[T, Integer]) private[remote] object OutboundCompressionState { def initial[T] = OutboundCompressionState[T](-1, ju.Collections.emptyMap()) } diff --git a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala index 3aa3c64773..cc013c3646 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala @@ -1,9 +1,13 @@ +/* + * 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 } +import akka.remote.artery.compress.{ CompressionTable, CompressionTestUtils, InboundCompressions, OutboundCompressions } import akka.testkit.AkkaSpec import akka.util.{ ByteString, OptionVal } @@ -28,11 +32,13 @@ class EnvelopeBufferSpec extends AkkaSpec { val idxToManifest = manifestToIdx.map(_.swap) override def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit = ??? // dynamic allocating not needed in these tests + override def actorRefCompressionTableVersion: Int = 0 override def compressActorRef(ref: ActorRef): Int = refToIdx.getOrElse(ref, -1) override def hitActorRef(originUid: Long, tableVersion: Int, remote: Address, ref: ActorRef): Unit = () override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = OptionVal(idxToRef(idx)) override def applyClassManifestCompressionTable(table: CompressionTable[String]): Unit = ??? // dynamic allocating not needed in these tests + override def classManifestCompressionTableVersion: Int = 0 override def compressClassManifest(manifest: String): Int = manifestToIdx.getOrElse(manifest, -1) override def hitClassManifest(originUid: Long, tableVersion: Int, remote: Address, manifest: String): Unit = () override def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] = OptionVal(idxToManifest(idx)) diff --git a/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala index e99219bad7..3dd3030fb6 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala @@ -1,3 +1,7 @@ +/* + * Copyright (C) 2009-2016 Lightbend Inc. + */ + package akka.remote.artery import java.io.{ File, IOException, RandomAccessFile } 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 index 673f70cbcc..efb1fa4ef9 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala @@ -9,6 +9,9 @@ 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, ReceivedCompressionTable } import com.typesafe.config.ConfigFactory import org.scalatest.BeforeAndAfter @@ -33,7 +36,8 @@ object HandshakeShouldDropCompressionTableSpec { enabled = on actor-refs { enabled = on - advertisement-interval = 3 seconds + # we'll trigger advertisement manually + advertisement-interval = 10 hours } } } @@ -57,59 +61,79 @@ class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDr } "Outgoing compression table" must { + // FIXME this is failing, we must rethink how tables are identified and updated "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[CompressionProtocol.Events.Event]) - systemB.eventStream.subscribe(b1Probe.ref, classOf[CompressionProtocol.Events.Event]) + system.eventStream.subscribe(aProbe.ref, classOf[Event]) + systemB.eventStream.subscribe(b1Probe.ref, classOf[Event]) - def voidSel = system.actorSelection(s"artery://systemB@localhost:$portB/user/void") - systemB.actorOf(TestActors.blackholeProps, "void") + def echoSel = system.actorSelection(s"artery://systemB@localhost:$portB/user/echo") + systemB.actorOf(TestActors.echoActorProps, "echo") // cause testActor-1 to become a heavy hitter - (1 to messagesToExchange).foreach { i ⇒ voidSel ! "hello" } // does not reply, but a hot receiver should be advertised - // give it enough time to advertise first table - val a0 = aProbe.expectMsgType[Events.ReceivedCompressionTable[ActorRef]](10.seconds) + (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[ReceivedCompressionTable[ActorRef]](10.seconds) info("System [A] received: " + a0) - assertCompression[ActorRef](a0.table, 1, _.toString should include(testActor.path.name)) + a0.table.map.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 ⇒ voidSel.tell("hello", a1Probe.ref) } // does not reply, but a hot receiver should be advertised - // give it enough time to advertise first table - val a1 = aProbe.expectMsgType[Events.ReceivedCompressionTable[ActorRef]](10.seconds) + (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[ReceivedCompressionTable[ActorRef]](10.seconds) info("System [A] received: " + a1) - assertCompression[ActorRef](a1.table, 2, _.toString should include(a1Probe.ref.path.name)) + a1.table.map.keySet should contain(a1Probe.ref) log.warning("SHUTTING DOWN system {}...", systemB) shutdown(systemB) systemB = ActorSystem("systemB", configB) - Thread.sleep(5000) + Thread.sleep(1000) log.warning("SYSTEM READY {}...", systemB) val aNewProbe = TestProbe() - system.eventStream.subscribe(aNewProbe.ref, classOf[CompressionProtocol.Events.Event]) + system.eventStream.subscribe(aNewProbe.ref, classOf[Event]) - systemB.actorOf(TestActors.blackholeProps, "void") // start it again - (1 to messagesToExchange).foreach { i ⇒ voidSel ! "hello" } // does not reply, but a hot receiver should be advertised - // compression triggered again - val a2 = aNewProbe.expectMsgType[Events.ReceivedCompressionTable[ActorRef]](10.seconds) + 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[ReceivedCompressionTable[ActorRef]](10.seconds) info("System [A] received: " + a2) - assertCompression[ActorRef](a2.table, 1, _.toString should include(testActor.path.name)) + a2.table.map.keySet should contain(testActor) val aNew2Probe = TestProbe() - (1 to messagesToExchange).foreach { i ⇒ voidSel.tell("hello", aNew2Probe.ref) } // does not reply, but a hot receiver should be advertised - // compression triggered again - val a3 = aNewProbe.expectMsgType[Events.ReceivedCompressionTable[ActorRef]](10.seconds) - info("Received second compression: " + a3) - assertCompression[ActorRef](a3.table, 2, _.toString should include(aNew2Probe.ref.path.name)) - } + (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[ReceivedCompressionTable[ActorRef]](10.seconds) + info("Received second compression: " + a3) + a3.table.map.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"artery://${_system}@localhost:$port/user/$name") From d2657a5969a4a86e4d11063f4844796bca3ab52e Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 4 Jul 2016 15:59:44 +0200 Subject: [PATCH 088/186] adaptive sampling of hit counting * when rate exceeds 1000 msg/s adaptive sampling of the heavy hitters tracking is enabled by sampling every 256th message * also fixed some bugs related to advertise in progress * update InboundCompression state atomically * enable compression in LatencySpec --- .../akka/remote/artery/LatencySpec.scala | 42 ++++-- .../remote/artery/MaxThroughputSpec.scala | 53 +++++-- .../scala/akka/remote/MessageSerializer.scala | 9 +- .../akka/remote/artery/ArteryTransport.scala | 36 +++-- .../scala/akka/remote/artery/BufferPool.scala | 6 +- .../scala/akka/remote/artery/Codecs.scala | 81 +++++++++-- .../remote/artery/NoLiteralCompression.scala | 6 +- .../artery/compress/ActualCompressions.scala | 28 ++-- .../artery/compress/AllCompressions.scala | 6 +- .../artery/compress/CompressionProtocol.scala | 27 +++- .../artery/compress/DecompressionTable.scala | 3 +- .../artery/compress/InboundCompressions.scala | 133 ++++++++++++------ .../OutboundActorRefCompression.scala | 2 +- .../remote/artery/EnvelopeBufferSpec.scala | 6 +- .../compress/CompressionIntegrationSpec.scala | 8 +- .../artery/compress/CompressionTestKit.scala | 4 +- ...dshakeShouldDropCompressionTableSpec.scala | 10 +- project/AkkaBuild.scala | 2 +- 18 files changed, 320 insertions(+), 142 deletions(-) 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 index a6030f71e1..7746658e89 100644 --- 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 @@ -54,6 +54,12 @@ object LatencySpec extends MultiNodeConfig { remote.artery { enabled = on advanced.idle-cpu-level=8 + + advanced.compression { + enabled = on + actor-refs.advertisement-interval = 2 second + manifests.advertisement-interval = 2 second + } } } """))) @@ -89,14 +95,17 @@ object LatencySpec extends MultiNodeConfig { def receive = { case bytes: Array[Byte] ⇒ - if (bytes.length != payloadSize) throw new IllegalArgumentException("Invalid message") - reporter.onMessage(1, payloadSize) - count += 1 - val d = System.nanoTime() - sendTimes.get(count - 1) - histogram.recordValue(d) - if (count == totalMessages) { - printTotal(testName, bytes.length, histogram) - context.stop(self) + // length 0 is used for warmup + if (bytes.length != 0) { + if (bytes.length != payloadSize) throw new IllegalArgumentException("Invalid message") + reporter.onMessage(1, payloadSize) + count += 1 + val d = System.nanoTime() - sendTimes.get(count - 1) + histogram.recordValue(d) + if (count == totalMessages) { + printTotal(testName, bytes.length, histogram) + context.stop(self) + } } } @@ -227,13 +236,22 @@ abstract class LatencySpec histogram.reset() val receiver = system.actorOf(receiverProps(rep, testSettings, totalMessages, sendTimes, histogram, plotProbe.ref)) - Source(1 to totalMessages) - .throttle(messageRate, 1.second, math.max(messageRate / 10, 1), ThrottleMode.Shaping) + // warmup for 3 seconds to init compression + val warmup = Source(1 to 30) + .throttle(10, 1.second, 10, ThrottleMode.Shaping) .runForeach { n ⇒ - sendTimes.set(n - 1, System.nanoTime()) - echo.tell(payload, receiver) + echo.tell(Array.emptyByteArray, receiver) } + warmup.foreach { _ ⇒ + Source(1 to totalMessages) + .throttle(messageRate, 1.second, math.max(messageRate / 10, 1), ThrottleMode.Shaping) + .runForeach { n ⇒ + sendTimes.set(n - 1, System.nanoTime()) + echo.tell(payload, receiver) + } + } + watch(receiver) expectTerminated(receiver, ((totalMessages / messageRate) + 10).seconds) val p = plotProbe.expectMsgType[LatencyPlots] 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 index 3dfee899bd..50eeb1035e 100644 --- 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 @@ -6,7 +6,6 @@ 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.RemoteActorRefProvider @@ -20,6 +19,8 @@ import akka.serialization.ByteBufferSerializer import akka.serialization.SerializerWithStringManifest import akka.testkit._ import com.typesafe.config.ConfigFactory +import akka.remote.artery.compress.CompressionProtocol.Events.ReceivedActorRefCompressionTable +import akka.remote.RARP object MaxThroughputSpec extends MultiNodeConfig { val first = role("first") @@ -32,7 +33,8 @@ object MaxThroughputSpec extends MultiNodeConfig { # for serious measurements you should increase the totalMessagesFactor (20) akka.test.MaxThroughputSpec.totalMessagesFactor = 1.0 akka { - loglevel = ERROR + loglevel = INFO + log-dead-letters = 1000000 # avoid TestEventListener loggers = ["akka.event.Logging$$DefaultLogger"] testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s @@ -58,11 +60,9 @@ object MaxThroughputSpec extends MultiNodeConfig { #advanced.aeron-dir = "target/aeron" advanced.compression { - enabled = off - actor-refs { - enabled = on - advertisement-interval = 1 second - } + enabled = on + actor-refs.advertisement-interval = 2 second + manifests.advertisement-interval = 2 second } } } @@ -108,13 +108,39 @@ object MaxThroughputSpec extends MultiNodeConfig { var remaining = totalMessages var maxRoundTripMillis = 0L + context.system.eventStream.subscribe(self, classOf[ReceivedActorRefCompressionTable]) + + val compressionEnabled = + RARP(context.system).provider.transport.isInstanceOf[ArteryTransport] && + RARP(context.system).provider.remoteSettings.ArteryCompressionSettings.enabled + def receive = { case Run ⇒ - // first some warmup - sendBatch() - // then Start, which will echo back here - target ! Start + 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.map.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") @@ -152,6 +178,8 @@ object MaxThroughputSpec extends MultiNodeConfig { s"$took ms to deliver $totalReceived messages") plotRef ! PlotResult().add(testName, throughput * payloadSize * testSettings.senderReceiverPairs / 1024 / 1024) context.stop(self) + + case c: ReceivedActorRefCompressionTable ⇒ } def sendBatch(): Unit = { @@ -280,8 +308,7 @@ abstract class MaxThroughputSpec totalMessages = adjustedTotalMessages(20000), burstSize = 200, // don't exceed the send queue capacity 200*5*3=3000 payloadSize = 100, - senderReceiverPairs = 5) - ) + senderReceiverPairs = 5)) def test(testSettings: TestSettings): Unit = { import testSettings._ diff --git a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala index 981b2b5bed..b09f9cbd01 100644 --- a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala @@ -71,11 +71,8 @@ private[akka] object MessageSerializer { } } - def deserializeForArtery(system: ExtendedActorSystem, originUid: Long, serialization: Serialization, headerBuilder: HeaderBuilder, - envelope: EnvelopeBuffer): AnyRef = { - serialization.deserializeByteBuffer( - envelope.byteBuffer, - headerBuilder.serializer, - headerBuilder.manifest(originUid)) // FIXME currently compression will not work for manifests + 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/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 42dcb5bad5..b94fbb38a1 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -290,8 +290,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R @volatile private[this] var aeron: Aeron = _ @volatile private[this] var aeronErrorLogTask: Cancellable = _ - // this is only used to allow triggering compression advertisements or state from tests - @volatile private[this] var activeCompressions = Set.empty[InboundCompressions] + @volatile private[this] var inboundCompressions: Option[InboundCompressions] = None override def localAddress: UniqueAddress = _localAddress override def defaultAddress: Address = localAddress.address @@ -517,6 +516,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R 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) @@ -547,18 +547,27 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R controlSubject.attach(new ControlMessageObserver { override def notify(inboundEnvelope: InboundEnvelope): Unit = { + inboundEnvelope.message match { case m: CompressionMessage ⇒ + import CompressionProtocol._ m match { - case CompressionProtocol.ActorRefCompressionAdvertisement(from, table) ⇒ + case ActorRefCompressionAdvertisement(from, table) ⇒ log.debug("Incoming ActorRef compression advertisement from [{}], table: [{}]", from, table) - association(from.address).outboundCompression.applyActorRefCompressionTable(table) - system.eventStream.publish(CompressionProtocol.Events.ReceivedCompressionTable(from, table)) - - case CompressionProtocol.ClassManifestCompressionAdvertisement(from, table) ⇒ + val a = association(from.address) + a.outboundCompression.applyActorRefCompressionTable(table) + a.sendControl(ActorRefCompressionAdvertisementAck(localAddress, table.version)) + system.eventStream.publish(Events.ReceivedActorRefCompressionTable(from, table)) + case ActorRefCompressionAdvertisementAck(from, tableVersion) ⇒ + inboundCompressions.foreach(_.confirmActorRefCompressionAdvertisement(from.uid, tableVersion)) + case ClassManifestCompressionAdvertisement(from, table) ⇒ log.debug("Incoming Class Manifest compression advertisement from [{}], table: [{}]", from, table) - association(from.address).outboundCompression.applyClassManifestCompressionTable(table) - system.eventStream.publish(CompressionProtocol.Events.ReceivedCompressionTable(from, table)) + val a = association(from.address) + a.outboundCompression.applyClassManifestCompressionTable(table) + a.sendControl(ClassManifestCompressionAdvertisementAck(localAddress, table.version)) + system.eventStream.publish(Events.ReceivedClassManifestCompressionTable(from, table)) + case ClassManifestCompressionAdvertisementAck(from, tableVersion) ⇒ + inboundCompressions.foreach(_.confirmActorRefCompressionAdvertisement(from.uid, tableVersion)) } case Quarantined(from, to) if to == localAddress ⇒ @@ -768,11 +777,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R Flow.fromGraph(new Encoder(localAddress, system, compression, outboundEnvelopePool, bufferPool)) private def createInboundCompressions(inboundContext: InboundContext): InboundCompressions = - if (remoteSettings.ArteryCompressionSettings.enabled) { - val comp = new InboundCompressionsImpl(system, inboundContext) - activeCompressions += comp - comp - } else NoInboundCompressions + if (remoteSettings.ArteryCompressionSettings.enabled) new InboundCompressionsImpl(system, inboundContext) + else NoInboundCompressions def createEncoder(pool: EnvelopeBufferPool, compression: OutboundCompressions): Flow[OutboundEnvelope, EnvelopeBuffer, NotUsed] = Flow.fromGraph(new Encoder(localAddress, system, compression, outboundEnvelopePool, pool)) @@ -849,7 +855,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R /** INTERNAL API: for testing only. */ private[remote] def triggerCompressionAdvertisements(actorRef: Boolean, manifest: Boolean) = { - activeCompressions.foreach { + inboundCompressions.foreach { case c: InboundCompressionsImpl if actorRef || manifest ⇒ log.info("Triggering compression table advertisement for {}", c) if (actorRef) c.runNextActorRefAdvertisement() diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index d529c1d42e..0b4bfbf558 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -75,7 +75,7 @@ private[remote] object EnvelopeBuffer { 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". + // we inject no-op compression's of the "other side". def in(compression: InboundCompressions): HeaderBuilder = new HeaderBuilderImpl(compression, NoOutboundCompressions) def out(compression: OutboundCompressions): HeaderBuilder = new HeaderBuilderImpl(NoInboundCompressions, compression) @@ -193,7 +193,9 @@ private[remote] final class HeaderBuilderImpl(inboundCompression: InboundCompres def setRecipientActorRef(ref: ActorRef): Unit = { _recipientActorRefIdx = outboundCompression.compressActorRef(ref) - if (_recipientActorRefIdx == -1) _recipientActorRef = ref.path.toSerializationFormat + if (_recipientActorRefIdx == -1) { + _recipientActorRef = ref.path.toSerializationFormat + } } def recipientActorRef(originUid: Long): OptionVal[ActorRef] = if (_recipientActorRef eq null) inboundCompression.decompressActorRef(originUid, actorRefCompressionTableVersion, _recipientActorRefIdx) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 426107b849..848369d2e4 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -15,6 +15,7 @@ import akka.util.{ ByteString, OptionVal, PrettyByteString } import akka.actor.EmptyLocalActorRef import akka.remote.artery.compress.{ InboundCompressions, OutboundCompressions, OutboundCompressionsImpl } import akka.stream.stage.TimerGraphStageLogic +import java.util.concurrent.TimeUnit /** * INTERNAL API @@ -48,6 +49,13 @@ private[remote] class Encoder( val envelope = bufferPool.acquire() // FIXME: OMG race between setting the version, and using the table!!!! + // incoming messages are concurrent to outgoing ones + // incoming message may be table advertisement + // which swaps the table in Outgoing*Compression for the new one (n+1) + // by itself it does so atomically, + // race: however here we store the compression table version separately from actually using it (storing the refs / manifests etc). + // so there is a slight race IF the table is swapped right between us setting the version n here [then the table being swapped to n+1] and then we use the n+1 version to compressions the compressions (which the receiving end will fail to read, since the encoding could be completely different, and it picks the table based on the version Int). + // A solution would be to getTable => use it to set and compress things headerBuilder setActorRefCompressionTableVersion compression.actorRefCompressionTableVersion headerBuilder setClassManifestCompressionTableVersion compression.classManifestCompressionTableVersion @@ -113,6 +121,8 @@ private[remote] object Decoder { attemptsLeft: Int, recipientPath: String, inboundEnvelope: InboundEnvelope) + + private object Tick } /** @@ -125,6 +135,7 @@ private[remote] class Decoder( compression: InboundCompressions, // TODO has to do demuxing on remote address It would seem, as decoder does not yet know 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) @@ -139,9 +150,22 @@ private[remote] class Decoder( private val retryResolveRemoteDeployedRecipientInterval = 50.millis private val retryResolveRemoteDeployedRecipientAttempts = 20 + // adaptive sampling when rate > 1000 msg/s + private var messageCount = 0L + private val HeavyHitterMask = (1 << 8) - 1 // sample every 256nth message + private var adaptiveSampling = false + 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) envelope.parseHeader(headerBuilder) @@ -166,22 +190,40 @@ private[remote] class Decoder( OptionVal.None } - // --- hit refs and manifests for heavy-hitter counting - association match { - case OptionVal.Some(assoc) ⇒ - val remoteAddress = assoc.remoteAddress - if (sender.isDefined) compression.hitActorRef(originUid, headerBuilder.actorRefCompressionTableVersion, remoteAddress, sender.get) - if (recipient.isDefined) compression.hitActorRef(originUid, headerBuilder.actorRefCompressionTableVersion, remoteAddress, recipient.get) - compression.hitClassManifest(originUid, headerBuilder.classManifestCompressionTableVersion, remoteAddress, headerBuilder.manifest(originUid)) - 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?") + val classManifest = headerBuilder.manifest(originUid) + + if (!adaptiveSampling || (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, headerBuilder.actorRefCompressionTableVersion, + remoteAddress, snd, 1) + case OptionVal.None ⇒ + } + + recipient match { + case OptionVal.Some(rcp) ⇒ + compression.hitActorRef(originUid, headerBuilder.actorRefCompressionTableVersion, + remoteAddress, rcp, 1) + case OptionVal.None ⇒ + } + + compression.hitClassManifest(originUid, headerBuilder.classManifestCompressionTableVersion, + 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 } - // --- end of hit refs and manifests for heavy-hitter counting try { val deserializedMessage = MessageSerializer.deserializeForArtery( - system, originUid, serialization, headerBuilder, envelope) + system, originUid, serialization, headerBuilder.serializer, classManifest, envelope) val decoded = inEnvelopePool.acquire().init( recipient, @@ -203,7 +245,7 @@ private[remote] class Decoder( case NonFatal(e) ⇒ log.warning( "Failed to deserialize message with serializer id [{}] and manifest [{}]. {}", - headerBuilder.serializer, headerBuilder.manifest(originUid), e.getMessage) + headerBuilder.serializer, classManifest, e.getMessage) pull(in) } finally { bufferPool.release(envelope) @@ -225,6 +267,19 @@ private[remote] class Decoder( override protected def onTimer(timerKey: Any): Unit = { timerKey match { + case Tick ⇒ + val now = System.nanoTime() + val d = now - tickTimestamp + val oldAdaptiveSampling = adaptiveSampling + adaptiveSampling = (d == 0 || + (messageCount - tickMessageCount) * TimeUnit.SECONDS.toNanos(1) / d > adaptiveSamplingRateThreshold) + if (!oldAdaptiveSampling && adaptiveSampling) + log.info("Turning on adaptive sampling ({}nth message) of compression hit counting", HeavyHitterMask + 1) + else if (oldAdaptiveSampling && !adaptiveSampling) + log.info("Turning off adaptive sampling of compression hit counting") + tickMessageCount = messageCount + tickTimestamp = now + case RetryResolveRemoteDeployedRecipient(attemptsLeft, recipientPath, inboundEnvelope) ⇒ resolveRecipient(recipientPath) match { case OptionVal.None ⇒ diff --git a/akka-remote/src/main/scala/akka/remote/artery/NoLiteralCompression.scala b/akka-remote/src/main/scala/akka/remote/artery/NoLiteralCompression.scala index 03b1a4b6a6..63886945d5 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/NoLiteralCompression.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/NoLiteralCompression.scala @@ -13,15 +13,17 @@ import akka.util.OptionVal * Literarily, no compression! */ case object NoInboundCompressions extends InboundCompressions { - override def hitActorRef(originUid: Long, tableVersion: Int, remote: Address, ref: ActorRef): Unit = () + override def hitActorRef(originUid: Long, tableVersion: Int, remote: Address, ref: ActorRef, n: Int): Unit = () override def decompressActorRef(originUid: Long, tableVersion: Int, 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: Int): Unit = () - override def hitClassManifest(originUid: Long, tableVersion: Int, remote: Address, manifest: String): Unit = () + override def hitClassManifest(originUid: Long, tableVersion: Int, remote: Address, manifest: String, n: Int): Unit = () override def decompressClassManifest(originUid: Long, tableVersion: Int, 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: Int): Unit = () } /** diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressions.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressions.scala index 04eb467c6a..fdbc067d61 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressions.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressions.scala @@ -17,14 +17,14 @@ private[remote] final class OutboundCompressionsImpl(system: ActorSystem, remote private val actorRefsOut = new OutboundActorRefCompression(system, remoteAddress) private val classManifestsOut = new OutboundClassManifestCompression(system, remoteAddress) - // actor ref compression --- + // actor ref compression --- override def compressActorRef(ref: ActorRef): Int = actorRefsOut.compress(ref) override def actorRefCompressionTableVersion: Int = actorRefsOut.activeCompressionTableVersion override def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit = actorRefsOut.flipTable(table) - // class manifest compression --- + // class manifest compression --- override def compressClassManifest(manifest: String): Int = classManifestsOut.compress(manifest) override def classManifestCompressionTableVersion: Int = classManifestsOut.activeCompressionTableVersion @@ -39,8 +39,7 @@ private[remote] final class OutboundCompressionsImpl(system: ActorSystem, remote */ private[remote] final class InboundCompressionsImpl( system: ActorSystem, - inboundContext: InboundContext -) extends InboundCompressions { + inboundContext: InboundContext) extends InboundCompressions { private val settings = CompressionSettings(system) @@ -65,22 +64,23 @@ private[remote] final class InboundCompressionsImpl( private def classManifestsIn(originUid: Long): InboundManifestCompression = _classManifestsIns.computeIfAbsent(originUid, createInboundManifestsForOrigin) - // actor ref compression --- + // actor ref compression --- - override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = { + override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = actorRefsIn(originUid).decompress(tableVersion, idx) - } - override def hitActorRef(originUid: Long, tableVersion: Int, address: Address, ref: ActorRef): Unit = { - actorRefsIn(originUid).increment(address, ref, 1L) - } + override def hitActorRef(originUid: Long, tableVersion: Int, address: Address, ref: ActorRef, n: Int): Unit = + actorRefsIn(originUid).increment(address, ref, n) + override def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = + actorRefsIn(originUid).confirmAdvertisement(tableVersion) - // class manifest compression --- + // class manifest compression --- override def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] = classManifestsIn(originUid).decompress(tableVersion, idx) - override def hitClassManifest(originUid: Long, tableVersion: Int, address: Address, manifest: String): Unit = { - classManifestsIn(originUid).increment(address, manifest, 1L) - } + override def hitClassManifest(originUid: Long, tableVersion: Int, address: Address, manifest: String, n: Int): Unit = + classManifestsIn(originUid).increment(address, manifest, n) + override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = + actorRefsIn(originUid).confirmAdvertisement(tableVersion) // testing utilities --- diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/AllCompressions.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/AllCompressions.scala index c0729636ed..e90db8bd31 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/AllCompressions.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/AllCompressions.scala @@ -14,11 +14,13 @@ import akka.util.OptionVal * One per inbound message stream thus must demux by originUid to use the right tables. */ private[remote] trait InboundCompressions { - def hitActorRef(originUid: Long, tableVersion: Int, remote: Address, ref: ActorRef): Unit + def hitActorRef(originUid: Long, tableVersion: Int, remote: Address, ref: ActorRef, n: Int): Unit def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] + def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit - def hitClassManifest(originUid: Long, tableVersion: Int, remote: Address, manifest: String): Unit + def hitClassManifest(originUid: Long, tableVersion: Int, remote: Address, manifest: String, n: Int): Unit def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] + def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit } /** * INTERNAL API 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 index 5cee5f77f3..6aeada84d5 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionProtocol.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionProtocol.scala @@ -9,7 +9,7 @@ import akka.actor.{ ActorRef, Address } import akka.remote.UniqueAddress import akka.remote.artery.ControlMessage -// FIXME serialization +// FIXME serialization /** INTERNAL API */ object CompressionProtocol { @@ -23,6 +23,16 @@ object CompressionProtocol { private[remote] final case class ActorRefCompressionAdvertisement(from: UniqueAddress, table: CompressionTable[ActorRef]) extends ControlMessage with CompressionMessage + /** + * 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: Int) + extends ControlMessage with CompressionMessage + /** * INTERNAL API * Sent by the "receiving" node after allocating a compression id to a given class manifest @@ -30,6 +40,16 @@ object CompressionProtocol { private[remote] final case class ClassManifestCompressionAdvertisement(from: UniqueAddress, table: CompressionTable[String]) extends ControlMessage with CompressionMessage + /** + * 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: Int) + extends ControlMessage with CompressionMessage + /** INTERNAL API */ private[akka] object Events { /** INTERNAL API */ @@ -39,7 +59,10 @@ object CompressionProtocol { final case class HeavyHitterDetected(key: Any, id: Int, count: Long) extends Event /** INTERNAL API */ - final case class ReceivedCompressionTable[T](from: UniqueAddress, table: CompressionTable[T]) extends Event + 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/DecompressionTable.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala index 4e5b74bf0b..e25ce85489 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala @@ -25,8 +25,7 @@ private[remote] final case class DecompressionTable[T](version: Int, table: Arra s"(version: $version, " + ( if (length == 0) "[empty]" - else s"table: [${table.zipWithIndex.map({ case (t, i) ⇒ s"$i -> $t" }).mkString(",")}" - ) + "])" + else s"table: [${table.zipWithIndex.map({ case (t, i) ⇒ s"$i -> $t" }).mkString(",")}") + "])" } /** INTERNAL API */ 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 index 6ec6863585..7fd5b5bfd4 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala @@ -8,8 +8,8 @@ import akka.actor.{ ActorRef, ActorSystem, Address } import akka.event.{ Logging, NoLogging } import akka.remote.artery.{ InboundContext, OutboundContext } import akka.util.{ OptionVal, PrettyDuration } - import scala.concurrent.duration.{ Duration, FiniteDuration } +import java.util.concurrent.atomic.AtomicReference /** * INTERNAL API @@ -33,9 +33,9 @@ private[remote] final class InboundActorRefCompression( allocations foreach { case ref ⇒ increment(null, ref, 100000) } } - override def decompress(tableId: Long, idx: Int): OptionVal[ActorRef] = + override def decompress(tableVersion: Int, idx: Int): OptionVal[ActorRef] = if (idx == 0) OptionVal.Some(system.deadLetters) - else super.decompress(tableId, idx) + else super.decompress(tableVersion, idx) scheduleNextTableAdvertisement() override protected def tableAdvertisementInterval = settings.actorRefs.advertisementInterval @@ -63,6 +63,34 @@ final class InboundManifestCompression( outboundContext.sendControl(CompressionProtocol.ClassManifestCompressionAdvertisement(inboundContext.localAddress, table)) } } +/** + * INTERNAL API + */ +private[remote] object InboundCompression { + + object State { + def empty[T] = State( + oldTable = DecompressionTable.empty[T].copy(version = -1), + 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] = + State( + oldTable = activeTable, + activeTable = nextTable, + nextTable = DecompressionTable.empty[T].copy(version = nextTable.version + 1), + advertisementInProgress = None) + } + +} /** * INTERNAL API @@ -77,19 +105,13 @@ private[remote] abstract class InboundCompression[T >: Null]( lazy val log = Logging(system, getClass.getSimpleName) - // TODO atomic / state machine? the InbouncCompression could even extend ActomicReference[State]! - // TODO NOTE: there exist edge cases around, we advertise table 1, accumulate table 2, the remote system has not used 2 yet, // yet we technically could already prepare table 3, then it starts using table 1 suddenly. Edge cases like that. // SOLUTION 1: We don't start building new tables until we've seen the previous one be used (move from new to active) // This is nice as it practically disables all the "build the table" work when the other side is not interested in using it. // SOLUTION 2: We end up dropping messages when old table comes in (we do that anyway) - // TODO have a marker that "advertised table XXX", so we don't generate a new-new one until the new one is in use? - - // 2 tables are used, one is "still in use", and the - @volatile private[this] var activeTable = DecompressionTable.empty[T] - @volatile private[this] var nextTable = DecompressionTable.empty[T].copy(version = 1) + private[this] val state: AtomicReference[InboundCompression.State[T]] = new AtomicReference(InboundCompression.State.empty) // TODO calibrate properly (h/w have direct relation to preciseness and max capacity) private[this] val cms = new CountMinSketch(100, 100, System.currentTimeMillis().toInt) @@ -104,30 +126,50 @@ private[remote] abstract class InboundCompression[T >: Null]( * @throws UnknownCompressedIdException if given id is not known, this may indicate a bug – such situation should not happen. */ // not tailrec because we allow special casing in sub-class, however recursion is always at most 1 level deep - def decompress(incomingTableVersion: Long, idx: Int): OptionVal[T] = { - val activeVersion = activeTable.version + def decompress(incomingTableVersion: Int, idx: Int): OptionVal[T] = { + val current = state.get + val oldVersion = current.oldTable.version + val activeVersion = current.activeTable.version if (incomingTableVersion == -1) OptionVal.None // no compression, bail out early else if (incomingTableVersion == activeVersion) { - val value: T = activeTable.get(idx) + 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 (incomingTableVersion < activeVersion) { log.warning("Received value compressed with old table: [{}], current table version is: [{}]", incomingTableVersion, activeVersion) OptionVal.None - } else if (incomingTableVersion == nextTable.version) { - advertisementInProgress = false - log.debug("Received first value compressed using the next prepared compression table, flipping to it (version: {})", nextTable.version) - startUsingNextTable() + } else if (incomingTableVersion == current.nextTable.version) { + log.debug( + "Received first value compressed using the next prepared compression table, flipping to it (version: {})", + current.nextTable.version) + confirmAdvertisement(incomingTableVersion) decompress(incomingTableVersion, idx) // recurse, activeTable will not be able to handle this } else { // which means that incoming version was > nextTable.version, which likely is a bug log.error( "Inbound message is using compression table version higher than the highest allocated table on this node. " + "This should not happen! State: activeTable: {}, nextTable: {}, incoming tableVersion: {}", - activeVersion, nextTable.version, incomingTableVersion) + activeVersion, current.nextTable.version, incomingTableVersion) OptionVal.None } + } + + def confirmAdvertisement(tableVersion: Int): 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 {}", tableVersion) + case Some(inProgress) if tableVersion != inProgress.version ⇒ + log.debug("Confirmed compression table version {} but in progress {}", tableVersion, inProgress.version) + case None ⇒ // already confirmed + } } @@ -182,9 +224,6 @@ private[remote] abstract class InboundCompression[T >: Null]( finally scheduleNextTableAdvertisement() } - // FIXME use AtomicBoolean instead? - @volatile private[this] var advertisementInProgress = false - /** * Entry point to advertising a new compression table. * @@ -195,20 +234,34 @@ private[remote] abstract class InboundCompression[T >: Null]( * 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 (!advertisementInProgress) - inboundContext.association(originUid) match { - case OptionVal.Some(association) ⇒ - advertisementInProgress = true - val table = prepareCompressionAdvertisement() - nextTable = table.invert // TODO expensive, check if building the other way wouldn't be faster? - advertiseCompressionTable(association, table) + private[remote] def runNextTableAdvertisement() = { + val current = state.get + current.advertisementInProgress match { + case None ⇒ + inboundContext.association(originUid) match { + case OptionVal.Some(association) ⇒ + 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)) + advertiseCompressionTable(association, table) - 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.warning("No Association for originUid [{}] yet, unable to advertise compression table.", 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.warning("No Association for originUid [{}] yet, unable to advertise compression table.", originUid) + } + + case Some(inProgress) ⇒ + // The ActorRefCompressionAdvertisement message is resent because it can be lost + log.debug("Advertisment in progress for version {}, resending", inProgress.version) + inboundContext.association(originUid) match { + case OptionVal.Some(association) ⇒ + advertiseCompressionTable(association, inProgress) // resend + case OptionVal.None ⇒ + } + } + } /** * Must be implementeed by extending classes in order to send a [[akka.remote.artery.ControlMessage]] @@ -216,17 +269,9 @@ private[remote] abstract class InboundCompression[T >: Null]( */ protected def advertiseCompressionTable(association: OutboundContext, table: CompressionTable[T]): Unit - /** Drop `activeTable` and start using the `nextTable` in its place. */ - private def startUsingNextTable(): Unit = { - log.debug("Swaping active decompression table to version {}.", nextTable.version) - activeTable = nextTable - nextTable = DecompressionTable.empty - // TODO we want to keep the currentTableVersion in State too, update here as well then - } - - private def prepareCompressionAdvertisement(): CompressionTable[T] = { + private def prepareCompressionAdvertisement(nextTableVersion: Int): CompressionTable[T] = { // TODO surely we can do better than that, optimise - CompressionTable(activeTable.version + 1, Map(heavyHitters.snapshot.filterNot(_ == null).zipWithIndex: _*)) + CompressionTable(nextTableVersion, Map(heavyHitters.snapshot.filterNot(_ == null).zipWithIndex: _*)) } override def toString = diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompression.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompression.scala index c70dd95a54..59756afe32 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompression.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompression.scala @@ -61,7 +61,7 @@ private[remote] class OutboundCompressionTable[T](system: ActorSystem, remoteAdd else flipTable(activate) // retry else if (state.version == activate.version) - log.warning("Received duplicate compression table (version: {})! Ignoring it.", state.version) + log.debug("Received duplicate compression table (version: {})! Ignoring it.", state.version) else log.error("Received unexpected compression table with version nr [{}]! " + "Current version number is [{}].", activate.version, state.version) diff --git a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala index cc013c3646..d9a3040a28 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala @@ -34,14 +34,16 @@ class EnvelopeBufferSpec extends AkkaSpec { override def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit = ??? // dynamic allocating not needed in these tests override def actorRefCompressionTableVersion: Int = 0 override def compressActorRef(ref: ActorRef): Int = refToIdx.getOrElse(ref, -1) - override def hitActorRef(originUid: Long, tableVersion: Int, remote: Address, ref: ActorRef): Unit = () + override def hitActorRef(originUid: Long, tableVersion: Int, remote: Address, ref: ActorRef, n: Int): Unit = () override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = OptionVal(idxToRef(idx)) + override def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = () override def applyClassManifestCompressionTable(table: CompressionTable[String]): Unit = ??? // dynamic allocating not needed in these tests override def classManifestCompressionTableVersion: Int = 0 override def compressClassManifest(manifest: String): Int = manifestToIdx.getOrElse(manifest, -1) - override def hitClassManifest(originUid: Long, tableVersion: Int, remote: Address, manifest: String): Unit = () + override def hitClassManifest(originUid: Long, tableVersion: Int, remote: Address, manifest: String, n: Int): Unit = () override def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] = OptionVal(idxToManifest(idx)) + override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = () } "EnvelopeBuffer" must { 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 index f312f118d1..9e3b602bd0 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala @@ -22,7 +22,7 @@ object CompressionIntegrationSpec { val commonConfig = ConfigFactory.parseString(s""" akka { loglevel = INFO - + actor.provider = "akka.remote.RemoteActorRefProvider" remote.artery.enabled = on remote.artery.advanced { @@ -32,7 +32,7 @@ object CompressionIntegrationSpec { remote.artery.hostname = localhost remote.artery.port = 0 remote.handshake-timeout = 10s - + remote.artery.advanced.compression { enabled = on actor-refs { @@ -40,7 +40,7 @@ object CompressionIntegrationSpec { advertisement-interval = 3 seconds } } - + } """) @@ -75,7 +75,7 @@ class CompressionIntegrationSpec extends AkkaSpec(CompressionIntegrationSpec.com // cause testActor-1 to become a heavy hitter (1 to messagesToExchange).foreach { i ⇒ voidSel ! "hello" } // does not reply, but a hot receiver should be advertised - val a1 = aProbe.expectMsgType[Events.ReceivedCompressionTable[ActorRef]](10.seconds) + val a1 = aProbe.expectMsgType[Events.ReceivedActorRefCompressionTable](10.seconds) info("System [A] received: " + a1) assertCompression[ActorRef](a1.table, 0, _ should ===(system.deadLetters)) assertCompression[ActorRef](a1.table, 1, _ should ===(testActor)) diff --git a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTestKit.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTestKit.scala index 9872374252..7bdd8973e8 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTestKit.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTestKit.scala @@ -5,7 +5,7 @@ package akka.remote.artery.compress /* INTERNAL API */ -private[remote] trait CompressionTestKit { +private[akka] trait CompressionTestKit { def assertCompression[T](table: CompressionTable[T], id: Int, assertion: T ⇒ Unit): Unit = { table.map.find(_._2 == id) .orElse { throw new AssertionError(s"No key was compressed to the id [$id]! Table was: $table") } @@ -14,4 +14,4 @@ private[remote] trait CompressionTestKit { } /* INTERNAL API */ -private[remote] object CompressionTestKit extends CompressionTestKit +private[akka] object CompressionTestKit extends CompressionTestKit 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 index efb1fa4ef9..4ee33cb9a0 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala @@ -11,7 +11,7 @@ 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, ReceivedCompressionTable } +import akka.remote.artery.compress.CompressionProtocol.Events.{ Event, ReceivedActorRefCompressionTable } import com.typesafe.config.ConfigFactory import org.scalatest.BeforeAndAfter @@ -82,7 +82,7 @@ class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDr waitForEcho(this, s"hello-$messagesToExchange") systemBTransport.triggerCompressionAdvertisements(actorRef = true, manifest = false) - val a0 = aProbe.expectMsgType[ReceivedCompressionTable[ActorRef]](10.seconds) + val a0 = aProbe.expectMsgType[ReceivedActorRefCompressionTable](10.seconds) info("System [A] received: " + a0) a0.table.map.keySet should contain(testActor) @@ -91,7 +91,7 @@ class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDr waitForEcho(a1Probe, s"hello-$messagesToExchange") systemBTransport.triggerCompressionAdvertisements(actorRef = true, manifest = false) - val a1 = aProbe.expectMsgType[ReceivedCompressionTable[ActorRef]](10.seconds) + val a1 = aProbe.expectMsgType[ReceivedActorRefCompressionTable](10.seconds) info("System [A] received: " + a1) a1.table.map.keySet should contain(a1Probe.ref) @@ -113,7 +113,7 @@ class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDr waitForEcho(this, s"hello-$messagesToExchange", max = 10.seconds) systemBTransport.triggerCompressionAdvertisements(actorRef = true, manifest = false) - val a2 = aNewProbe.expectMsgType[ReceivedCompressionTable[ActorRef]](10.seconds) + val a2 = aNewProbe.expectMsgType[ReceivedActorRefCompressionTable](10.seconds) info("System [A] received: " + a2) a2.table.map.keySet should contain(testActor) @@ -122,7 +122,7 @@ class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDr waitForEcho(aNew2Probe, s"hello-$messagesToExchange") systemBTransport.triggerCompressionAdvertisements(actorRef = true, manifest = false) - val a3 = aNewProbe.expectMsgType[ReceivedCompressionTable[ActorRef]](10.seconds) + val a3 = aNewProbe.expectMsgType[ReceivedActorRefCompressionTable](10.seconds) info("Received second compression: " + a3) a3.table.map.keySet should contain(aNew2Probe.ref) } diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index cd7ac5e57b..87856727ef 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -82,7 +82,7 @@ object AkkaBuild extends Build { protobuf, remote, remoteTests, - samples, +// samples, slf4j, stream, streamTestkit, From c376ac0c532fbfc9535a8069add0c4b3ee236489 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 15 Jun 2016 15:12:33 +0200 Subject: [PATCH 089/186] remove burstiness in latency tests * throttle generates bursts but for fair latency tests we want the messages to be spread uniformly * not much need for exploratory testing with AeronStreamsApp any longer, not worth to maintain it * make it possible to run MaxThroughputSpec with old remoting * add metrics for the task runner, with flight recorder * tune idle-cpu-level --- .../artery/AeronStreamLatencySpec.scala | 83 +++-- .../akka/remote/artery/LatencySpec.scala | 88 ++--- .../remote/artery/MaxThroughputSpec.scala | 42 ++- .../remote/artery/TaskRunnerMetrics.scala | 55 +++ .../scala/akka/remote/artery/AeronSink.scala | 23 +- .../akka/remote/artery/AeronSource.scala | 34 +- .../akka/remote/artery/ArteryTransport.scala | 22 +- .../remote/artery/FlightRecorderEvents.scala | 10 +- .../scala/akka/remote/artery/TaskRunner.scala | 39 ++- .../src/test/resources/aeron.properties | 5 + .../akka/remote/artery/AeronStreamsApp.scala | 315 ------------------ 11 files changed, 281 insertions(+), 435 deletions(-) create mode 100644 akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TaskRunnerMetrics.scala delete mode 100644 akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala 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 index 9dcf1c7167..240a1c1c2f 100644 --- 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 @@ -3,35 +3,38 @@ */ 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 java.util.concurrent.atomic.AtomicBoolean - -import akka.stream.KillSwitches -import akka.Done import org.agrona.IoUtil -import java.io.File -import java.io.File - -import akka.util.ByteString -import io.aeron.CncFileDescriptor +import org.agrona.concurrent.ManyToOneConcurrentArrayQueue object AeronStreamLatencySpec extends MultiNodeConfig { val first = role("first") @@ -135,7 +138,7 @@ abstract class AeronStreamLatencySpec super.afterAll() } - def printTotal(testName: String, payloadSize: Long, histogram: Histogram, lastRepeat: Boolean): Unit = { + 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 = @@ -144,10 +147,13 @@ abstract class AeronStreamLatencySpec 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"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) @@ -181,6 +187,11 @@ abstract class AeronStreamLatencySpec 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, @@ -200,6 +211,7 @@ abstract class AeronStreamLatencySpec 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() @@ -217,7 +229,8 @@ abstract class AeronStreamLatencySpec val d = System.nanoTime() - sendTimes.get(c - 1) histogram.recordValue(d) if (c == totalMessages) { - printTotal(testName, bytes.length, histogram, lastRepeat.get) + val totalDurationNanos = System.nanoTime() - startTime.get + printTotal(testName, bytes.length, histogram, totalDurationNanos, lastRepeat.get) barrier.await() // this is always the last party } } @@ -236,21 +249,53 @@ abstract class AeronStreamLatencySpec started.expectMsg(Done) } - for (n ← 1 to repeat) { + for (rep ← 1 to repeat) { histogram.reset() count.set(0) - lastRepeat.set(n == repeat) + lastRepeat.set(rep == repeat) - Source(1 to totalMessages) - .throttle(messageRate, 1.second, math.max(messageRate / 10, 1), ThrottleMode.Shaping) - .map { n ⇒ + val sendFlow = Flow[Unit] + .map { _ ⇒ val envelope = pool.acquire() envelope.byteBuffer.put(payload) envelope.byteBuffer.flip() - sendTimes.set(n - 1, System.nanoTime()) envelope } - .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) + + val queueValue = Source.fromGraph(new SendQueue[Unit]) + .via(sendFlow) + .to(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter, 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) } 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 index 7746658e89..185221267c 100644 --- 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 @@ -3,10 +3,9 @@ */ package akka.remote.artery -import java.net.InetAddress import java.util.concurrent.Executors -import scala.collection.AbstractIterator -import scala.concurrent.Await +import java.util.concurrent.atomic.AtomicLongArray +import java.util.concurrent.locks.LockSupport import scala.concurrent.duration._ import akka.actor._ import akka.remote.testconductor.RoleName @@ -14,21 +13,11 @@ 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 java.util.concurrent.CyclicBarrier -import java.util.concurrent.atomic.AtomicInteger -import java.util.concurrent.atomic.AtomicLongArray import org.HdrHistogram.Histogram +import akka.stream.scaladsl.Source import akka.stream.ThrottleMode -import java.io.StringWriter -import java.io.PrintStream -import java.io.OutputStreamWriter -import java.io.BufferedOutputStream -import java.io.ByteArrayOutputStream object LatencySpec extends MultiNodeConfig { val first = role("first") @@ -38,7 +27,7 @@ object LatencySpec extends MultiNodeConfig { commonConfig(debugConfig(on = false).withFallback( ConfigFactory.parseString(s""" - # for serious measurements you should increase the totalMessagesFactor (10) and repeatCount (3) + # for serious measurements you should increase the totalMessagesFactor (30) and repeatCount (3) akka.test.LatencySpec.totalMessagesFactor = 1.0 akka.test.LatencySpec.repeatCount = 1 akka { @@ -53,7 +42,7 @@ object LatencySpec extends MultiNodeConfig { } remote.artery { enabled = on - advanced.idle-cpu-level=8 + advanced.idle-cpu-level=7 advanced.compression { enabled = on @@ -92,24 +81,27 @@ object LatencySpec extends MultiNodeConfig { import settings._ var count = 0 + var startTime = System.nanoTime() + val taskRunnerMetrics = new TaskRunnerMetrics(context.system) def receive = { case bytes: Array[Byte] ⇒ - // length 0 is used for warmup if (bytes.length != 0) { + if (count == 0) + startTime = System.nanoTime() if (bytes.length != payloadSize) throw new IllegalArgumentException("Invalid message") reporter.onMessage(1, payloadSize) count += 1 val d = System.nanoTime() - sendTimes.get(count - 1) histogram.recordValue(d) if (count == totalMessages) { - printTotal(testName, bytes.length, histogram) + printTotal(testName, bytes.length, histogram, System.nanoTime() - startTime) context.stop(self) } } } - def printTotal(testName: String, payloadSize: Long, histogram: Histogram): Unit = { + 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 = @@ -118,13 +110,18 @@ object LatencySpec extends MultiNodeConfig { value.getPercentileLevelIteratedTo < (p + 0.5) ⇒ value.getValueIteratedTo / 1000.0 }.getOrElse(Double.NaN) + val throughput = 1000.0 * histogram.getTotalCount / math.min(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"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)), @@ -155,23 +152,11 @@ abstract class LatencySpec var plots = LatencyPlots() - val aeron = { - val ctx = new Aeron.Context - val driver = MediaDriver.launchEmbedded() - ctx.aeronDirectoryName(driver.aeronDirectoryName) - Aeron.connect(ctx) - } - 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}" - } - lazy val reporterExecutor = Executors.newFixedThreadPool(1) def reporter(name: String): TestRateReporter = { val r = new TestRateReporter(name) @@ -195,6 +180,11 @@ abstract class LatencySpec } val scenarios = List( + TestSettings( + testName = "warmup", + messageRate = 10000, + payloadSize = 100, + repeat = repeatCount), TestSettings( testName = "rate-100-size-100", messageRate = 100, @@ -210,6 +200,11 @@ abstract class LatencySpec 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, @@ -244,16 +239,33 @@ abstract class LatencySpec } warmup.foreach { _ ⇒ - Source(1 to totalMessages) - .throttle(messageRate, 1.second, math.max(messageRate / 10, 1), ThrottleMode.Shaping) - .runForeach { n ⇒ - sendTimes.set(n - 1, System.nanoTime()) - echo.tell(payload, receiver) + + 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) } + + echo.tell(payload, receiver) + i += 1 + } } watch(receiver) - expectTerminated(receiver, ((totalMessages / messageRate) + 10).seconds) + expectTerminated(receiver, ((totalMessages / messageRate) + 20).seconds) val p = plotProbe.expectMsgType[LatencyPlots] // only use the last repeat for the plots if (n == repeat) { 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 index 50eeb1035e..0ad6c1c2bb 100644 --- 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 @@ -75,11 +75,12 @@ object MaxThroughputSpec extends MultiNodeConfig { final case class EndResult(totalReceived: Long) final case class FlowControl(burstStartTime: Long) extends Echo - def receiverProps(reporter: RateReporter, payloadSize: Int): Props = - Props(new Receiver(reporter, payloadSize)).withDispatcher("akka.remote.default-remote-dispatcher") + 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) extends Actor { + 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] ⇒ @@ -90,6 +91,8 @@ object MaxThroughputSpec extends MultiNodeConfig { c = 0 sender() ! Start case End ⇒ + if (printTaskRunnerMetrics) + taskRunnerMetrics.printHistograms() sender() ! EndResult(c) context.stop(self) case m: Echo ⇒ @@ -98,15 +101,18 @@ object MaxThroughputSpec extends MultiNodeConfig { } } - def senderProps(target: ActorRef, testSettings: TestSettings, plotRef: ActorRef): Props = - Props(new Sender(target, testSettings, plotRef)) + 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) extends Actor { + 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]) @@ -176,6 +182,10 @@ object MaxThroughputSpec extends MultiNodeConfig { 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) @@ -232,8 +242,18 @@ object MaxThroughputSpec extends MultiNodeConfig { case FlowControlManifest ⇒ FlowControl(buf.getLong) } - override def toBinary(o: AnyRef): Array[Byte] = ??? - override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef = ??? + 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) } } @@ -319,7 +339,9 @@ abstract class MaxThroughputSpec runOn(second) { val rep = reporter(testName) for (n ← 1 to senderReceiverPairs) { - val receiver = system.actorOf(receiverProps(rep, payloadSize), receiverName + n) + val receiver = system.actorOf( + receiverProps(rep, payloadSize, printTaskRunnerMetrics = n == 1), + receiverName + n) } enterBarrier(receiverName + "-started") enterBarrier(testName + "-done") @@ -333,7 +355,7 @@ abstract class MaxThroughputSpec val receiver = identifyReceiver(receiverName + n) val plotProbe = TestProbe() val snd = system.actorOf( - senderProps(receiver, testSettings, plotProbe.ref), + senderProps(receiver, testSettings, plotProbe.ref, printTaskRunnerMetrics = n == 1), testName + "-snd" + n) val terminationProbe = TestProbe() terminationProbe.watch(snd) 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..f939669d81 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TaskRunnerMetrics.scala @@ -0,0 +1,55 @@ +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 _ ⇒ + } + } + }) + 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/src/main/scala/akka/remote/artery/AeronSink.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala index a580e39834..f93a6ab013 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala @@ -5,7 +5,6 @@ package akka.remote.artery import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicInteger - import scala.annotation.tailrec import scala.concurrent.Future import scala.concurrent.Promise @@ -14,7 +13,6 @@ 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 @@ -26,6 +24,7 @@ import akka.stream.stage.InHandler import io.aeron.Aeron import io.aeron.Publication import org.agrona.concurrent.UnsafeBuffer +import org.agrona.hints.ThreadHints object AeronSink { @@ -95,15 +94,17 @@ class AeronSink( private var completedValue: Try[Done] = Success(Done) - // FIXME measure and adjust with IdleCpuLevel - private val spinning = 1000 + // 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(_ ⇒ onOfferSuccess()), + private val offerTask = new OfferTask(pub, null, lastMsgSize, getAsyncCallback(_ ⇒ taskOnOfferSuccess()), giveUpSendAfter, getAsyncCallback(_ ⇒ onGiveUp())) 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") @@ -135,10 +136,10 @@ class AeronSink( @tailrec private def publish(): Unit = { val result = pub.offer(envelopeInFlight.aeronBuffer, 0, lastMsgSize) // FIXME handle Publication.CLOSED - // TODO the backoff strategy should be measured and tuned if (result < 0) { backoffCount -= 1 if (backoffCount > 0) { + ThreadHints.onSpinWait() publish() // recursive } else { // delegate backoff to shared TaskRunner @@ -146,14 +147,22 @@ class AeronSink( // 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, lastMsgSize) + flightRecorder.hiFreq(AeronSink_DelegateToTaskRunner, countBeforeDelegate) } } else { + countBeforeDelegate += 1 onOfferSuccess() } } + 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 diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala index 629ddf5b66..a6133d0104 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala @@ -4,10 +4,8 @@ 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 @@ -22,6 +20,7 @@ import io.aeron.logbuffer.FragmentHandler import io.aeron.logbuffer.Header import org.agrona.DirectBuffer import org.agrona.concurrent.BackoffIdleStrategy +import org.agrona.hints.ThreadHints object AeronSource { @@ -80,18 +79,15 @@ class AeronSource( new GraphStageLogic(shape) with OutHandler { private val sub = aeron.addSubscription(channel, streamId) - // FIXME measure and adjust with IdleCpuLevel - private val spinning = 1000 - private val yielding = 0 - private val parking = 0 - private val idleStrategy = new BackoffIdleStrategy( - spinning, yielding, TimeUnit.MICROSECONDS.toNanos(1), TimeUnit.MICROSECONDS.toNanos(100)) - private val idleStrategyRetries = spinning + yielding + parking - private var backoffCount = idleStrategyRetries + // 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(onMessage))) + private val addPollTask: Add = Add(pollTask(sub, messageHandler, getAsyncCallback(taskOnMessage))) private val channelMetadata = channel.getBytes("US-ASCII") @@ -107,8 +103,7 @@ class AeronSource( // OutHandler override def onPull(): Unit = { - idleStrategy.reset() - backoffCount = idleStrategyRetries + backoffCount = spinning subscriberLoop() } @@ -118,24 +113,31 @@ class AeronSource( 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 { - // TODO the backoff strategy should be measured and tuned backoffCount -= 1 if (backoffCount > 0) { - idleStrategy.idle() + ThreadHints.onSpinWait() subscriberLoop() // recursive } else { // delegate backoff to shared TaskRunner - flightRecorder.hiFreq(AeronSource_DelegateToTaskRunner, 0) + flightRecorder.hiFreq(AeronSource_DelegateToTaskRunner, countBeforeDelegate) + delegateTaskStartTime = System.nanoTime() taskRunner.command(addPollTask) } } } + private def taskOnMessage(data: EnvelopeBuffer): Unit = { + countBeforeDelegate = 0 + flightRecorder.hiFreq(AeronSource_ReturnFromTaskRunner, System.nanoTime() - delegateTaskStartTime) + onMessage(data) + } + private def onMessage(data: EnvelopeBuffer): Unit = { flightRecorder.hiFreq(AeronSource_Received, data.byteBuffer.limit) push(out, data) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index b94fbb38a1..92fa84e5da 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -6,6 +6,7 @@ package akka.remote.artery import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.TimeUnit +import java.util.concurrent.TimeUnit.MICROSECONDS import akka.remote.artery.compress.CompressionProtocol.CompressionMessage import scala.concurrent.Future @@ -415,25 +416,30 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R if (remoteSettings.AeronDirectoryName.nonEmpty) driverContext.aeronDirectoryName(remoteSettings.AeronDirectoryName) // FIXME settings from config - driverContext.conductorIdleStrategy() driverContext.clientLivenessTimeoutNs(SECONDS.toNanos(20)) driverContext.imageLivenessTimeoutNs(SECONDS.toNanos(20)) driverContext.driverTimeoutMs(SECONDS.toNanos(20)) - if (remoteSettings.IdleCpuLevel == 10) { + val idleCpuLevel = remoteSettings.IdleCpuLevel + if (idleCpuLevel == 10) { driverContext .threadingMode(ThreadingMode.DEDICATED) .conductorIdleStrategy(new BackoffIdleStrategy(1, 1, 1, 1)) - .receiverIdleStrategy(new BusySpinIdleStrategy) - .senderIdleStrategy(new BusySpinIdleStrategy) - } else if (remoteSettings.IdleCpuLevel == 1) { + .receiverIdleStrategy(TaskRunner.createIdleStrategy(idleCpuLevel)) + .senderIdleStrategy(TaskRunner.createIdleStrategy(idleCpuLevel)) + } else if (idleCpuLevel == 1) { driverContext .threadingMode(ThreadingMode.SHARED) - //FIXME measure: .sharedIdleStrategy(new BackoffIdleStrategy(20, 50, 1, 200)) - } else if (remoteSettings.IdleCpuLevel <= 5) { + .sharedIdleStrategy(TaskRunner.createIdleStrategy(idleCpuLevel)) + } else if (idleCpuLevel <= 7) { driverContext .threadingMode(ThreadingMode.SHARED_NETWORK) - //FIXME measure: .sharedNetworkIdleStrategy(new BackoffIdleStrategy(20, 50, 1, 20 * (11 - remoteSettings.IdleCpuLevel))) + .sharedNetworkIdleStrategy(TaskRunner.createIdleStrategy(idleCpuLevel)) + } else { + driverContext + .threadingMode(ThreadingMode.DEDICATED) + .receiverIdleStrategy(TaskRunner.createIdleStrategy(idleCpuLevel)) + .senderIdleStrategy(TaskRunner.createIdleStrategy(idleCpuLevel)) } val driver = MediaDriver.launchEmbedded(driverContext) diff --git a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala index 93d034622c..4d1497025c 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala @@ -28,12 +28,14 @@ object FlightRecorderEvents { val AeronSink_EnvelopeOffered = 18 val AeronSink_GaveUpEnvelope = 19 val AeronSink_DelegateToTaskRunner = 20 + val AeronSink_ReturnFromTaskRunner = 21 // Aeron Source events - val AeronSource_Started = 21 - val AeronSource_Stopped = 22 - val AeronSource_Received = 23 - val AeronSource_DelegateToTaskRunner = 24 + val AeronSource_Started = 22 + val AeronSource_Stopped = 23 + val AeronSource_Received = 24 + val AeronSource_DelegateToTaskRunner = 25 + val AeronSource_ReturnFromTaskRunner = 26 // Compression events val Compression_CompressedActorRef = 25 diff --git a/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala b/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala index 6778260759..a8e1d67750 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala @@ -3,7 +3,7 @@ */ package akka.remote.artery -import java.util.concurrent.TimeUnit +import java.util.concurrent.TimeUnit.MICROSECONDS import scala.util.control.NonFatal import akka.actor.ExtendedActorSystem import akka.dispatch.AbstractNodeQueue @@ -80,12 +80,29 @@ private[akka] object TaskRunner { override def toString(): String = elements.filterNot(_ eq null).mkString("[", ",", "]") } + + def createIdleStrategy(idleCpuLevel: Int): IdleStrategy = { + if (idleCpuLevel == 1) { + val maxParkMicros = 400 + new BackoffIdleStrategy(100, 1, MICROSECONDS.toNanos(1), MICROSECONDS.toNanos(maxParkMicros)) + } 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, idleCpuLevel: Int) extends Runnable { +private[akka] class TaskRunner(system: ExtendedActorSystem, val idleCpuLevel: Int) extends Runnable { import TaskRunner._ private val log = Logging(system, getClass) @@ -93,21 +110,7 @@ private[akka] class TaskRunner(system: ExtendedActorSystem, idleCpuLevel: Int) e private[this] val cmdQueue = new CommandQueue private[this] val tasks = new ArrayBag[Task] - // TODO the backoff strategy should be measured and tuned - private val idleStrategy: IdleStrategy = { - if (idleCpuLevel == 1) { - val maxParkMicros = 400 - new BackoffIdleStrategy(1, 1, TimeUnit.MICROSECONDS.toNanos(1), TimeUnit.MICROSECONDS.toNanos(maxParkMicros)) - } else if (idleCpuLevel == 10) - new BusySpinIdleStrategy - else { - val spinning = 100000 * idleCpuLevel - val yielding = 2 * idleCpuLevel - val maxParkMicros = 40 * (11 - idleCpuLevel) - new BackoffIdleStrategy( - spinning, yielding, TimeUnit.MICROSECONDS.toNanos(1), TimeUnit.MICROSECONDS.toNanos(maxParkMicros)) - } - } + private val idleStrategy = createIdleStrategy(idleCpuLevel) private var reset = false def start(): Unit = { @@ -132,8 +135,8 @@ private[akka] class TaskRunner(system: ExtendedActorSystem, idleCpuLevel: Int) e try { running = true while (running) { - executeTasks() processCommand(cmdQueue.poll()) + executeTasks() if (reset) { reset = false idleStrategy.reset() diff --git a/akka-remote/src/test/resources/aeron.properties b/akka-remote/src/test/resources/aeron.properties index db195e1075..007050a287 100644 --- a/akka-remote/src/test/resources/aeron.properties +++ b/akka-remote/src/test/resources/aeron.properties @@ -11,6 +11,11 @@ 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/artery/AeronStreamsApp.scala b/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala deleted file mode 100644 index 6a8bf49089..0000000000 --- a/akka-remote/src/test/scala/akka/remote/artery/AeronStreamsApp.scala +++ /dev/null @@ -1,315 +0,0 @@ -/** - * Copyright (C) 2016 Lightbend Inc. - */ -package akka.remote.artery - -import scala.concurrent.duration._ -import akka.actor.ActorSystem -import akka.stream.ActorMaterializer -import akka.stream.scaladsl.Source -import io.aeron.Aeron -import io.aeron.driver.MediaDriver -import java.util.concurrent.Executors -import scala.util.Success -import scala.util.Failure -import scala.concurrent.Future -import akka.Done -import org.HdrHistogram.Histogram -import java.util.concurrent.atomic.AtomicInteger -import java.util.concurrent.CountDownLatch -import java.util.concurrent.CyclicBarrier -import java.util.concurrent.atomic.AtomicLongArray -import akka.stream.ThrottleMode -import org.agrona.ErrorHandler -import io.aeron.AvailableImageHandler -import io.aeron.UnavailableImageHandler -import io.aeron.Image -import io.aeron.AvailableImageHandler -import akka.actor.ExtendedActorSystem -import java.io.File -import io.aeron.CncFileDescriptor - -object AeronStreamsApp { - - val channel1 = "aeron:udp?endpoint=localhost:40123" - val channel2 = "aeron:udp?endpoint=localhost:40124" - val streamId = 1 - val throughputN = 10000000 - val latencyRate = 10000 // per second - val latencyN = 10 * latencyRate - val payload = ("0" * 100).getBytes("utf-8") - val giveUpSendAfter = 60.seconds - lazy val sendTimes = new AtomicLongArray(latencyN) - - lazy val driver = { - val driverContext = new MediaDriver.Context - driverContext.clientLivenessTimeoutNs(SECONDS.toNanos(10)) - driverContext.imageLivenessTimeoutNs(SECONDS.toNanos(10)) - driverContext.driverTimeoutMs(SECONDS.toNanos(10)) - MediaDriver.launchEmbedded(driverContext) - } - - lazy val stats = { - new AeronStat(AeronStat.mapCounters(new File(driver.aeronDirectoryName, CncFileDescriptor.CNC_FILE))) - } - - lazy val aeron = { - val ctx = new Aeron.Context - ctx.errorHandler(new ErrorHandler { - override def onError(cause: Throwable) { - println(s"# Aeron onError " + cause) // FIXME - } - }) - ctx.availableImageHandler(new AvailableImageHandler { - override def onAvailableImage(img: Image): Unit = { - println(s"onAvailableImage from ${img.sourceIdentity} session ${img.sessionId}") - } - }) - ctx.unavailableImageHandler(new UnavailableImageHandler { - override def onUnavailableImage(img: Image): Unit = { - println(s"onUnavailableImage from ${img.sourceIdentity} session ${img.sessionId}") - } - }) - - ctx.aeronDirectoryName(driver.aeronDirectoryName) - Aeron.connect(ctx) - } - - lazy val system = ActorSystem("AeronStreams") - lazy implicit val mat = ActorMaterializer()(system) - - val idleCpuLevel = 5 - lazy val taskRunner = { - val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem], idleCpuLevel) - r.start() - r - } - - lazy val reporter = new RateReporter(SECONDS.toNanos(1), new RateReporter.Reporter { - override def onReport(messagesPerSec: Double, bytesPerSec: Double, totalMessages: Long, totalBytes: Long): Unit = { - println("%.03g msgs/sec, %.03g bytes/sec, totals %d messages %d MB".format( - messagesPerSec, bytesPerSec, totalMessages, totalBytes / (1024 * 1024))) - } - }) - lazy val reporterExecutor = Executors.newFixedThreadPool(1) - - lazy val pool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumFrameSize) - - def stopReporter(): Unit = { - reporter.halt() - reporterExecutor.shutdown() - } - - def exit(status: Int): Unit = { - stopReporter() - - system.scheduler.scheduleOnce(10.seconds) { - mat.shutdown() - system.terminate() - new Thread { - Thread.sleep(3000) - System.exit(status) - }.run() - }(system.dispatcher) - } - - lazy val histogram = new Histogram(SECONDS.toNanos(10), 3) - - def printTotal(total: Int, pre: String, startTime: Long, payloadSize: Long): Unit = { - val d = (System.nanoTime - startTime).nanos.toMillis - println(f"### $total $pre of size ${payloadSize} bytes took $d ms, " + - f"${1000.0 * total / d}%.03g msg/s, ${1000.0 * total * payloadSize / d}%.03g bytes/s") - - if (histogram.getTotalCount > 0) { - println("Histogram of RTT latencies in microseconds.") - histogram.outputPercentileDistribution(System.out, 1000.0) - } - } - - def main(args: Array[String]): Unit = { - - // receiver of plain throughput testing - if (args.length == 0 || args(0) == "receiver") - runReceiver() - - // sender of plain throughput testing - if (args.length == 0 || args(0) == "sender") - runSender() - - // sender of ping-pong latency testing - if (args.length != 0 && args(0) == "echo-sender") - runEchoSender() - - // echo receiver of ping-pong latency testing - if (args.length != 0 && args(0) == "echo-receiver") - runEchoReceiver() - - if (args.length != 0 && args(0) == "debug-receiver") - runDebugReceiver() - - if (args.length != 0 && args(0) == "debug-sender") - runDebugSender() - - if (args.length >= 2 && args(1) == "stats") - runStats() - } - - def runReceiver(): Unit = { - import system.dispatcher - reporterExecutor.execute(reporter) - val r = reporter - var t0 = System.nanoTime() - var count = 0L - var payloadSize = 0L - Source.fromGraph(new AeronSource(channel1, streamId, aeron, taskRunner, pool, IgnoreEventSink)) - .map { envelope ⇒ - r.onMessage(1, envelope.byteBuffer.limit) - envelope - } - .runForeach { envelope ⇒ - count += 1 - if (count == 1) { - t0 = System.nanoTime() - payloadSize = envelope.byteBuffer.limit - } else if (count == throughputN) { - exit(0) - printTotal(throughputN, "receive", t0, payloadSize) - } - pool.release(envelope) - }.onFailure { - case e ⇒ - e.printStackTrace - exit(-1) - } - - } - - def runSender(): Unit = { - reporterExecutor.execute(reporter) - val r = reporter - val t0 = System.nanoTime() - Source(1 to throughputN) - .map { n ⇒ - if (n == throughputN) { - exit(0) - printTotal(throughputN, "send", t0, payload.length) - } - n - } - .map { _ ⇒ - r.onMessage(1, payload.length) - val envelope = pool.acquire() - envelope.byteBuffer.put(payload) - envelope.byteBuffer.flip() - envelope - } - .runWith(new AeronSink(channel1, streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) - } - - def runEchoReceiver(): Unit = { - // just echo back on channel2 - reporterExecutor.execute(reporter) - val r = reporter - Source.fromGraph(new AeronSource(channel1, streamId, aeron, taskRunner, pool, IgnoreEventSink)) - .map { envelope ⇒ - r.onMessage(1, envelope.byteBuffer.limit) - envelope - } - .runWith(new AeronSink(channel2, streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) - } - - def runEchoSender(): Unit = { - import system.dispatcher - reporterExecutor.execute(reporter) - val r = reporter - - val barrier = new CyclicBarrier(2) - var repeat = 3 - val count = new AtomicInteger - var t0 = System.nanoTime() - Source.fromGraph(new AeronSource(channel2, streamId, aeron, taskRunner, pool, IgnoreEventSink)) - .map { envelope ⇒ - r.onMessage(1, envelope.byteBuffer.limit) - envelope - } - .runForeach { envelope ⇒ - val c = count.incrementAndGet() - val d = System.nanoTime() - sendTimes.get(c - 1) - if (c % (latencyN / 10) == 0) - println(s"# receive offset $c => ${d / 1000} µs") // FIXME - histogram.recordValue(d) - if (c == latencyN) { - printTotal(latencyN, "ping-pong", t0, envelope.byteBuffer.limit) - barrier.await() // this is always the last party - } - pool.release(envelope) - }.onFailure { - case e ⇒ - e.printStackTrace - exit(-1) - } - - while (repeat > 0) { - repeat -= 1 - histogram.reset() - count.set(0) - t0 = System.nanoTime() - - Source(1 to latencyN) - .throttle(latencyRate, 1.second, latencyRate / 10, ThrottleMode.Shaping) - .map { n ⇒ - if (n % (latencyN / 10) == 0) - println(s"# send offset $n") // FIXME - sendTimes.set(n - 1, System.nanoTime()) - val envelope = pool.acquire() - envelope.byteBuffer.put(payload) - envelope.byteBuffer.flip() - envelope - } - .runWith(new AeronSink(channel1, streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) - - barrier.await() - } - - exit(0) - } - - def runDebugReceiver(): Unit = { - import system.dispatcher - Source.fromGraph(new AeronSource(channel1, streamId, aeron, taskRunner, pool, IgnoreEventSink)) - .map { envelope ⇒ - val bytes = Array.ofDim[Byte](envelope.byteBuffer.limit) - envelope.byteBuffer.get(bytes) - pool.release(envelope) - new String(bytes, "utf-8") - } - .runForeach { s ⇒ - println(s) - }.onFailure { - case e ⇒ - e.printStackTrace - exit(-1) - } - - } - - def runDebugSender(): Unit = { - val fill = "0000" - Source(1 to 1000) - .throttle(1, 1.second, 1, ThrottleMode.Shaping) - .map { n ⇒ - val s = (fill + n.toString).takeRight(4) - println(s) - val envelope = pool.acquire() - envelope.byteBuffer.put(s.getBytes("utf-8")) - envelope.byteBuffer.flip() - envelope - } - .runWith(new AeronSink(channel1, streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) - } - - def runStats(): Unit = { - Source.tick(10.second, 10.second, "tick").runForeach { _ ⇒ stats.print(System.out) } - } - -} From 95a81e41f93bf1c9f59e0991a2da4603b266981a Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 6 Jul 2016 22:49:10 +0200 Subject: [PATCH 090/186] enable compression by default --- .../scala/akka/remote/artery/LatencySpec.scala | 9 ++++----- .../scala/akka/remote/artery/MaxThroughputSpec.scala | 11 +++++------ akka-remote/src/main/resources/reference.conf | 4 ++-- .../remote/artery/compress/CompressionSettings.scala | 3 --- .../scala/akka/remote/artery/HandshakeRetrySpec.scala | 3 +-- .../artery/compress/CompressionIntegrationSpec.scala | 1 - .../HandshakeShouldDropCompressionTableSpec.scala | 2 -- 7 files changed, 12 insertions(+), 21 deletions(-) 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 index 185221267c..56c46241a6 100644 --- 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 @@ -45,7 +45,6 @@ object LatencySpec extends MultiNodeConfig { advanced.idle-cpu-level=7 advanced.compression { - enabled = on actor-refs.advertisement-interval = 2 second manifests.advertisement-interval = 2 second } @@ -73,11 +72,11 @@ object LatencySpec extends MultiNodeConfig { } def receiverProps(reporter: RateReporter, settings: TestSettings, totalMessages: Int, - sendTimes: AtomicLongArray, histogram: Histogram, plotsRef: ActorRef): Props = + 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 { + sendTimes: AtomicLongArray, histogram: Histogram, plotsRef: ActorRef) extends Actor { import settings._ var count = 0 @@ -131,10 +130,10 @@ object LatencySpec extends MultiNodeConfig { } final case class TestSettings( - testName: String, + testName: String, messageRate: Int, // msg/s payloadSize: Int, - repeat: Int) + repeat: Int) } 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 index 0ad6c1c2bb..d12dba57db 100644 --- 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 @@ -60,7 +60,6 @@ object MaxThroughputSpec extends MultiNodeConfig { #advanced.aeron-dir = "target/aeron" advanced.compression { - enabled = on actor-refs.advertisement-interval = 2 second manifests.advertisement-interval = 2 second } @@ -102,7 +101,7 @@ object MaxThroughputSpec extends MultiNodeConfig { } def senderProps(target: ActorRef, testSettings: TestSettings, plotRef: ActorRef, - printTaskRunnerMetrics: Boolean): Props = + printTaskRunnerMetrics: Boolean): Props = Props(new Sender(target, testSettings, plotRef, printTaskRunnerMetrics)) class Sender(target: ActorRef, testSettings: TestSettings, plotRef: ActorRef, printTaskRunnerMetrics: Boolean) @@ -212,10 +211,10 @@ object MaxThroughputSpec extends MultiNodeConfig { } final case class TestSettings( - testName: String, - totalMessages: Long, - burstSize: Int, - payloadSize: Int, + testName: String, + totalMessages: Long, + burstSize: Int, + payloadSize: Int, senderReceiverPairs: Int) { // data based on measurement def totalSize(system: ActorSystem) = payloadSize + (if (CompressionSettings(system).enabled) 38 else 110) diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index f4e466d7df..80c8f4d2a8 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -142,8 +142,8 @@ akka { # compression of common strings in remoting messages, like actor destinations, serializers etc compression { - # global flag to disable all compression - enabled = off + # possibility to disable compression by setting this to off + enabled = on # unlocks additional very verbose debug logging of compression events (on DEBUG log level) debug = off diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionSettings.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionSettings.scala index 1b7e1ca6ef..82a9c7e752 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionSettings.scala @@ -13,21 +13,18 @@ import scala.concurrent.duration._ /** INTERNAL API */ private[akka] class CompressionSettings(_config: Config) { val enabled = _config.getBoolean("enabled") - @inline private def globalEnabled = enabled val debug = _config.getBoolean("debug") object actorRefs { private val c = _config.getConfig("actor-refs") - val enabled = globalEnabled && c.getBoolean("enabled") val advertisementInterval = c.getDuration("advertisement-interval", TimeUnit.MILLISECONDS).millis val max = c.getInt("max") } object manifests { private val c = _config.getConfig("manifests") - val enabled = globalEnabled && c.getBoolean("enabled") val advertisementInterval = c.getDuration("advertisement-interval", TimeUnit.MILLISECONDS).millis val max = c.getInt("max") } diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala index 16a7605a17..0edc30d3eb 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala @@ -20,11 +20,10 @@ object HandshakeRetrySpec { akka { actor.provider = remote remote.artery.enabled = on - remote.artery.compression.enabled = on remote.artery.hostname = localhost remote.artery.port = 0 remote.handshake-timeout = 10s - + } """) 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 index 9e3b602bd0..a9aab071d3 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala @@ -26,7 +26,6 @@ object CompressionIntegrationSpec { actor.provider = "akka.remote.RemoteActorRefProvider" remote.artery.enabled = on remote.artery.advanced { - compression.enabled = on compression.debug = on } remote.artery.hostname = localhost 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 index 4ee33cb9a0..74ac512e67 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala @@ -33,9 +33,7 @@ object HandshakeShouldDropCompressionTableSpec { remote.handshake-timeout = 10s remote.artery.advanced.compression { - enabled = on actor-refs { - enabled = on # we'll trigger advertisement manually advertisement-interval = 10 hours } From 57ca2739033542d6d2d57aec74a9a9d4e4099c35 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 7 Jul 2016 10:27:24 +0200 Subject: [PATCH 091/186] adjust the hit count sampling with the rate --- .../akka/remote/artery/LatencySpec.scala | 34 ++++++++++++------- .../remote/artery/MaxThroughputSpec.scala | 12 +++---- .../scala/akka/remote/artery/Codecs.scala | 25 ++++++++------ .../compress/CompressionIntegrationSpec.scala | 6 +--- 4 files changed, 42 insertions(+), 35 deletions(-) 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 index 56c46241a6..896e48124d 100644 --- 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 @@ -72,11 +72,11 @@ object LatencySpec extends MultiNodeConfig { } def receiverProps(reporter: RateReporter, settings: TestSettings, totalMessages: Int, - sendTimes: AtomicLongArray, histogram: Histogram, plotsRef: ActorRef): Props = + 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 { + sendTimes: AtomicLongArray, histogram: Histogram, plotsRef: ActorRef) extends Actor { import settings._ var count = 0 @@ -109,7 +109,7 @@ object LatencySpec extends MultiNodeConfig { value.getPercentileLevelIteratedTo < (p + 0.5) ⇒ value.getValueIteratedTo / 1000.0 }.getOrElse(Double.NaN) - val throughput = 1000.0 * histogram.getTotalCount / math.min(1, totalDurationNanos.nanos.toMillis) + 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, " + @@ -130,10 +130,10 @@ object LatencySpec extends MultiNodeConfig { } final case class TestSettings( - testName: String, + testName: String, messageRate: Int, // msg/s payloadSize: Int, - repeat: Int) + repeat: Int) } @@ -224,6 +224,15 @@ abstract class LatencySpec 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) @@ -238,16 +247,8 @@ abstract class LatencySpec } warmup.foreach { _ ⇒ - 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) @@ -261,6 +262,13 @@ abstract class LatencySpec 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) 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 index d12dba57db..8dad9e0c6e 100644 --- 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 @@ -101,7 +101,7 @@ object MaxThroughputSpec extends MultiNodeConfig { } def senderProps(target: ActorRef, testSettings: TestSettings, plotRef: ActorRef, - printTaskRunnerMetrics: Boolean): Props = + printTaskRunnerMetrics: Boolean): Props = Props(new Sender(target, testSettings, plotRef, printTaskRunnerMetrics)) class Sender(target: ActorRef, testSettings: TestSettings, plotRef: ActorRef, printTaskRunnerMetrics: Boolean) @@ -211,10 +211,10 @@ object MaxThroughputSpec extends MultiNodeConfig { } final case class TestSettings( - testName: String, - totalMessages: Long, - burstSize: Int, - payloadSize: Int, + testName: String, + totalMessages: Long, + burstSize: Int, + payloadSize: Int, senderReceiverPairs: Int) { // data based on measurement def totalSize(system: ActorSystem) = payloadSize + (if (CompressionSettings(system).enabled) 38 else 110) @@ -306,7 +306,7 @@ abstract class MaxThroughputSpec senderReceiverPairs = 1), TestSettings( testName = "1-to-1", - totalMessages = adjustedTotalMessages(20000), + totalMessages = adjustedTotalMessages(50000), burstSize = 1000, payloadSize = 100, senderReceiverPairs = 1), diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 848369d2e4..6c9a1149dc 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -152,8 +152,7 @@ private[remote] class Decoder( // adaptive sampling when rate > 1000 msg/s private var messageCount = 0L - private val HeavyHitterMask = (1 << 8) - 1 // sample every 256nth message - private var adaptiveSampling = false + 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 @@ -192,7 +191,7 @@ private[remote] class Decoder( val classManifest = headerBuilder.manifest(originUid) - if (!adaptiveSampling || (messageCount & HeavyHitterMask) == 0) { + if ((messageCount & heavyHitterMask) == 0) { // --- hit refs and manifests for heavy-hitter counting association match { case OptionVal.Some(assoc) ⇒ @@ -269,14 +268,18 @@ private[remote] class Decoder( timerKey match { case Tick ⇒ val now = System.nanoTime() - val d = now - tickTimestamp - val oldAdaptiveSampling = adaptiveSampling - adaptiveSampling = (d == 0 || - (messageCount - tickMessageCount) * TimeUnit.SECONDS.toNanos(1) / d > adaptiveSamplingRateThreshold) - if (!oldAdaptiveSampling && adaptiveSampling) - log.info("Turning on adaptive sampling ({}nth message) of compression hit counting", HeavyHitterMask + 1) - else if (oldAdaptiveSampling && !adaptiveSampling) - log.info("Turning off adaptive sampling of compression hit counting") + 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 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 index a9aab071d3..4fd819062c 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala @@ -33,11 +33,7 @@ object CompressionIntegrationSpec { remote.handshake-timeout = 10s remote.artery.advanced.compression { - enabled = on - actor-refs { - enabled = on - advertisement-interval = 3 seconds - } + actor-refs.advertisement-interval = 3 seconds } } From 5e90d4db40cc7376d11ea3ff5ee13904bd41f8a1 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 8 Jul 2016 01:00:41 +0200 Subject: [PATCH 092/186] =art place OutboundTestStage after SystemMessageDelivery stage (#20899) * failing test was akka.cluster.AttemptSysMsgRedelivery when running with Artery * we rely on that system messages are not dropped before the redelivery stage, i.e. blackhole must be after that --- .../scala/akka/remote/artery/ArteryTransport.scala | 14 +++++++++++--- .../scala/akka/remote/artery/Association.scala | 6 ++++-- 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 92fa84e5da..cc114df912 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -765,18 +765,26 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R envelopePool, giveUpSendAfter, createFlightRecorderEventSink()))(Keep.right) } - def outboundControl(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[OutboundEnvelope, (OutboundControlIngress, Future[Done])] = { + /** + * The outbound stream is defined as two parts to be able to add test stage in-between. + * System messages must not be dropped before the SystemMessageDelivery stage. + */ + def outboundControlPart1(outboundContext: OutboundContext): Flow[OutboundEnvelope, OutboundEnvelope, SharedKillSwitch] = { Flow.fromGraph(killSwitch.flow[OutboundEnvelope]) .via(new OutboundHandshake(system, outboundContext, outboundEnvelopePool, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) .via(new SystemMessageDelivery(outboundContext, system.deadLetters, systemMessageResendInterval, remoteSettings.SysMsgBufferSize)) + + // FIXME we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages + } + + def outboundControlPart2(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[OutboundEnvelope, (OutboundControlIngress, Future[Done])] = { + Flow[OutboundEnvelope] .viaMat(new OutboundControlJunction(outboundContext, outboundEnvelopePool))(Keep.right) .via(encoder(compression)) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), controlStreamId, aeron, taskRunner, envelopePool, Duration.Inf, createFlightRecorderEventSink()))(Keep.both) - - // FIXME we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages } def createEncoder(compression: OutboundCompressions, bufferPool: EnvelopeBufferPool): Flow[OutboundEnvelope, EnvelopeBuffer, NotUsed] = diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index fdbfca9747..da3936b065 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -310,14 +310,16 @@ private[remote] class Association( if (transport.remoteSettings.TestMode) { val ((queueValue, mgmt), (control, completed)) = Source.fromGraph(new SendQueue[OutboundEnvelope]) + .via(transport.outboundControlPart1(this)) .viaMat(transport.outboundTestFlow(this))(Keep.both) - .toMat(transport.outboundControl(this, compression))(Keep.both) + .toMat(transport.outboundControlPart2(this, compression))(Keep.both) .run()(materializer) _testStages.add(mgmt) (queueValue, (control, completed)) } else { Source.fromGraph(new SendQueue[OutboundEnvelope]) - .toMat(transport.outboundControl(this, compression))(Keep.both) + .via(transport.outboundControlPart1(this)) + .toMat(transport.outboundControlPart2(this, compression))(Keep.both) .run()(materializer) } From 5df0f5ea870d00414a9a424c273b2e154162c2b8 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 23 Aug 2016 20:04:22 +0200 Subject: [PATCH 093/186] update to Aeron 1.0 --- project/Dependencies.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/Dependencies.scala b/project/Dependencies.scala index a057e19ac7..8e78983a82 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -78,8 +78,8 @@ 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" % "0.9.9" // ApacheV2 - val aeronClient = "io.aeron" % "aeron-client" % "0.9.9" // ApacheV2 + val aeronDriver = "io.aeron" % "aeron-driver" % "1.0" // ApacheV2 + val aeronClient = "io.aeron" % "aeron-client" % "1.0" // ApacheV2 object Docs { val sprayJson = "io.spray" %% "spray-json" % "1.3.2" % "test" From 21a4899054a936739a0d4467b173e701418f0d8c Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 23 Aug 2016 20:38:39 +0200 Subject: [PATCH 094/186] use the new WildcardIndex --- .../scala/akka/remote/artery/ArteryTransport.scala | 12 +++++++----- .../main/scala/akka/remote/artery/Association.scala | 11 +++++------ 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index cc114df912..ae5344629c 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -46,7 +46,7 @@ import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Source import akka.util.Helpers.ConfigOps import akka.util.Helpers.Requiring -import akka.util.WildcardTree +import akka.util.WildcardIndex import io.aeron.Aeron import io.aeron.AvailableImageHandler import io.aeron.Image @@ -320,11 +320,13 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val remoteDispatcher = system.dispatchers.lookup(remoteSettings.Dispatcher) private val largeMessageDestinations = - system.settings.config.getStringList("akka.remote.artery.large-message-destinations").asScala.foldLeft(WildcardTree[NotUsed]()) { (tree, entry) ⇒ + system.settings.config.getStringList("akka.remote.artery.large-message-destinations").asScala.foldLeft(WildcardIndex[NotUsed]()) { (tree, entry) ⇒ val segments = entry.split('/').tail - tree.insert(segments.iterator, NotUsed) + tree.insert(segments, NotUsed) } - private val largeMessageDestinationsEnabled = largeMessageDestinations.children.nonEmpty + // TODO use WildcardIndex.isEmpty when merged from master + val largeMessageChannelEnabled = + !largeMessageDestinations.wildcardTree.isEmpty || !largeMessageDestinations.doubleWildcardTree.isEmpty private def inboundChannel = s"aeron:udp?endpoint=${localAddress.address.host.get}:${localAddress.address.port.get}" private def outboundChannel(a: Address) = s"aeron:udp?endpoint=${a.host.get}:${a.port.get}" @@ -526,7 +528,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R runInboundControlStream(noCompressions) // TODO should understand compressions too runInboundOrdinaryMessagesStream(compressions) - if (largeMessageDestinationsEnabled) { + if (largeMessageChannelEnabled) { runInboundLargeMessagesStream() } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index da3936b065..65706abf07 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -35,7 +35,7 @@ import akka.stream.AbruptTerminationException import akka.stream.Materializer import akka.stream.scaladsl.Keep import akka.stream.scaladsl.Source -import akka.util.{ Unsafe, WildcardTree } +import akka.util.{ Unsafe, WildcardIndex } import org.agrona.concurrent.ManyToOneConcurrentArrayQueue import akka.util.OptionVal import akka.remote.QuarantinedEvent @@ -62,7 +62,7 @@ private[remote] class Association( val materializer: Materializer, override val remoteAddress: Address, override val controlSubject: ControlMessageSubject, - largeMessageDestinations: WildcardTree[NotUsed], + largeMessageDestinations: WildcardIndex[NotUsed], outboundEnvelopePool: ObjectPool[ReusableOutboundEnvelope]) extends AbstractAssociation with OutboundContext { import Association._ @@ -77,7 +77,6 @@ private[remote] class Association( private val restartTimeout: FiniteDuration = 5.seconds // FIXME config private val maxRestarts = 5 // FIXME config private val restartCounter = new RestartCounter(maxRestarts, restartTimeout) - private val largeMessageChannelEnabled = largeMessageDestinations.children.nonEmpty // 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 @@ -200,7 +199,7 @@ private[remote] class Association( case _ ⇒ val outboundEnvelope = createOutboundEnvelope() val offerOk = - if (largeMessageChannelEnabled && isLargeMessageDestination(recipient)) + if (transport.largeMessageChannelEnabled && isLargeMessageDestination(recipient)) largeQueue.offer(outboundEnvelope) else queue.offer(outboundEnvelope) @@ -216,7 +215,7 @@ private[remote] class Association( case OptionVal.Some(r) ⇒ if (r.cachedLargeMessageDestinationFlag ne null) r.cachedLargeMessageDestinationFlag eq LargeDestination - else if (largeMessageDestinations.find(r.path.elements.iterator).data.isEmpty) { + else if (largeMessageDestinations.find(r.path.elements).isEmpty) { r.cachedLargeMessageDestinationFlag = RegularDestination false } else { @@ -293,7 +292,7 @@ private[remote] class Association( runOutboundControlStream(disableCompression) runOutboundOrdinaryMessagesStream(CurrentAssociationStateOutboundCompressionsProxy) - if (largeMessageChannelEnabled) { + if (transport.largeMessageChannelEnabled) { runOutboundLargeMessagesStream(disableCompression) } } From af5eb4c6bf142c400b2a9ae185561ad7eb9ddc03 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Fri, 26 Aug 2016 14:44:33 +0200 Subject: [PATCH 095/186] WIP separate prio artery channel (#21278) * First incorrect stab - separate prio channel * Send prio messages over the control stream --- .../akka/remote/RemoteActorRefProvider.scala | 11 +++-- .../src/main/scala/akka/remote/Remoting.scala | 2 + .../akka/remote/artery/ArteryTransport.scala | 37 ++++++++++----- .../akka/remote/artery/Association.scala | 47 +++++++++++-------- .../artery/LargeMessagesStreamSpec.scala | 8 ++-- 5 files changed, 64 insertions(+), 41 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index ac83fc7d54..607ef745ba 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -458,9 +458,10 @@ private[akka] trait RemoteRef extends ActorRefScope { /** * INTERNAL API */ -private[remote] sealed abstract class LargeMessageDestinationFlag -private[remote] case object RegularDestination extends LargeMessageDestinationFlag -private[remote] case object LargeDestination extends LargeMessageDestinationFlag +private[remote] sealed abstract class MessageDestinationFlag +private[remote] case object RegularDestination extends MessageDestinationFlag +private[remote] case object LargeDestination extends MessageDestinationFlag +private[remote] case object PriorityDestination extends MessageDestinationFlag /** * INTERNAL API @@ -486,8 +487,8 @@ private[akka] class RemoteActorRef private[akka] ( } @volatile private[remote] var cachedAssociation: artery.Association = null - // used by artery to direct messages to a separate stream for large messages - @volatile private[remote] var cachedLargeMessageDestinationFlag: LargeMessageDestinationFlag = null + // used by artery to direct messages to separate specialized streams + @volatile private[remote] var cachedMessageDestinationFlag: MessageDestinationFlag = null def getChild(name: Iterator[String]): InternalActorRef = { val s = name.toStream diff --git a/akka-remote/src/main/scala/akka/remote/Remoting.scala b/akka-remote/src/main/scala/akka/remote/Remoting.scala index 8a35a63126..35817c99d0 100644 --- a/akka-remote/src/main/scala/akka/remote/Remoting.scala +++ b/akka-remote/src/main/scala/akka/remote/Remoting.scala @@ -54,6 +54,8 @@ 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 diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index ae5344629c..6131cb2cc6 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -328,11 +328,21 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R val largeMessageChannelEnabled = !largeMessageDestinations.wildcardTree.isEmpty || !largeMessageDestinations.doubleWildcardTree.isEmpty + private val priorityMessageDestinations = + WildcardIndex[NotUsed]() + // this comes from remoting so is semi-ok to be hardcoded here + .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=${localAddress.address.host.get}:${localAddress.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 = 3 private val largeStreamId = 4 + private val taskRunner = new TaskRunner(system, remoteSettings.IdleCpuLevel) private val restartTimeout: FiniteDuration = 5.seconds // FIXME config @@ -367,7 +377,13 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R createFlightRecorderEventSink(synchr = true) private val associationRegistry = new AssociationRegistry( - remoteAddress ⇒ new Association(this, materializer, remoteAddress, controlSubject, largeMessageDestinations, + remoteAddress ⇒ new Association( + this, + materializer, + remoteAddress, + controlSubject, + largeMessageDestinations, + priorityMessageDestinations, outboundEnvelopePool)) def remoteSettings: RemoteSettings = provider.remoteSettings @@ -749,21 +765,18 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R association(remoteAddress).quarantine(reason = "", uid.map(_.toLong)) } - def outbound(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[OutboundEnvelope, Future[Done]] = { - Flow.fromGraph(killSwitch.flow[OutboundEnvelope]) - .via(new OutboundHandshake(system, outboundContext, outboundEnvelopePool, handshakeTimeout, - handshakeRetryInterval, injectHandshakeInterval)) - .via(encoder(compression)) - .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), ordinaryStreamId, aeron, taskRunner, - envelopePool, giveUpSendAfter, createFlightRecorderEventSink()))(Keep.right) - } + def outbound(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[OutboundEnvelope, Future[Done]] = + createOutboundSink(ordinaryStreamId, outboundContext, compression, envelopePool) - def outboundLarge(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[OutboundEnvelope, Future[Done]] = { + def outboundLarge(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[OutboundEnvelope, Future[Done]] = + createOutboundSink(largeStreamId, outboundContext, compression, largeEnvelopePool) + + private def createOutboundSink(streamId: Int, outboundContext: OutboundContext, compression: OutboundCompressions, bufferPool: EnvelopeBufferPool): Sink[OutboundEnvelope, Future[Done]] = { Flow.fromGraph(killSwitch.flow[OutboundEnvelope]) .via(new OutboundHandshake(system, outboundContext, outboundEnvelopePool, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) - .via(createEncoder(largeEnvelopePool, compression)) - .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), largeStreamId, aeron, taskRunner, + .via(createEncoder(bufferPool, compression)) + .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), streamId, aeron, taskRunner, envelopePool, giveUpSendAfter, createFlightRecorderEventSink()))(Keep.right) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 65706abf07..6c3a29e908 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -25,11 +25,12 @@ import akka.actor.Address import akka.actor.RootActorPath import akka.dispatch.sysmsg.SystemMessage import akka.event.Logging -import akka.remote.{ LargeDestination, RegularDestination, RemoteActorRef, UniqueAddress } +import akka.remote._ import akka.remote.artery.AeronSink.GaveUpSendingException import akka.remote.artery.InboundControlJunction.ControlMessageSubject import akka.remote.artery.OutboundControlJunction.OutboundControlIngress import akka.remote.artery.OutboundHandshake.HandshakeTimeoutException +import akka.remote.artery.SendQueue.ProducerApi import akka.remote.artery.SystemMessageDelivery.ClearSystemMessageDelivery import akka.stream.AbruptTerminationException import akka.stream.Materializer @@ -38,8 +39,6 @@ import akka.stream.scaladsl.Source import akka.util.{ Unsafe, WildcardIndex } import org.agrona.concurrent.ManyToOneConcurrentArrayQueue import akka.util.OptionVal -import akka.remote.QuarantinedEvent -import akka.remote.DaemonMsgCreate import akka.remote.artery.compress.CompressionProtocol._ /** @@ -63,6 +62,7 @@ private[remote] class Association( override val remoteAddress: Address, override val controlSubject: ControlMessageSubject, largeMessageDestinations: WildcardIndex[NotUsed], + priorityMessageDestinations: WildcardIndex[NotUsed], outboundEnvelopePool: ObjectPool[ReusableOutboundEnvelope]) extends AbstractAssociation with OutboundContext { import Association._ @@ -198,11 +198,8 @@ private[remote] class Association( transport.system.deadLetters ! outboundEnvelope2 case _ ⇒ val outboundEnvelope = createOutboundEnvelope() - val offerOk = - if (transport.largeMessageChannelEnabled && isLargeMessageDestination(recipient)) - largeQueue.offer(outboundEnvelope) - else - queue.offer(outboundEnvelope) + val queue = selectQueue(recipient) + val offerOk = queue.offer(outboundEnvelope) if (!offerOk) transport.system.deadLetters ! outboundEnvelope } @@ -210,20 +207,30 @@ private[remote] class Association( log.debug("Dropping message to quarantined system {}", remoteAddress) } - private def isLargeMessageDestination(recipient: OptionVal[RemoteActorRef]): Boolean = { + @tailrec + private def selectQueue(recipient: OptionVal[RemoteActorRef]): ProducerApi[OutboundEnvelope] = { recipient match { case OptionVal.Some(r) ⇒ - if (r.cachedLargeMessageDestinationFlag ne null) - r.cachedLargeMessageDestinationFlag eq LargeDestination - else if (largeMessageDestinations.find(r.path.elements).isEmpty) { - r.cachedLargeMessageDestinationFlag = RegularDestination - false - } else { - log.debug("Using large message stream for {}", r.path) - r.cachedLargeMessageDestinationFlag = LargeDestination - true + r.cachedMessageDestinationFlag match { + case RegularDestination ⇒ queue + case PriorityDestination ⇒ controlQueue + case LargeDestination ⇒ largeQueue + case null ⇒ + // only happens when messages are sent to new remote destination + // and is then cached on the RemoteActorRef + val elements = r.path.elements + if (priorityMessageDestinations.find(elements).isDefined) { + log.debug("Using priority message stream for {}", r.path) + r.cachedMessageDestinationFlag = PriorityDestination + } else if (transport.largeMessageChannelEnabled && largeMessageDestinations.find(elements).isDefined) { + log.debug("Using large message stream for {}", r.path) + r.cachedMessageDestinationFlag = LargeDestination + } else { + r.cachedMessageDestinationFlag = RegularDestination + } + selectQueue(recipient) } - case OptionVal.None ⇒ false + case OptionVal.None ⇒ queue } } @@ -370,7 +377,7 @@ private[remote] class Association( } private def runOutboundLargeMessagesStream(compression: OutboundCompressions): Unit = { - val wrapper = getOrCreateQueueWrapper(queue, largeQueueSize) + val wrapper = getOrCreateQueueWrapper(largeQueue, largeQueueSize) largeQueue = wrapper // use new underlying queue immediately for restarts val (queueValue, completed) = diff --git a/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala index 1eaaa5d103..bb584ea8f2 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala @@ -51,7 +51,7 @@ class LargeMessagesStreamSpec extends ArteryMultiNodeSpec( senderProbeA.expectMsg(Pong(0)) // flag should be cached now - regularRemote.asInstanceOf[RemoteActorRef].cachedLargeMessageDestinationFlag should ===(RegularDestination) + regularRemote.asInstanceOf[RemoteActorRef].cachedMessageDestinationFlag should ===(RegularDestination) } @@ -75,7 +75,7 @@ class LargeMessagesStreamSpec extends ArteryMultiNodeSpec( senderProbeA.expectMsg(Pong(0)) // flag should be cached now - largeRemote.asInstanceOf[RemoteActorRef].cachedLargeMessageDestinationFlag should ===(LargeDestination) + largeRemote.asInstanceOf[RemoteActorRef].cachedMessageDestinationFlag should ===(LargeDestination) } @@ -112,8 +112,8 @@ class LargeMessagesStreamSpec extends ArteryMultiNodeSpec( remoteProbe.expectMsg(10.seconds, Pong(largeBytes)) // cached flags should be set now - largeRemote.asInstanceOf[RemoteActorRef].cachedLargeMessageDestinationFlag should ===(LargeDestination) - regularRemote.asInstanceOf[RemoteActorRef].cachedLargeMessageDestinationFlag should ===(RegularDestination) + largeRemote.asInstanceOf[RemoteActorRef].cachedMessageDestinationFlag should ===(LargeDestination) + regularRemote.asInstanceOf[RemoteActorRef].cachedMessageDestinationFlag should ===(RegularDestination) } } From 0c0e3c5efd1b01d66d92a02d3ae2ecf3d52c57f2 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 24 Aug 2016 19:52:07 +0200 Subject: [PATCH 096/186] Refactoring of outbound compression, #21210 * outbound compression is now immutable, by simply using CompressionTable[ActorRef] and CompressionTable[String] * immutable outbound compression will make it possible to use them from multiple Encoder instances, when we add several lanes for parallel serialization * outbound compression tables not shared via AssociationState * the advertised tables are sent to the Encoder stage via async callback, no need to reference the tables in other places than the Encoder stage, no more races via shared mutable state * when outbound stream is started or restarted it can start out without compression, until next advertisement is received * ensure outbound compression is cleared before handshake is signaled complete --- .../akka/remote/artery/CodecBenchmark.scala | 13 +- .../compress/HeavyHittersBenchmark.scala | 2 +- .../akka/remote/artery/ArteryTransport.scala | 125 ++++++------ .../akka/remote/artery/Association.scala | 178 +++++++++--------- .../scala/akka/remote/artery/BufferPool.scala | 76 +++++--- .../scala/akka/remote/artery/Codecs.scala | 110 ++++++++--- .../scala/akka/remote/artery/Handshake.scala | 44 +++-- .../artery/compress/ActualCompressions.scala | 98 ---------- .../artery/compress/AllCompressions.scala | 39 ---- .../artery/compress/CompressionTable.scala | 9 + .../artery/compress/InboundCompressions.scala | 91 ++++++++- .../NoInboundCompressions.scala} | 21 +-- .../OutboundActorRefCompression.scala | 114 ----------- .../remote/artery/EnvelopeBufferSpec.scala | 30 ++- .../akka/remote/artery/TestContext.scala | 15 +- .../compress/OutboundCompressionSpec.scala | 29 +-- 16 files changed, 454 insertions(+), 540 deletions(-) delete mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressions.scala delete mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/AllCompressions.scala rename akka-remote/src/main/scala/akka/remote/artery/{NoLiteralCompression.scala => compress/NoInboundCompressions.scala} (51%) delete mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompression.scala 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 index f08d5f87af..98974a245b 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -29,6 +29,8 @@ 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) @@ -57,7 +59,6 @@ class CodecBenchmark { private val inboundEnvelopePool = ReusableInboundEnvelope.createObjectPool(capacity = 16) private val outboundEnvelopePool = ReusableOutboundEnvelope.createObjectPool(capacity = 16) - val compressionOut = NoOutboundCompressions val headerIn = HeaderBuilder.in(NoInboundCompressions) val envelopeTemplateBuffer = ByteBuffer.allocate(ArteryTransport.MaximumFrameSize).order(ByteOrder.LITTLE_ENDIAN) @@ -73,7 +74,7 @@ class CodecBenchmark { // 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): Unit = ??? + override def completeHandshake(peer: UniqueAddress): Future[Done] = ??? } private var materializer: ActorMaterializer = _ @@ -136,8 +137,8 @@ class CodecBenchmark { val latch = new CountDownLatch(1) val N = 100000 - val encoder: Flow[OutboundEnvelope, EnvelopeBuffer, NotUsed] = - Flow.fromGraph(new Encoder(uniqueLocalAddress, system, compressionOut, outboundEnvelopePool, envelopePool)) + val encoder: Flow[OutboundEnvelope, EnvelopeBuffer, Encoder.ChangeOutboundCompression] = + Flow.fromGraph(new Encoder(uniqueLocalAddress, system, outboundEnvelopePool, envelopePool)) Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) .map(msg ⇒ outboundEnvelopePool.acquire().init(OptionVal.None, payload, OptionVal.Some(remoteRefB))) @@ -193,8 +194,8 @@ class CodecBenchmark { val latch = new CountDownLatch(1) val N = 100000 - val encoder: Flow[OutboundEnvelope, EnvelopeBuffer, NotUsed] = - Flow.fromGraph(new Encoder(uniqueLocalAddress, system, compressionOut, outboundEnvelopePool, envelopePool)) + val encoder: Flow[OutboundEnvelope, EnvelopeBuffer, Encoder.ChangeOutboundCompression] = + Flow.fromGraph(new Encoder(uniqueLocalAddress, system, outboundEnvelopePool, envelopePool)) val localRecipient = resolvedRef.path.toSerializationFormatWithAddress(uniqueLocalAddress.address) val provider = RARP(system).provider 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 index 4297dbb171..b6e14f9465 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/compress/HeavyHittersBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/compress/HeavyHittersBenchmark.scala @@ -49,7 +49,7 @@ class HeavyHittersBenchmark { @Param(Array("8192")) var n: Int = 0 - var topN: TopHeavyHitters[String] = _ + private var topN: TopHeavyHitters[String] = _ val rand = new Random(1001021) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 6131cb2cc6..3c4c10d5ec 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -3,22 +3,30 @@ */ package akka.remote.artery +import java.io.File +import java.net.InetSocketAddress +import java.nio.channels.{ DatagramChannel, FileChannel } + import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicLong -import java.util.concurrent.TimeUnit.MICROSECONDS -import akka.remote.artery.compress.CompressionProtocol.CompressionMessage - +import scala.annotation.tailrec +import scala.collection.JavaConverters._ 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.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 @@ -30,13 +38,17 @@ import akka.remote.RemoteTransport import akka.remote.RemotingLifecycleEvent import akka.remote.ThisActorSystemQuarantinedEvent import akka.remote.UniqueAddress +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.artery.compress._ import akka.stream.AbruptTerminationException import akka.stream.ActorMaterializer +import akka.stream.ActorMaterializerSettings import akka.stream.KillSwitches import akka.stream.Materializer import akka.stream.SharedKillSwitch @@ -46,35 +58,19 @@ import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Source import akka.util.Helpers.ConfigOps import akka.util.Helpers.Requiring +import akka.util.OptionVal import akka.util.WildcardIndex import io.aeron.Aeron import io.aeron.AvailableImageHandler +import io.aeron.CncFileDescriptor import io.aeron.Image import io.aeron.UnavailableImageHandler import io.aeron.driver.MediaDriver +import io.aeron.driver.ThreadingMode import io.aeron.exceptions.ConductorServiceTimeoutException import org.agrona.ErrorHandler import org.agrona.IoUtil -import java.io.File -import java.net.InetSocketAddress -import java.nio.channels.{ DatagramChannel, FileChannel } - -import akka.remote.artery.OutboundControlJunction.OutboundControlIngress -import io.aeron.CncFileDescriptor -import java.util.concurrent.atomic.AtomicLong - -import scala.collection.JavaConverters._ -import akka.stream.ActorMaterializerSettings - -import scala.annotation.tailrec -import akka.util.OptionVal -import io.aeron.driver.ThreadingMode import org.agrona.concurrent.BackoffIdleStrategy -import org.agrona.concurrent.BusySpinIdleStrategy - -import scala.util.control.NonFatal -import akka.actor.Props -import akka.actor.Actor /** * INTERNAL API @@ -105,7 +101,7 @@ private[akka] trait InboundContext { */ def association(uid: Long): OptionVal[OutboundContext] - def completeHandshake(peer: UniqueAddress): Unit + def completeHandshake(peer: UniqueAddress): Future[Done] } @@ -117,8 +113,7 @@ private[akka] object AssociationState { new AssociationState( incarnation = 1, uniqueRemoteAddressPromise = Promise(), - quarantined = ImmutableLongMap.empty[QuarantinedTimestamp], - outboundCompressions = NoOutboundCompressions) + quarantined = ImmutableLongMap.empty[QuarantinedTimestamp]) final case class QuarantinedTimestamp(nanoTime: Long) { override def toString: String = @@ -130,10 +125,9 @@ private[akka] object AssociationState { * INTERNAL API */ private[akka] final class AssociationState( - val incarnation: Int, + val incarnation: Int, val uniqueRemoteAddressPromise: Promise[UniqueAddress], - val quarantined: ImmutableLongMap[AssociationState.QuarantinedTimestamp], - val outboundCompressions: OutboundCompressions) { + val quarantined: ImmutableLongMap[AssociationState.QuarantinedTimestamp]) { import AssociationState.QuarantinedTimestamp @@ -159,11 +153,8 @@ private[akka] final class AssociationState( } } - def withCompression(compression: OutboundCompressions) = - new AssociationState(incarnation, uniqueRemoteAddressPromise, quarantined, compression) - - def newIncarnation(remoteAddressPromise: Promise[UniqueAddress], compression: OutboundCompressions): AssociationState = - new AssociationState(incarnation + 1, remoteAddressPromise, quarantined, compression) + def newIncarnation(remoteAddressPromise: Promise[UniqueAddress]): AssociationState = + new AssociationState(incarnation + 1, remoteAddressPromise, quarantined) def newQuarantined(): AssociationState = uniqueRemoteAddressPromise.future.value match { @@ -171,8 +162,7 @@ private[akka] final class AssociationState( new AssociationState( incarnation, uniqueRemoteAddressPromise, - quarantined = quarantined.updated(a.uid, QuarantinedTimestamp(System.nanoTime())), - outboundCompressions = NoOutboundCompressions) // after quarantine no compression needed anymore, drop it + quarantined = quarantined.updated(a.uid, QuarantinedTimestamp(System.nanoTime()))) case _ ⇒ this } @@ -235,7 +225,7 @@ private[akka] trait OutboundContext { */ private[remote] object FlushOnShutdown { def props(done: Promise[Done], timeout: FiniteDuration, - inboundContext: InboundContext, associations: Set[Association]): Props = { + inboundContext: InboundContext, associations: Set[Association]): Props = { require(associations.nonEmpty) Props(new FlushOnShutdown(done, timeout, inboundContext, associations)) } @@ -247,7 +237,7 @@ private[remote] object FlushOnShutdown { * INTERNAL API */ private[remote] class FlushOnShutdown(done: Promise[Done], timeout: FiniteDuration, - inboundContext: InboundContext, associations: Set[Association]) extends Actor { + inboundContext: InboundContext, associations: Set[Association]) extends Actor { var remaining = associations.flatMap(_.associationState.uniqueRemoteAddressValue) @@ -330,7 +320,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val priorityMessageDestinations = WildcardIndex[NotUsed]() - // this comes from remoting so is semi-ok to be hardcoded here + // 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) @@ -579,17 +570,27 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R case ActorRefCompressionAdvertisement(from, table) ⇒ log.debug("Incoming ActorRef compression advertisement from [{}], table: [{}]", from, table) val a = association(from.address) - a.outboundCompression.applyActorRefCompressionTable(table) - a.sendControl(ActorRefCompressionAdvertisementAck(localAddress, table.version)) - system.eventStream.publish(Events.ReceivedActorRefCompressionTable(from, table)) + // 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)) + } + } case ActorRefCompressionAdvertisementAck(from, tableVersion) ⇒ inboundCompressions.foreach(_.confirmActorRefCompressionAdvertisement(from.uid, tableVersion)) case ClassManifestCompressionAdvertisement(from, table) ⇒ log.debug("Incoming Class Manifest compression advertisement from [{}], table: [{}]", from, table) val a = association(from.address) - a.outboundCompression.applyClassManifestCompressionTable(table) - a.sendControl(ClassManifestCompressionAdvertisementAck(localAddress, table.version)) - system.eventStream.publish(Events.ReceivedClassManifestCompressionTable(from, table)) + // 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)) + } + } case ClassManifestCompressionAdvertisementAck(from, tableVersion) ⇒ inboundCompressions.foreach(_.confirmActorRefCompressionAdvertisement(from.uid, tableVersion)) } @@ -719,8 +720,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R if (testStages.isEmpty) Future.successful(false) else { - import scala.collection.JavaConverters._ import system.dispatcher + import scala.collection.JavaConverters._ val allTestStages = testStages.asScala.toVector ++ associationRegistry.allAssociations.flatMap(_.testStages) Future.sequence(allTestStages.map(_.send(cmd))).map(_ ⇒ true) } @@ -752,7 +753,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R override def association(uid: Long): OptionVal[Association] = associationRegistry.association(uid) - override def completeHandshake(peer: UniqueAddress): Unit = { + override def completeHandshake(peer: UniqueAddress): Future[Done] = { val a = associationRegistry.setUID(peer) a.completeHandshake(peer) } @@ -765,19 +766,22 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R association(remoteAddress).quarantine(reason = "", uid.map(_.toLong)) } - def outbound(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[OutboundEnvelope, Future[Done]] = - createOutboundSink(ordinaryStreamId, outboundContext, compression, envelopePool) + def outbound(outboundContext: OutboundContext): Sink[OutboundEnvelope, (ChangeOutboundCompression, Future[Done])] = + createOutboundSink(ordinaryStreamId, outboundContext, envelopePool) - def outboundLarge(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[OutboundEnvelope, Future[Done]] = - createOutboundSink(largeStreamId, outboundContext, compression, largeEnvelopePool) + def outboundLarge(outboundContext: OutboundContext): Sink[OutboundEnvelope, Future[Done]] = + createOutboundSink(largeStreamId, outboundContext, largeEnvelopePool) + .mapMaterializedValue { case (_, d) ⇒ d } + + private def createOutboundSink(streamId: Int, outboundContext: OutboundContext, + bufferPool: EnvelopeBufferPool): Sink[OutboundEnvelope, (ChangeOutboundCompression, Future[Done])] = { - private def createOutboundSink(streamId: Int, outboundContext: OutboundContext, compression: OutboundCompressions, bufferPool: EnvelopeBufferPool): Sink[OutboundEnvelope, Future[Done]] = { Flow.fromGraph(killSwitch.flow[OutboundEnvelope]) .via(new OutboundHandshake(system, outboundContext, outboundEnvelopePool, handshakeTimeout, handshakeRetryInterval, injectHandshakeInterval)) - .via(createEncoder(bufferPool, compression)) + .viaMat(createEncoder(bufferPool))(Keep.right) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), streamId, aeron, taskRunner, - envelopePool, giveUpSendAfter, createFlightRecorderEventSink()))(Keep.right) + envelopePool, giveUpSendAfter, createFlightRecorderEventSink()))(Keep.both) } /** @@ -794,25 +798,20 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // FIXME we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages } - def outboundControlPart2(outboundContext: OutboundContext, compression: OutboundCompressions): Sink[OutboundEnvelope, (OutboundControlIngress, Future[Done])] = { + def outboundControlPart2(outboundContext: OutboundContext): Sink[OutboundEnvelope, (OutboundControlIngress, Future[Done])] = { Flow[OutboundEnvelope] .viaMat(new OutboundControlJunction(outboundContext, outboundEnvelopePool))(Keep.right) - .via(encoder(compression)) + .via(createEncoder(envelopePool)) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), controlStreamId, aeron, taskRunner, envelopePool, Duration.Inf, createFlightRecorderEventSink()))(Keep.both) } - def createEncoder(compression: OutboundCompressions, bufferPool: EnvelopeBufferPool): Flow[OutboundEnvelope, EnvelopeBuffer, NotUsed] = - Flow.fromGraph(new Encoder(localAddress, system, compression, outboundEnvelopePool, bufferPool)) - private def createInboundCompressions(inboundContext: InboundContext): InboundCompressions = if (remoteSettings.ArteryCompressionSettings.enabled) new InboundCompressionsImpl(system, inboundContext) else NoInboundCompressions - def createEncoder(pool: EnvelopeBufferPool, compression: OutboundCompressions): Flow[OutboundEnvelope, EnvelopeBuffer, NotUsed] = - Flow.fromGraph(new Encoder(localAddress, system, compression, outboundEnvelopePool, pool)) - - def encoder(compression: OutboundCompressions): Flow[OutboundEnvelope, EnvelopeBuffer, NotUsed] = createEncoder(envelopePool, compression) + def createEncoder(pool: EnvelopeBufferPool): Flow[OutboundEnvelope, EnvelopeBuffer, ChangeOutboundCompression] = + Flow.fromGraph(new Encoder(localAddress, system, outboundEnvelopePool, pool)) def aeronSource(streamId: Int, pool: EnvelopeBufferPool): Source[EnvelopeBuffer, NotUsed] = Source.fromGraph(new AeronSource(inboundChannel, streamId, aeron, taskRunner, pool, diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 6c3a29e908..82dc5bf01f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -4,41 +4,43 @@ package akka.remote.artery import java.util.Queue -import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.CountDownLatch import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicReference -import akka.remote.artery.compress.{ CompressionProtocol, CompressionTable, OutboundCompressions, OutboundCompressionsImpl } - import scala.annotation.tailrec import scala.concurrent.Future import scala.concurrent.Promise import scala.concurrent.duration._ import scala.concurrent.duration.FiniteDuration -import scala.util.Success + import akka.{ Done, NotUsed } import akka.actor.ActorRef import akka.actor.ActorSelectionMessage import akka.actor.Address -import akka.actor.RootActorPath import akka.dispatch.sysmsg.SystemMessage import akka.event.Logging import akka.remote._ +import akka.remote.DaemonMsgCreate +import akka.remote.QuarantinedEvent import akka.remote.artery.AeronSink.GaveUpSendingException +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.SendQueue.ProducerApi 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.Materializer import akka.stream.scaladsl.Keep import akka.stream.scaladsl.Source import akka.util.{ Unsafe, WildcardIndex } -import org.agrona.concurrent.ManyToOneConcurrentArrayQueue import akka.util.OptionVal +import org.agrona.concurrent.ManyToOneConcurrentArrayQueue import akka.remote.artery.compress.CompressionProtocol._ /** @@ -82,9 +84,6 @@ private[remote] class Association( // the `SendQueue` after materialization. Using same underlying queue. This makes it possible to // start sending (enqueuing) to the Association immediate after construction. - /** Accesses the currently active outbound compression. */ - def outboundCompression: OutboundCompressions = associationState.outboundCompressions - def createQueue(capacity: Int): Queue[OutboundEnvelope] = new ManyToOneConcurrentArrayQueue[OutboundEnvelope](capacity) @@ -93,6 +92,25 @@ private[remote] class Association( @volatile private[this] var controlQueue: SendQueue.ProducerApi[OutboundEnvelope] = QueueWrapper(createQueue(controlQueueSize)) @volatile private[this] var _outboundControlIngress: OutboundControlIngress = _ @volatile private[this] var materializing = new CountDownLatch(1) + @volatile private[this] var changeOutboundCompression: Option[ChangeOutboundCompression] = None + + def changeActorRefCompression(table: CompressionTable[ActorRef]): Future[Done] = + changeOutboundCompression match { + case Some(c) ⇒ c.changeActorRefCompression(table) + case None ⇒ Future.failed(new ChangeOutboundCompressionFailed) + } + + def changeClassManifestCompression(table: CompressionTable[String]): Future[Done] = + changeOutboundCompression match { + case Some(c) ⇒ c.changeClassManifestCompression(table) + case None ⇒ Future.failed(new ChangeOutboundCompressionFailed) + } + + def clearCompression(): Future[Done] = + changeOutboundCompression match { + case Some(c) ⇒ c.clearCompression() + case None ⇒ Future.failed(new ChangeOutboundCompressionFailed) + } private val _testStages: CopyOnWriteArrayList[TestManagementApi] = new CopyOnWriteArrayList @@ -138,31 +156,41 @@ private[remote] class Association( def associationState: AssociationState = Unsafe.instance.getObjectVolatile(this, AbstractAssociation.sharedStateOffset).asInstanceOf[AssociationState] - def completeHandshake(peer: UniqueAddress): Unit = { + 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.uniqueRemoteAddressPromise.trySuccess(peer) + current.uniqueRemoteAddressValue() match { case Some(`peer`) ⇒ - // our value - if (current.outboundCompressions == NoOutboundCompressions) { - // enable outbound compression (here, since earlier we don't know the remote address) - swapState(current, current.withCompression(createOutboundCompressions(remoteAddress))) - } + // handshake already completed + Future.successful(Done) case _ ⇒ - val newState = current.newIncarnation(Promise.successful(peer), createOutboundCompressions(remoteAddress)) - if (swapState(current, newState)) { + // 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 + clearCompression().map { _ ⇒ + current.uniqueRemoteAddressPromise.trySuccess(peer) current.uniqueRemoteAddressValue() match { - case Some(old) ⇒ - log.debug( - "Incarnation {} of association to [{}] with new UID [{}] (old UID [{}])", - newState.incarnation, peer.address, peer.uid, old.uid) - case None ⇒ - // Failed, nothing to do + case Some(`peer`) ⇒ + // our value + case _ ⇒ + val newState = current.newIncarnation(Promise.successful(peer)) + if (swapState(current, newState)) { + current.uniqueRemoteAddressValue() match { + case Some(old) ⇒ + log.debug( + "Incarnation {} of association to [{}] with new UID [{}] (old UID [{}])", + newState.incarnation, peer.address, peer.uid, old.uid) + case None ⇒ + // Failed, nothing to do + } + // if swap failed someone else completed before us, and that is fine + } } - // if swap failed someone else completed before us, and that is fine + Done } } } @@ -253,6 +281,8 @@ private[remote] class Association( log.warning( "Association to [{}] with UID [{}] is irrecoverably failed. Quarantining address. {}", remoteAddress, u, reason) + // clear outbound compression + clearCompression() // FIXME when we complete the switch to Long UID we must use Long here also, issue #20644 transport.eventPublisher.notifyListeners(QuarantinedEvent(remoteAddress, u.toInt)) // end delivery of system messages to that incarnation after this point @@ -291,20 +321,16 @@ private[remote] class Association( } private def runOutboundStreams(): Unit = { - // TODO no compression for control / large streams currently - val disableCompression = NoOutboundCompressions - // it's important to materialize the outboundControl stream first, // so that outboundControlIngress is ready when stages for all streams start - runOutboundControlStream(disableCompression) - runOutboundOrdinaryMessagesStream(CurrentAssociationStateOutboundCompressionsProxy) + runOutboundControlStream() + runOutboundOrdinaryMessagesStream() - if (transport.largeMessageChannelEnabled) { - runOutboundLargeMessagesStream(disableCompression) - } + if (transport.largeMessageChannelEnabled) + runOutboundLargeMessagesStream() } - private def runOutboundControlStream(compression: OutboundCompressions): Unit = { + private def runOutboundControlStream(): Unit = { // stage in the control stream may access the outboundControlIngress before returned here // using CountDownLatch to make sure that materialization is completed before accessing outboundControlIngress materializing = new CountDownLatch(1) @@ -318,14 +344,14 @@ private[remote] class Association( Source.fromGraph(new SendQueue[OutboundEnvelope]) .via(transport.outboundControlPart1(this)) .viaMat(transport.outboundTestFlow(this))(Keep.both) - .toMat(transport.outboundControlPart2(this, compression))(Keep.both) + .toMat(transport.outboundControlPart2(this))(Keep.both) .run()(materializer) _testStages.add(mgmt) (queueValue, (control, completed)) } else { Source.fromGraph(new SendQueue[OutboundEnvelope]) .via(transport.outboundControlPart1(this)) - .toMat(transport.outboundControlPart2(this, compression))(Keep.both) + .toMat(transport.outboundControlPart2(this))(Keep.both) .run()(materializer) } @@ -335,7 +361,7 @@ private[remote] class Association( _outboundControlIngress = control materializing.countDown() attachStreamRestart("Outbound control stream", completed, cause ⇒ { - runOutboundControlStream(compression) + runOutboundControlStream() cause match { case _: HandshakeTimeoutException ⇒ // ok, quarantine not possible without UID case _ ⇒ quarantine("Outbound control stream restarted") @@ -351,32 +377,33 @@ private[remote] class Association( QueueWrapper(createQueue(capacity)) } - private def runOutboundOrdinaryMessagesStream(compression: OutboundCompressions): Unit = { + private def runOutboundOrdinaryMessagesStream(): Unit = { val wrapper = getOrCreateQueueWrapper(queue, queueSize) queue = wrapper // use new underlying queue immediately for restarts - val (queueValue, completed) = + val (queueValue, (changeCompression, completed)) = if (transport.remoteSettings.TestMode) { val ((queueValue, mgmt), completed) = Source.fromGraph(new SendQueue[OutboundEnvelope]) .viaMat(transport.outboundTestFlow(this))(Keep.both) - .toMat(transport.outbound(this, compression))(Keep.both) + .toMat(transport.outbound(this))(Keep.both) .run()(materializer) _testStages.add(mgmt) (queueValue, completed) } else { Source.fromGraph(new SendQueue[OutboundEnvelope]) - .toMat(transport.outbound(this, compression))(Keep.both) + .toMat(transport.outbound(this))(Keep.both) .run()(materializer) } queueValue.inject(wrapper.queue) // replace with the materialized value, still same underlying queue queue = queueValue + changeOutboundCompression = Some(changeCompression) - attachStreamRestart("Outbound message stream", completed, _ ⇒ runOutboundOrdinaryMessagesStream(compression)) + attachStreamRestart("Outbound message stream", completed, _ ⇒ runOutboundOrdinaryMessagesStream()) } - private def runOutboundLargeMessagesStream(compression: OutboundCompressions): Unit = { + private def runOutboundLargeMessagesStream(): Unit = { val wrapper = getOrCreateQueueWrapper(largeQueue, largeQueueSize) largeQueue = wrapper // use new underlying queue immediately for restarts @@ -384,20 +411,20 @@ private[remote] class Association( if (transport.remoteSettings.TestMode) { val ((queueValue, mgmt), completed) = Source.fromGraph(new SendQueue[OutboundEnvelope]) .viaMat(transport.outboundTestFlow(this))(Keep.both) - .toMat(transport.outboundLarge(this, compression))(Keep.both) + .toMat(transport.outboundLarge(this))(Keep.both) .run()(materializer) _testStages.add(mgmt) (queueValue, completed) } else { Source.fromGraph(new SendQueue[OutboundEnvelope]) - .toMat(transport.outboundLarge(this, compression))(Keep.both) + .toMat(transport.outboundLarge(this))(Keep.both) .run()(materializer) } queueValue.inject(wrapper.queue) // replace with the materialized value, still same underlying queue largeQueue = queueValue - attachStreamRestart("Outbound large message stream", completed, _ ⇒ runOutboundLargeMessagesStream(compression)) + attachStreamRestart("Outbound large message stream", completed, _ ⇒ runOutboundLargeMessagesStream()) } private def attachStreamRestart(streamName: String, streamCompleted: Future[Done], restart: Throwable ⇒ Unit): Unit = { @@ -421,41 +448,6 @@ private[remote] class Association( } } - // TODO: Make sure that once other channels use Compression, each gets it's own - private def createOutboundCompressions(remoteAddress: Address): OutboundCompressions = { - if (transport.provider.remoteSettings.ArteryCompressionSettings.enabled) { - val compression = new OutboundCompressionsImpl(transport.system, remoteAddress) - log.debug("Creating Outbound compression table to [{}]", remoteAddress) - compression - } else NoOutboundCompressions - } - - /** - * This proxy uses the current associationStates compression table, which is reset for a new incarnation. - * This way the same outgoing stream will switch to using the new table without the need of restarting it. - */ - private object CurrentAssociationStateOutboundCompressionsProxy extends OutboundCompressions { - - override def actorRefCompressionTableVersion: Int = - associationState.outboundCompressions.actorRefCompressionTableVersion - override def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit = { - associationState.outboundCompressions.applyActorRefCompressionTable(table) - } - override final def compressActorRef(ref: ActorRef): Int = { - associationState.outboundCompressions.compressActorRef(ref) - } - - override def classManifestCompressionTableVersion: Int = - associationState.outboundCompressions.classManifestCompressionTableVersion - override def applyClassManifestCompressionTable(table: CompressionTable[String]): Unit = - associationState.outboundCompressions.applyClassManifestCompressionTable(table) - override final def compressClassManifest(manifest: String): Int = - associationState.outboundCompressions.compressClassManifest(manifest) - - override def toString = - s"${Logging.simpleName(getClass)}(current delegate: ${associationState.outboundCompressions})" - } - override def toString: String = s"Association($localAddress -> $remoteAddress with $associationState)" @@ -489,17 +481,23 @@ private[remote] class AssociationRegistry(createAssociation: Address ⇒ Associa @tailrec final def setUID(peer: UniqueAddress): Association = { val currentMap = associationsByUid.get val a = association(peer.address) - // make sure we don't overwrite same UID with different association + currentMap.get(peer.uid) match { - case OptionVal.Some(previous) if (previous ne a) ⇒ - throw new IllegalArgumentException(s"UID collision old [$previous] new [$a]") - case _ ⇒ // ok + 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 } - val newMap = currentMap.updated(peer.uid, a) - if (associationsByUid.compareAndSet(currentMap, newMap)) - a - else - setUID(peer) // lost CAS, retry } def allAssociations: Set[Association] = diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index 0b4bfbf558..dc07002d35 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -9,12 +9,13 @@ import java.nio.{ ByteBuffer, ByteOrder } import akka.actor.{ ActorRef, Address } import akka.remote.artery.compress.CompressionProtocol._ -import akka.remote.artery.compress.{ CompressionTable, InboundCompressions, OutboundCompressions } +import akka.remote.artery.compress.{ CompressionTable, InboundCompressions } import akka.serialization.Serialization import org.agrona.concurrent.{ ManyToManyConcurrentArrayQueue, UnsafeBuffer } import akka.util.{ OptionVal, Unsafe } import scala.util.control.NonFatal +import akka.remote.artery.compress.NoInboundCompressions /** * INTERNAL API @@ -77,25 +78,34 @@ 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, NoOutboundCompressions) - def out(compression: OutboundCompressions): HeaderBuilder = new HeaderBuilderImpl(NoInboundCompressions, compression) + 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]) /** INTERNAL API, FOR TESTING ONLY */ - private[remote] def bothWays(in: InboundCompressions, out: OutboundCompressions): HeaderBuilder = new HeaderBuilderImpl(in, out) + private[remote] def bothWays( + in: InboundCompressions, + outboundActorRefCompression: CompressionTable[ActorRef], + outboundClassManifestCompression: CompressionTable[String]): HeaderBuilder = + new HeaderBuilderImpl(in, outboundActorRefCompression, outboundClassManifestCompression) } /** * INTERNAL API */ -sealed trait HeaderBuilder { +private[remote] sealed trait HeaderBuilder { def setVersion(v: Int): Unit def version: Int - def setActorRefCompressionTableVersion(v: Int): Unit - def actorRefCompressionTableVersion: Int + def inboundActorRefCompressionTableVersion: Int + def inboundClassManifestCompressionTableVersion: Int - def setClassManifestCompressionTableVersion(v: Int): Unit - def classManifestCompressionTableVersion: Int + 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 @@ -140,12 +150,15 @@ sealed trait HeaderBuilder { /** * INTERNAL API */ -private[remote] final class HeaderBuilderImpl(inboundCompression: InboundCompressions, outboundCompression: OutboundCompressions) extends HeaderBuilder { +private[remote] final class HeaderBuilderImpl( + inboundCompression: InboundCompressions, + var _outboundActorRefCompression: CompressionTable[ActorRef], + var _outboundClassManifestCompression: CompressionTable[String]) extends HeaderBuilder { // Fields only available for EnvelopeBuffer var _version: Int = _ var _uid: Long = _ - var _actorRefCompressionTableVersion: Int = 0 - var _classManifestCompressionTableVersion: Int = 0 + var _inboundActorRefCompressionTableVersion: Int = 0 + var _inboundClassManifestCompressionTableVersion: Int = 0 var _senderActorRef: String = null var _senderActorRefIdx: Int = -1 @@ -162,14 +175,19 @@ private[remote] final class HeaderBuilderImpl(inboundCompression: InboundCompres override def setUid(uid: Long) = _uid = uid override def uid: Long = _uid - override def setActorRefCompressionTableVersion(v: Int): Unit = _actorRefCompressionTableVersion = v - override def actorRefCompressionTableVersion: Int = _actorRefCompressionTableVersion + override def inboundActorRefCompressionTableVersion: Int = _inboundActorRefCompressionTableVersion + override def inboundClassManifestCompressionTableVersion: Int = _inboundClassManifestCompressionTableVersion - override def setClassManifestCompressionTableVersion(v: Int): Unit = _classManifestCompressionTableVersion = v - override def classManifestCompressionTableVersion: Int = _classManifestCompressionTableVersion + 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 = { - _senderActorRefIdx = outboundCompression.compressActorRef(ref) + _senderActorRefIdx = outboundActorRefCompression.compress(ref) if (_senderActorRefIdx == -1) _senderActorRef = Serialization.serializedActorPath(ref) // includes local address from `currentTransportInformation` } override def setNoSender(): Unit = { @@ -179,7 +197,8 @@ private[remote] final class HeaderBuilderImpl(inboundCompression: InboundCompres override def isNoSender: Boolean = (_senderActorRef eq null) && _senderActorRefIdx == EnvelopeBuffer.DeadLettersCode override def senderActorRef(originUid: Long): OptionVal[ActorRef] = - if (_senderActorRef eq null) inboundCompression.decompressActorRef(originUid, actorRefCompressionTableVersion, _senderActorRefIdx) + if (_senderActorRef eq null) + inboundCompression.decompressActorRef(originUid, inboundActorRefCompressionTableVersion, _senderActorRefIdx) else OptionVal.None def senderActorRefPath: OptionVal[String] = OptionVal(_senderActorRef) @@ -192,13 +211,14 @@ private[remote] final class HeaderBuilderImpl(inboundCompression: InboundCompres (_recipientActorRef eq null) && _recipientActorRefIdx == EnvelopeBuffer.DeadLettersCode def setRecipientActorRef(ref: ActorRef): Unit = { - _recipientActorRefIdx = outboundCompression.compressActorRef(ref) + _recipientActorRefIdx = outboundActorRefCompression.compress(ref) if (_recipientActorRefIdx == -1) { _recipientActorRef = ref.path.toSerializationFormat } } def recipientActorRef(originUid: Long): OptionVal[ActorRef] = - if (_recipientActorRef eq null) inboundCompression.decompressActorRef(originUid, actorRefCompressionTableVersion, _recipientActorRefIdx) + if (_recipientActorRef eq null) + inboundCompression.decompressActorRef(originUid, inboundActorRefCompressionTableVersion, _recipientActorRefIdx) else OptionVal.None def recipientActorRefPath: OptionVal[String] = OptionVal(_recipientActorRef) @@ -210,13 +230,15 @@ private[remote] final class HeaderBuilderImpl(inboundCompression: InboundCompres _serializer override def setManifest(manifest: String): Unit = { - _manifestIdx = outboundCompression.compressClassManifest(manifest) + _manifestIdx = outboundClassManifestCompression.compress(manifest) if (_manifestIdx == -1) _manifest = manifest } override def manifest(originUid: Long): String = { if (_manifest ne null) _manifest else { - _manifest = inboundCompression.decompressClassManifest(originUid, classManifestCompressionTableVersion, _manifestIdx).get + _manifest = inboundCompression.decompressClassManifest( + originUid, + inboundClassManifestCompressionTableVersion, _manifestIdx).get _manifest } } @@ -224,8 +246,6 @@ private[remote] final class HeaderBuilderImpl(inboundCompression: InboundCompres override def toString = "HeaderBuilderImpl(" + "version:" + version + ", " + - "actorRefCompressionTableVersion:" + actorRefCompressionTableVersion + ", " + - "classManifestCompressionTableVersion:" + classManifestCompressionTableVersion + ", " + "uid:" + uid + ", " + "_senderActorRef:" + _senderActorRef + ", " + "_senderActorRefIdx:" + _senderActorRefIdx + ", " + @@ -257,8 +277,8 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { byteBuffer.putInt(header.serializer) // compression table version numbers - byteBuffer.putInt(ActorRefCompressionTableVersionTagOffset, header._actorRefCompressionTableVersion | TagTypeMask) - byteBuffer.putInt(ClassManifestCompressionTableVersionTagOffset, header._classManifestCompressionTableVersion | TagTypeMask) + byteBuffer.putInt(ActorRefCompressionTableVersionTagOffset, header.outboundActorRefCompression.version | TagTypeMask) + byteBuffer.putInt(ClassManifestCompressionTableVersionTagOffset, header.outboundClassManifestCompression.version | TagTypeMask) // Write compressable, variable-length parts always to the actual position of the buffer // Write tag values explicitly in their proper offset @@ -294,11 +314,11 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { // compression table versions (stored in the Tag) val refCompressionVersionTag = byteBuffer.getInt(ActorRefCompressionTableVersionTagOffset) if ((refCompressionVersionTag & TagTypeMask) != 0) { - header setActorRefCompressionTableVersion refCompressionVersionTag & TagValueMask + header._inboundActorRefCompressionTableVersion = refCompressionVersionTag & TagValueMask } val manifestCompressionVersionTag = byteBuffer.getInt(ClassManifestCompressionTableVersionTagOffset) if ((manifestCompressionVersionTag & TagTypeMask) != 0) { - header setClassManifestCompressionTableVersion manifestCompressionVersionTag & TagValueMask + header._inboundClassManifestCompressionTableVersion = manifestCompressionVersionTag & TagValueMask } // Read compressable, variable-length parts always from the actual position of the buffer diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 6c9a1149dc..5c2243d97b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -13,9 +13,28 @@ import akka.stream._ import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler } import akka.util.{ ByteString, OptionVal, PrettyByteString } import akka.actor.EmptyLocalActorRef -import akka.remote.artery.compress.{ InboundCompressions, OutboundCompressions, OutboundCompressionsImpl } +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 + +/** + * 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 @@ -23,42 +42,49 @@ import java.util.concurrent.TimeUnit private[remote] class Encoder( uniqueLocalAddress: UniqueAddress, system: ActorSystem, - compression: OutboundCompressions, outboundEnvelopePool: ObjectPool[ReusableOutboundEnvelope], bufferPool: EnvelopeBufferPool) - extends GraphStage[FlowShape[OutboundEnvelope, EnvelopeBuffer]] { + 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 createLogic(inheritedAttributes: Attributes): GraphStageLogic = - new GraphStageLogic(shape) with InHandler with OutHandler with StageLogging { + 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(compression) + 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 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] override def onPush(): Unit = { val outboundEnvelope = grab(in) val envelope = bufferPool.acquire() - // FIXME: OMG race between setting the version, and using the table!!!! - // incoming messages are concurrent to outgoing ones - // incoming message may be table advertisement - // which swaps the table in Outgoing*Compression for the new one (n+1) - // by itself it does so atomically, - // race: however here we store the compression table version separately from actually using it (storing the refs / manifests etc). - // so there is a slight race IF the table is swapped right between us setting the version n here [then the table being swapped to n+1] and then we use the n+1 version to compressions the compressions (which the receiving end will fail to read, since the encoding could be completely different, and it picks the table based on the version Int). - // A solution would be to getTable => use it to set and compress things - headerBuilder setActorRefCompressionTableVersion compression.actorRefCompressionTableVersion - headerBuilder setClassManifestCompressionTableVersion compression.classManifestCompressionTableVersion - // internally compression is applied by the builder: outboundEnvelope.recipient match { case OptionVal.Some(r) ⇒ headerBuilder setRecipientActorRef r @@ -109,8 +135,49 @@ private[remote] class Encoder( override def onPull(): Unit = pull(in) + /** + * 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) + } } /** @@ -198,20 +265,17 @@ private[remote] class Decoder( val remoteAddress = assoc.remoteAddress sender match { case OptionVal.Some(snd) ⇒ - compression.hitActorRef(originUid, headerBuilder.actorRefCompressionTableVersion, - remoteAddress, snd, 1) + compression.hitActorRef(originUid, remoteAddress, snd, 1) case OptionVal.None ⇒ } recipient match { case OptionVal.Some(rcp) ⇒ - compression.hitActorRef(originUid, headerBuilder.actorRefCompressionTableVersion, - remoteAddress, rcp, 1) + compression.hitActorRef(originUid, remoteAddress, rcp, 1) case OptionVal.None ⇒ } - compression.hitClassManifest(originUid, headerBuilder.classManifestCompressionTableVersion, - remoteAddress, classManifest, 1) + compression.hitClassManifest(originUid, remoteAddress, classManifest, 1) case _ ⇒ // we don't want to record hits for compression while handshake is still in progress. diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala index a7c8420377..5f4a50a3b4 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -18,6 +18,8 @@ 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 /** * INTERNAL API @@ -177,8 +179,9 @@ private[akka] class InboundHandshake(inboundContext: InboundContext, inControlSt env.message match { case HandshakeReq(from) ⇒ onHandshakeReq(from) case HandshakeRsp(from) ⇒ - inboundContext.completeHandshake(from) - pull(in) + after(inboundContext.completeHandshake(from)) { + pull(in) + } case _ ⇒ onMessage(env) } @@ -197,21 +200,32 @@ private[akka] class InboundHandshake(inboundContext: InboundContext, inControlSt }) private def onHandshakeReq(from: UniqueAddress): Unit = { - inboundContext.completeHandshake(from) - inboundContext.sendControl(from.address, HandshakeRsp(inboundContext.localAddress)) - pull(in) + after(inboundContext.completeHandshake(from)) { + inboundContext.sendControl(from.address, HandshakeRsp(inboundContext.localAddress)) + 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 { - // FIXME remove, only debug - log.warning( - 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, inboundContext.localAddress.uid, env.originUid) if (log.isDebugEnabled) log.debug( s"Dropping message [{}] from unknown system with UID [{}]. " + @@ -222,8 +236,12 @@ private[akka] class InboundHandshake(inboundContext: InboundContext, inControlSt } } - private def isKnownOrigin(env: InboundEnvelope): Boolean = - env.association.isDefined + 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) diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressions.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressions.scala deleted file mode 100644 index fdbc067d61..0000000000 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/ActualCompressions.scala +++ /dev/null @@ -1,98 +0,0 @@ -/** - * Copyright (C) 2016 Lightbend Inc. - */ - -package akka.remote.artery.compress - -import java.util.function.LongFunction - -import akka.actor.{ ActorRef, ActorSystem, Address } -import akka.remote.artery._ -import akka.util.OptionVal -import org.agrona.collections.Long2ObjectHashMap - -/** INTERNAL API */ -private[remote] final class OutboundCompressionsImpl(system: ActorSystem, remoteAddress: Address) extends OutboundCompressions { - - private val actorRefsOut = new OutboundActorRefCompression(system, remoteAddress) - private val classManifestsOut = new OutboundClassManifestCompression(system, remoteAddress) - - // actor ref compression --- - - override def compressActorRef(ref: ActorRef): Int = actorRefsOut.compress(ref) - override def actorRefCompressionTableVersion: Int = actorRefsOut.activeCompressionTableVersion - override def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit = - actorRefsOut.flipTable(table) - - // class manifest compression --- - - override def compressClassManifest(manifest: String): Int = classManifestsOut.compress(manifest) - override def classManifestCompressionTableVersion: Int = classManifestsOut.activeCompressionTableVersion - override def applyClassManifestCompressionTable(table: CompressionTable[String]): Unit = - classManifestsOut.flipTable(table) -} - -/** - * 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) extends InboundCompressions { - - private val settings = CompressionSettings(system) - - // FIXME we also must remove the ones that won't be used anymore - when quarantine triggers - private[this] val _actorRefsIns = new Long2ObjectHashMap[InboundActorRefCompression]() - private val createInboundActorRefsForOrigin = new LongFunction[InboundActorRefCompression] { - override def apply(originUid: Long): InboundActorRefCompression = { - val actorRefHitters = new TopHeavyHitters[ActorRef](settings.actorRefs.max) - new InboundActorRefCompression(system, settings, originUid, inboundContext, actorRefHitters) - } - } - private def actorRefsIn(originUid: Long): InboundActorRefCompression = - _actorRefsIns.computeIfAbsent(originUid, createInboundActorRefsForOrigin) - - private[this] val _classManifestsIns = new Long2ObjectHashMap[InboundManifestCompression]() - private val createInboundManifestsForOrigin = new LongFunction[InboundManifestCompression] { - override def apply(originUid: Long): InboundManifestCompression = { - val manifestHitters = new TopHeavyHitters[String](settings.manifests.max) - new InboundManifestCompression(system, settings, originUid, inboundContext, manifestHitters) - } - } - private def classManifestsIn(originUid: Long): InboundManifestCompression = - _classManifestsIns.computeIfAbsent(originUid, createInboundManifestsForOrigin) - - // actor ref compression --- - - override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = - actorRefsIn(originUid).decompress(tableVersion, idx) - override def hitActorRef(originUid: Long, tableVersion: Int, address: Address, ref: ActorRef, n: Int): Unit = - actorRefsIn(originUid).increment(address, ref, n) - override def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = - actorRefsIn(originUid).confirmAdvertisement(tableVersion) - - // class manifest compression --- - - override def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] = - classManifestsIn(originUid).decompress(tableVersion, idx) - override def hitClassManifest(originUid: Long, tableVersion: Int, address: Address, manifest: String, n: Int): Unit = - classManifestsIn(originUid).increment(address, manifest, n) - override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = - actorRefsIn(originUid).confirmAdvertisement(tableVersion) - - // testing utilities --- - - /** INTERNAL API: for testing only */ - private[remote] def runNextActorRefAdvertisement() = { - import scala.collection.JavaConverters._ - _actorRefsIns.values().asScala.foreach { inbound ⇒ inbound.runNextTableAdvertisement() } - } - - /** INTERNAL API: for testing only */ - private[remote] def runNextClassManifestAdvertisement() = { - import scala.collection.JavaConverters._ - _classManifestsIns.values().asScala.foreach { inbound ⇒ inbound.runNextTableAdvertisement() } - } -} diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/AllCompressions.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/AllCompressions.scala deleted file mode 100644 index e90db8bd31..0000000000 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/AllCompressions.scala +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Copyright (C) 2016 Lightbend Inc. - */ - -package akka.remote.artery.compress - -import akka.actor.{ ActorRef, Address } -import akka.util.OptionVal - -/** - * 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, tableVersion: Int, remote: Address, ref: ActorRef, n: Int): Unit - def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] - def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit - - def hitClassManifest(originUid: Long, tableVersion: Int, remote: Address, manifest: String, n: Int): Unit - def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] - def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit -} -/** - * INTERNAL API - * Compress outgoing data and handle compression advertisements to fill compression table. - * - * One per outgoing message stream. - */ -private[remote] trait OutboundCompressions { - def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit - def actorRefCompressionTableVersion: Int - def compressActorRef(ref: ActorRef): Int - - def applyClassManifestCompressionTable(table: CompressionTable[String]): Unit - def classManifestCompressionTableVersion: Int - def compressClassManifest(manifest: String): Int -} 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 index dfc612868b..d07866e239 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala @@ -6,6 +6,13 @@ package akka.remote.artery.compress /** INTERNAL API: Versioned compression table to be advertised between systems */ private[akka] final case class CompressionTable[T](version: Int, map: Map[T, Int]) { + import CompressionTable.NotCompressedId + + def compress(value: T): Int = + map.get(value) match { + case Some(id) ⇒ id + case None ⇒ NotCompressedId + } def invert: DecompressionTable[T] = if (map.isEmpty) DecompressionTable.empty[T].copy(version = version) @@ -25,6 +32,8 @@ private[akka] final case class CompressionTable[T](version: Int, map: Map[T, Int } /** INTERNAL API */ private[remote] object CompressionTable { + final val NotCompressedId = -1 + private[this] val _empty = new CompressionTable[Any](0, Map.empty) def empty[T] = _empty.asInstanceOf[CompressionTable[T]] } 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 index 7fd5b5bfd4..5be9354a32 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala @@ -4,12 +4,97 @@ 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, NoLogging } import akka.remote.artery.{ InboundContext, OutboundContext } import akka.util.{ OptionVal, PrettyDuration } -import scala.concurrent.duration.{ Duration, FiniteDuration } -import java.util.concurrent.atomic.AtomicReference +import org.agrona.collections.Long2ObjectHashMap + +/** + * 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: Int, idx: Int): OptionVal[ActorRef] + def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit + + def hitClassManifest(originUid: Long, remote: Address, manifest: String, n: Int): Unit + def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] + def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Int): 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) extends InboundCompressions { + + private val settings = CompressionSettings(system) + + // FIXME we also must remove the ones that won't be used anymore - when quarantine triggers + private[this] val _actorRefsIns = new Long2ObjectHashMap[InboundActorRefCompression]() + private val createInboundActorRefsForOrigin = new LongFunction[InboundActorRefCompression] { + override def apply(originUid: Long): InboundActorRefCompression = { + val actorRefHitters = new TopHeavyHitters[ActorRef](settings.actorRefs.max) + new InboundActorRefCompression(system, settings, originUid, inboundContext, actorRefHitters) + } + } + private def actorRefsIn(originUid: Long): InboundActorRefCompression = + _actorRefsIns.computeIfAbsent(originUid, createInboundActorRefsForOrigin) + + private[this] val _classManifestsIns = new Long2ObjectHashMap[InboundManifestCompression]() + private val createInboundManifestsForOrigin = new LongFunction[InboundManifestCompression] { + override def apply(originUid: Long): InboundManifestCompression = { + val manifestHitters = new TopHeavyHitters[String](settings.manifests.max) + new InboundManifestCompression(system, settings, originUid, inboundContext, manifestHitters) + } + } + private def classManifestsIn(originUid: Long): InboundManifestCompression = + _classManifestsIns.computeIfAbsent(originUid, createInboundManifestsForOrigin) + + // actor ref compression --- + + override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = + actorRefsIn(originUid).decompress(tableVersion, idx) + override def hitActorRef(originUid: Long, address: Address, ref: ActorRef, n: Int): Unit = + actorRefsIn(originUid).increment(address, ref, n) + override def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = + actorRefsIn(originUid).confirmAdvertisement(tableVersion) + + // class manifest compression --- + + override def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] = + classManifestsIn(originUid).decompress(tableVersion, idx) + override def hitClassManifest(originUid: Long, address: Address, manifest: String, n: Int): Unit = + classManifestsIn(originUid).increment(address, manifest, n) + override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = + actorRefsIn(originUid).confirmAdvertisement(tableVersion) + + // testing utilities --- + + /** INTERNAL API: for testing only */ + private[remote] def runNextActorRefAdvertisement() = { + import scala.collection.JavaConverters._ + _actorRefsIns.values().asScala.foreach { inbound ⇒ inbound.runNextTableAdvertisement() } + } + + /** INTERNAL API: for testing only */ + private[remote] def runNextClassManifestAdvertisement() = { + import scala.collection.JavaConverters._ + _classManifestsIns.values().asScala.foreach { inbound ⇒ inbound.runNextTableAdvertisement() } + } +} /** * INTERNAL API @@ -105,7 +190,7 @@ private[remote] abstract class InboundCompression[T >: Null]( lazy val log = Logging(system, getClass.getSimpleName) - // TODO NOTE: there exist edge cases around, we advertise table 1, accumulate table 2, the remote system has not used 2 yet, + // FIXME NOTE: there exist edge cases around, we advertise table 1, accumulate table 2, the remote system has not used 2 yet, // yet we technically could already prepare table 3, then it starts using table 1 suddenly. Edge cases like that. // SOLUTION 1: We don't start building new tables until we've seen the previous one be used (move from new to active) // This is nice as it practically disables all the "build the table" work when the other side is not interested in using it. diff --git a/akka-remote/src/main/scala/akka/remote/artery/NoLiteralCompression.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/NoInboundCompressions.scala similarity index 51% rename from akka-remote/src/main/scala/akka/remote/artery/NoLiteralCompression.scala rename to akka-remote/src/main/scala/akka/remote/artery/compress/NoInboundCompressions.scala index 63886945d5..7219cb0cfa 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/NoLiteralCompression.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/NoInboundCompressions.scala @@ -1,10 +1,9 @@ /** * Copyright (C) 2016 Lightbend Inc. */ -package akka.remote.artery +package akka.remote.artery.compress import akka.actor.{ ActorRef, Address } -import akka.remote.artery.compress.{ CompressionTable, InboundCompressions, OutboundCompressions } import akka.util.OptionVal /** @@ -13,30 +12,16 @@ import akka.util.OptionVal * Literarily, no compression! */ case object NoInboundCompressions extends InboundCompressions { - override def hitActorRef(originUid: Long, tableVersion: Int, remote: Address, ref: ActorRef, n: Int): Unit = () + override def hitActorRef(originUid: Long, remote: Address, ref: ActorRef, n: Int): Unit = () override def decompressActorRef(originUid: Long, tableVersion: Int, 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: Int): Unit = () - override def hitClassManifest(originUid: Long, tableVersion: Int, remote: Address, manifest: String, n: Int): Unit = () + override def hitClassManifest(originUid: Long, remote: Address, manifest: String, n: Int): Unit = () override def decompressClassManifest(originUid: Long, tableVersion: Int, 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: Int): Unit = () } -/** - * INTERNAL API - * - * Literarily, no compression! - */ -case object NoOutboundCompressions extends OutboundCompressions { - override def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit = () - override def actorRefCompressionTableVersion: Int = 0 - override def compressActorRef(ref: ActorRef): Int = -1 - - override def applyClassManifestCompressionTable(table: CompressionTable[String]): Unit = () - override def classManifestCompressionTableVersion: Int = 0 - override def compressClassManifest(manifest: String): Int = -1 -} diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompression.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompression.scala deleted file mode 100644 index 59756afe32..0000000000 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/OutboundActorRefCompression.scala +++ /dev/null @@ -1,114 +0,0 @@ -/* - * Copyright (C) 2016 Lightbend Inc. - */ - -package akka.remote.artery.compress - -import java.util.concurrent.atomic.AtomicReference -import java.{ util ⇒ ju } - -import akka.actor.{ ActorRef, ActorSystem, Address } -import akka.event.{ Logging, LoggingAdapter } -import akka.remote.artery.compress.OutboundCompression.OutboundCompressionState - -import scala.annotation.tailrec - -/** INTERNAL API */ -private[remote] final class OutboundActorRefCompression(system: ActorSystem, remoteAddress: Address) - extends OutboundCompressionTable[ActorRef](system, remoteAddress) { - - flipTable(CompressionTable( - version = 0, - map = Map( - system.deadLetters → 0))) -} - -/** INTERNAL API */ -private[remote] final class OutboundClassManifestCompression(system: ActorSystem, remoteAddress: Address) - extends OutboundCompressionTable[String](system, remoteAddress) { - - flipTable(CompressionTable(version = 0, Map.empty)) -} - -/** - * INTERNAL API - * Base class for all outgoing compression. - * Encapsulates the compressedId registration and lookup. - */ -private[remote] class OutboundCompressionTable[T](system: ActorSystem, remoteAddress: Address) - extends AtomicReference[OutboundCompressionState[T]](OutboundCompressionState.initial) { // TODO could be instead via Unsafe - import OutboundCompression._ - - // TODO: The compression map may benefit from padding if we want multiple compressions to be running in parallel - - protected val log: LoggingAdapter = Logging(system, Logging.simpleName(getClass)) - - // TODO this exposes us to a race between setting the Version and USING the table...? - def activeCompressionTableVersion = { - val version = get.version - version - } - - /** - * Flips the currently used compression table to the new one (iff the new one has a version number higher than the currently used one). - */ - // (╯°□°)╯︵ ┻━┻ - @tailrec final def flipTable(activate: CompressionTable[T]): Unit = { - val state = get() - if (activate.version > state.version) // TODO this should handle roll-over as we move to Byte - if (compareAndSet(state, prepareState(activate))) - log.debug(s"Successfully flipped compression table versions {}=>{}, for outgoing to [{}]", state.version, activate.version, remoteAddress) - else - flipTable(activate) // retry - else if (state.version == activate.version) - log.debug("Received duplicate compression table (version: {})! Ignoring it.", state.version) - else - log.error("Received unexpected compression table with version nr [{}]! " + - "Current version number is [{}].", activate.version, state.version) - - } - - // TODO this is crazy hot-path; optimised FastUtil-like Object->int hash map would perform better here (and avoid Integer) allocs - final def compress(value: T): Int = - get().table.getOrDefault(value, NotCompressedId) - - private final def prepareState(activate: CompressionTable[T]): OutboundCompressionState[T] = { - val size = activate.map.size - // load factor is `1` since we will never grow this table beyond the initial size, - // this way we can avoid any rehashing from happening. - val m = new ju.HashMap[T, Integer](size, 1.0f) // TODO could be replaced with primitive `int` specialized version - activate.map.foreach { - case (key, value) ⇒ m.put(key, value) // TODO boxing :< - } - OutboundCompressionState(activate.version, m) - } - - def toDebugString: String = { - s"""${Logging.simpleName(getClass)}( - | version: ${get.version} to [$remoteAddress] - | ${get.table} - |)""".stripMargin - } - - override def toString = { - val s = get - s"""${Logging.simpleName(getClass)}(to: $remoteAddress, version: ${s.version}, compressedEntries: ${s.table.size})""" - } - -} - -/** INTERNAL API */ -private[remote] object OutboundCompression { - // format: OFF - final val DeadLettersId = 0 - final val NotCompressedId = -1 - - // format: ON - - /** INTERNAL API */ - private[remote] final case class OutboundCompressionState[T](version: Int, table: ju.Map[T, Integer]) - private[remote] object OutboundCompressionState { - def initial[T] = OutboundCompressionState[T](-1, ju.Collections.emptyMap()) - } - -} diff --git a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala index d9a3040a28..644b321491 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala @@ -7,14 +7,14 @@ package akka.remote.artery import java.nio.{ ByteBuffer, ByteOrder } import akka.actor._ -import akka.remote.artery.compress.{ CompressionTable, CompressionTestUtils, InboundCompressions, OutboundCompressions } +import akka.remote.artery.compress.{ CompressionTable, CompressionTestUtils, InboundCompressions } import akka.testkit.AkkaSpec import akka.util.{ ByteString, OptionVal } class EnvelopeBufferSpec extends AkkaSpec { import CompressionTestUtils._ - object TestCompressor extends InboundCompressions with OutboundCompressions { + object TestCompressor extends InboundCompressions { val refToIdx: Map[ActorRef, Int] = Map( minimalRef("compressable0") → 0, minimalRef("compressable1") → 1, @@ -31,24 +31,24 @@ class EnvelopeBufferSpec extends AkkaSpec { "manifest1" → 1) val idxToManifest = manifestToIdx.map(_.swap) - override def applyActorRefCompressionTable(table: CompressionTable[ActorRef]): Unit = ??? // dynamic allocating not needed in these tests - override def actorRefCompressionTableVersion: Int = 0 - override def compressActorRef(ref: ActorRef): Int = refToIdx.getOrElse(ref, -1) - override def hitActorRef(originUid: Long, tableVersion: Int, remote: Address, ref: ActorRef, n: Int): Unit = () + val outboundActorRefTable: CompressionTable[ActorRef] = + CompressionTable(version = 0xCAFE, refToIdx) + + val outboundClassManifestTable: CompressionTable[String] = + CompressionTable(version = 0xBABE, manifestToIdx) + + override def hitActorRef(originUid: Long, remote: Address, ref: ActorRef, n: Int): Unit = () override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = OptionVal(idxToRef(idx)) override def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = () - override def applyClassManifestCompressionTable(table: CompressionTable[String]): Unit = ??? // dynamic allocating not needed in these tests - override def classManifestCompressionTableVersion: Int = 0 - override def compressClassManifest(manifest: String): Int = manifestToIdx.getOrElse(manifest, -1) - override def hitClassManifest(originUid: Long, tableVersion: Int, remote: Address, manifest: String, n: Int): Unit = () + override def hitClassManifest(originUid: Long, remote: Address, manifest: String, n: Int): Unit = () override def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] = OptionVal(idxToManifest(idx)) override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = () } "EnvelopeBuffer" must { - val headerIn = HeaderBuilder.bothWays(TestCompressor, TestCompressor) - val headerOut = HeaderBuilder.bothWays(TestCompressor, TestCompressor) + val headerIn = HeaderBuilder.bothWays(TestCompressor, TestCompressor.outboundActorRefTable, TestCompressor.outboundClassManifestTable) + val headerOut = HeaderBuilder.bothWays(TestCompressor, TestCompressor.outboundActorRefTable, TestCompressor.outboundClassManifestTable) val byteBuffer = ByteBuffer.allocate(1024).order(ByteOrder.LITTLE_ENDIAN) val envelope = new EnvelopeBuffer(byteBuffer) @@ -59,8 +59,6 @@ class EnvelopeBufferSpec extends AkkaSpec { headerIn setVersion 1 headerIn setUid 42 headerIn setSerializer 4 - headerIn setActorRefCompressionTableVersion 0xCAFE - headerIn setClassManifestCompressionTableVersion 0xBABE headerIn setRecipientActorRef minimalRef("compressable1") headerIn setSenderActorRef minimalRef("compressable0") @@ -74,8 +72,8 @@ class EnvelopeBufferSpec extends AkkaSpec { headerOut.version should ===(1) headerOut.uid should ===(42) - headerOut.actorRefCompressionTableVersion should ===(0xCAFE) - headerOut.classManifestCompressionTableVersion should ===(0xBABE) + headerOut.inboundActorRefCompressionTableVersion should ===(0xCAFE) + headerOut.inboundClassManifestCompressionTableVersion should ===(0xBABE) headerOut.serializer should ===(4) headerOut.senderActorRef(originUid).get.path.toSerializationFormat should ===("akka://EnvelopeBufferSpec/compressable0") headerOut.senderActorRefPath should ===(OptionVal.None) diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala index 41c12280f8..04233b14ef 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -18,6 +18,7 @@ import akka.remote.artery.InboundControlJunction.ControlMessageObserver import akka.remote.artery.InboundControlJunction.ControlMessageSubject import akka.util.OptionVal import akka.actor.InternalActorRef +import akka.dispatch.ExecutionContexts private[akka] class TestInboundContext( override val localAddress: UniqueAddress, @@ -47,10 +48,13 @@ private[akka] class TestInboundContext( override def association(uid: Long): OptionVal[OutboundContext] = OptionVal(associationsByUid.get(uid)) - override def completeHandshake(peer: UniqueAddress): Unit = { + override def completeHandshake(peer: UniqueAddress): Future[Done] = { val a = association(peer.address).asInstanceOf[TestOutboundContext] - a.completeHandshake(peer) - associationsByUid.put(peer.uid, a) + val done = a.completeHandshake(peer) + done.foreach { _ ⇒ + associationsByUid.put(peer.uid, a) + }(ExecutionContexts.sameThreadExecutionContext) + done } protected def createAssociation(remoteAddress: Address): TestOutboundContext = @@ -70,13 +74,14 @@ private[akka] class TestOutboundContext( _associationState } - def completeHandshake(peer: UniqueAddress): Unit = synchronized { + 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), NoOutboundCompressions) + _associationState = _associationState.newIncarnation(Promise.successful(peer)) } + Future.successful(Done) } override def quarantine(reason: String): Unit = synchronized { 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 index 723a4c9a13..ac04610e9b 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/OutboundCompressionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/OutboundCompressionSpec.scala @@ -10,40 +10,23 @@ import akka.testkit.AkkaSpec class OutboundCompressionSpec extends AkkaSpec { import CompressionTestUtils._ - val remoteAddress = Address("artery", "example", "localhost", 0) - - "OutboundCompression" must { - "not compress not-known values" in { - val table = new OutboundActorRefCompression(system, remoteAddress) - table.compress(minimalRef("banana")) should ===(-1) - } - } - - "OutboundActorRefCompression" must { + "Outbound ActorRef compression" must { val alice = minimalRef("alice") val bob = minimalRef("bob") - "always compress /deadLetters" in { - val table = new OutboundActorRefCompression(system, remoteAddress) - table.compress(system.deadLetters) should ===(0) - } - "not compress unknown actor ref" in { - val table = new OutboundActorRefCompression(system, remoteAddress) + val table = CompressionTable.empty[ActorRef] table.compress(alice) should ===(-1) // not compressed } "compress previously registered actor ref" in { - val compression = new OutboundActorRefCompression(system, remoteAddress) val table = CompressionTable(1, Map(system.deadLetters → 0, alice → 1)) - compression.flipTable(table) - compression.compress(alice) should ===(1) // compressed - compression.compress(bob) should ===(-1) // not compressed + table.compress(alice) should ===(1) // compressed + table.compress(bob) should ===(-1) // not compressed val table2 = table.copy(2, map = table.map.updated(bob, 2)) - compression.flipTable(table2) - compression.compress(alice) should ===(1) // compressed - compression.compress(bob) should ===(2) // compressed + table2.compress(alice) should ===(1) // compressed + table2.compress(bob) should ===(2) // compressed } } From ed05a77f9cc336bd48045c399bfb31148523d11e Mon Sep 17 00:00:00 2001 From: Konrad Malawski Date: Wed, 31 Aug 2016 10:09:07 +0100 Subject: [PATCH 097/186] =pro update aeron to 1.0.1 Changes are https://github.com/real-logic/Aeron/releases/tag/1.0.1 : > * Adjust incremental cleaning to avoid overwrite of cleaned log in loss scenarios to address Issue #271. > * Correctly initialise cleaning position for late joining a stream. Issue #268. > * Make clientLivenessTimeout configurable from Context. > * Shadow samples into aeron-all JAR. > * Support creating Agent threads via ThreadFactory to enable pinning. > * Update to Agrona 0.5.4. > * Update to ByteBuddy 1.4.20. --- project/Dependencies.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 41eea28d4f..c5935a8b2f 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -76,8 +76,8 @@ 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" // ApacheV2 - val aeronClient = "io.aeron" % "aeron-client" % "1.0" // ApacheV2 + 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" From 292face28aeefa8d3e4ad293d0478f3b754af689 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Martynas=20Mickevi=C4=8Dius?= Date: Thu, 1 Sep 2016 09:07:39 +0300 Subject: [PATCH 098/186] #20587 Clean artery configuration (#21279) * Move artery settings from remoting settings to dedicated class. * #20587 Move hardcoded settings to configuration file. * Copy reused settings from remote to the artery --- .../akka/remote/testconductor/Conductor.scala | 4 +- .../remote/artery/MaxThroughputSpec.scala | 7 +- akka-remote/src/main/resources/reference.conf | 141 ++++++++++++------ .../akka/remote/RemoteActorRefProvider.scala | 2 +- .../scala/akka/remote/RemoteSettings.scala | 21 +-- .../akka/remote/artery/ArterySettings.scala | 103 +++++++++++++ .../akka/remote/artery/ArteryTransport.scala | 97 +++++------- .../akka/remote/artery/Association.scala | 14 +- .../artery/compress/CompressionSettings.scala | 39 ----- .../artery/compress/InboundCompressions.scala | 21 ++- 10 files changed, 263 insertions(+), 186 deletions(-) create mode 100644 akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala delete mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/CompressionSettings.scala 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 cb1114fb12..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 @@ -123,8 +123,8 @@ trait Conductor { this: TestConductorExt ⇒ throttle(node, target, direction, 0f) private def requireTestConductorTranport(): Unit = { - if (transport.provider.remoteSettings.EnableArtery) { - if (!transport.provider.remoteSettings.TestMode) + 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 { 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 index 8dad9e0c6e..335d3bedbe 100644 --- 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 @@ -9,7 +9,6 @@ import java.util.concurrent.TimeUnit.NANOSECONDS import scala.concurrent.duration._ import akka.actor._ import akka.remote.RemoteActorRefProvider -import akka.remote.artery.compress.CompressionSettings import akka.remote.testconductor.RoleName import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeSpec @@ -117,7 +116,7 @@ object MaxThroughputSpec extends MultiNodeConfig { val compressionEnabled = RARP(context.system).provider.transport.isInstanceOf[ArteryTransport] && - RARP(context.system).provider.remoteSettings.ArteryCompressionSettings.enabled + RARP(context.system).provider.remoteSettings.Artery.Enabled def receive = { case Run ⇒ @@ -174,7 +173,7 @@ object MaxThroughputSpec extends MultiNodeConfig { 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 (CompressionSettings(context.system).enabled) ",compression" else "") + "), " + + (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, " + @@ -217,7 +216,7 @@ object MaxThroughputSpec extends MultiNodeConfig { payloadSize: Int, senderReceiverPairs: Int) { // data based on measurement - def totalSize(system: ActorSystem) = payloadSize + (if (CompressionSettings(system).enabled) 38 else 110) + 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 { diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index d40cc4047f..bf0fbd141c 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -32,19 +32,19 @@ akka { # "scala.Some" = akka-misc # "scala.None$" = akka-misc "akka.remote.DaemonMsgCreate" = daemon-create - + # 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 - + } serialization-identifiers { @@ -87,12 +87,12 @@ akka { artery { enabled = off port = 20200 - + # The hostname or ip clients should connect to. - # InetAddress.getLocalHost.getHostAddress is used if empty or + # InetAddress.getLocalHost.getHostAddress is used if empty or # "" is specified. - # InetAddress.getLocalHost.getHostName is used if - # "" is specified. + # InetAddress.getLocalHost.getHostName is used if + # "" is specified. hostname = "" # Actor paths to use the large message stream for when a message @@ -108,36 +108,99 @@ akka { # stream, to pass such messages through the large message stream the selections # but must be resolved to ActorRefs first. large-message-destinations = [] - + + # Sets the log granularity level at which Akka logs artery events. This setting + # can take the values OFF, ERROR, WARNING, INFO or DEBUG. Please note that the effective + # logging level is still determined by the global logging level of the actor system: + # for example debug level artery events will be only logged if the system + # is running with debug level logging. + # Failures to deserialize received messages also fall under this flag. + log-lifecycle-events = DEBUG + + # If set to a nonempty string artery will use the given dispatcher for + # its internal actors otherwise the default dispatcher is used. + use-dispatcher = "akka.remote.default-remote-dispatcher" + advanced { # 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} materializer { dispatcher = "akka.remote.default-remote-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 + # 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. + # backoff backpressure. # Level 1 strongly prefer low CPU consumption over low latency. - # Level 10 strongly prefer low latency over low CPU consumption. + # Level 10 strongly prefer low latency over low CPU consumption. idle-cpu-level = 5 - + + # 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 tainted. + system-message-buffer-size = 20000 + + # unacknowledged system messages are re-delivered with this interval + system-message-resend-interval = 1 second + + # 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-send-after = 60 seconds + + # 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 + + # Timeout after which the inbound stream is going to be restarted. + inbound-restart-timeout = 5 seconds + + # Max number of restarts for the inbound stream. + inbound-max-restarts = 5 + + # Timeout after which outbout stream is going to be restarted for every association. + outbound-restart-timeout = 5 seconds + + # Max number of restars of the outbound stream for every association. + outbound-max-restarts = 5 + + # 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. + image-liveness-timeout = 20 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 @@ -145,43 +208,34 @@ akka { # compression of common strings in remoting messages, like actor destinations, serializers etc compression { - # possibility to disable compression by setting this to off - enabled = on - - # unlocks additional very verbose debug logging of compression events (on DEBUG log level) - debug = off - + actor-refs { - enabled = off # TODO possibly remove on/off option once we have battle proven it? - # Max number of compressed actor-refs - # Note that compression tables are "rolling" (i.e. a new table replaces the old + # 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 + advertisement-interval = 1 minute # TODO find good number as default, for benchmarks trigger immediately } manifests { - enabled = off # TODO possibly remove on/off option once we have battle proven it? - # Max number of compressed manifests - # Note that compression tables are "rolling" (i.e. a new table replaces the old + # 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 + advertisement-interval = 1 minute # TODO find good number as default, for benchmarks trigger immediately } } } - + } ### General settings @@ -211,7 +265,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. @@ -230,11 +284,11 @@ akka { # 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 @@ -272,7 +326,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 @@ -281,7 +335,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. @@ -409,7 +463,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) @@ -649,7 +703,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 @@ -679,7 +733,7 @@ akka { } throughput = 10 } - + backoff-remote-dispatcher { type = Dispatcher executor = "fork-join-executor" @@ -689,8 +743,5 @@ akka { parallelism-max = 2 } } - - } - } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 607ef745ba..8f1735cdda 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -187,7 +187,7 @@ private[akka] class RemoteActorRefProvider( d }, serialization = SerializationExtension(system), - transport = if (remoteSettings.EnableArtery) new ArteryTransport(system, this) else new Remoting(system, this)) + transport = if (remoteSettings.Artery.Enabled) new ArteryTransport(system, this) else new Remoting(system, this)) _internals = internals remotingTerminator ! internals diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index a4b11f41bb..1ffd06ecca 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -3,7 +3,6 @@ */ package akka.remote -import akka.remote.artery.compress.CompressionSettings import com.typesafe.config.Config import scala.concurrent.duration._ import akka.util.Timeout @@ -14,29 +13,13 @@ import akka.actor.Props import akka.event.Logging import akka.event.Logging.LogLevel import akka.ConfigurationException -import java.net.InetAddress +import akka.remote.artery.ArterySettings final class RemoteSettings(val config: Config) { import config._ import scala.collection.JavaConverters._ - val EnableArtery: Boolean = getBoolean("akka.remote.artery.enabled") - val ArteryPort: Int = getInt("akka.remote.artery.port") - val ArteryHostname: String = getString("akka.remote.artery.hostname") match { - case "" | "" ⇒ InetAddress.getLocalHost.getHostAddress - case "" ⇒ InetAddress.getLocalHost.getHostName - case other ⇒ other - } - val EmbeddedMediaDriver = getBoolean("akka.remote.artery.advanced.embedded-media-driver") - val AeronDirectoryName = getString("akka.remote.artery.advanced.aeron-dir") requiring (dir ⇒ - EmbeddedMediaDriver || dir.nonEmpty, "aeron-dir must be defined when using external media driver") - val TestMode: Boolean = getBoolean("akka.remote.artery.advanced.test-mode") - val IdleCpuLevel: Int = getInt("akka.remote.artery.advanced.idle-cpu-level").requiring(level ⇒ - 1 <= level && level <= 10, "idle-cpu-level must be between 1 and 10") - - val FlightRecorderEnabled: Boolean = getBoolean("akka.remote.artery.advanced.flight-recorder.enabled") - - val ArteryCompressionSettings = CompressionSettings(getConfig("akka.remote.artery.advanced.compression")) + val Artery = ArterySettings(getConfig("akka.remote.artery")) val LogReceive: Boolean = getBoolean("akka.remote.log-received-messages") 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..c066790e79 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala @@ -0,0 +1,103 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +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.util.concurrent.TimeUnit + +/** INTERNAL API */ +private[akka] final class ArterySettings private (config: Config) { + import config._ + import ArterySettings._ + + val Enabled: Boolean = getBoolean("enabled") + val Port: Int = getInt("port") + val Hostname: String = getString("hostname") match { + case "" | "" ⇒ InetAddress.getLocalHost.getHostAddress + case "" ⇒ InetAddress.getLocalHost.getHostName + 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 LifecycleEventsLogLevel: LogLevel = Logging.levelFor(toRootLowerCase(getString("log-lifecycle-events"))) match { + case Some(level) ⇒ level + case None ⇒ throw new ConfigurationException("Logging level must be one of (off, debug, info, warning, error)") + } + val Dispatcher = getString("use-dispatcher") + + object Advanced { + val config = getConfig("advanced") + import config._ + + val TestMode: Boolean = getBoolean("test-mode") + val MaterializerSettings = ActorMaterializerSettings(config.getConfig("materializer")) + 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 SysMsgBufferSize: Int = getInt("system-message-buffer-size").requiring( + _ > 0, "system-message-buffer-size must be more than zero") + val SystemMessageResendInterval = config.getMillisDuration("system-message-resend-interval").requiring(interval ⇒ + interval > 0.seconds, "system-message-resend-interval must be more than zero") + val HandshakeRetryInterval = config.getMillisDuration("handshake-retry-interval").requiring(interval ⇒ + interval > 0.seconds, "handshake-retry-interval must be more than zero") + val InjectHandshakeInterval = config.getMillisDuration("inject-handshake-interval").requiring(interval ⇒ + interval > 0.seconds, "inject-handshake-interval must be more than zero") + val GiveUpSendAfter = config.getMillisDuration("give-up-send-after") + val ShutdownFlushTimeout = config.getMillisDuration("shutdown-flush-timeout") + val InboundRestartTimeout = config.getMillisDuration("inbound-restart-timeout") + val InboundMaxRestarts = getInt("inbound-max-restarts") + val OutboundRestartTimeout = config.getMillisDuration("outbound-restart-timeout") + val OutboundMaxRestarts = getInt("outbound-max-restarts") + val ClientLivenessTimeout = config.getMillisDuration("client-liveness-timeout") + val ImageLivenessTimeoutNs = config.getMillisDuration("image-liveness-timeout") + val DriverTimeout = config.getMillisDuration("driver-timeout") + val FlightRecorderEnabled: Boolean = getBoolean("flight-recorder.enabled") + val Compression = new Compression(getConfig("compression")) + } +} + +/** INTERNAL API */ +private[akka] object ArterySettings { + def apply(config: Config) = new ArterySettings(config) + + /** INTERNAL API */ + private[akka] final class Compression private[ArterySettings] (config: Config) { + import config._ + + // Compile time constants + final val Enabled = true + final val Debug = false // unlocks additional very verbose debug logging of compression events (on DEBUG log level) + + 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") + } + } +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 3c4c10d5ec..a07a1e12b6 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -33,7 +33,6 @@ import akka.remote.AddressUidExtension import akka.remote.EventPublisher import akka.remote.RemoteActorRef import akka.remote.RemoteActorRefProvider -import akka.remote.RemoteSettings import akka.remote.RemoteTransport import akka.remote.RemotingLifecycleEvent import akka.remote.ThisActorSystemQuarantinedEvent @@ -125,9 +124,9 @@ private[akka] object AssociationState { * INTERNAL API */ private[akka] final class AssociationState( - val incarnation: Int, + val incarnation: Int, val uniqueRemoteAddressPromise: Promise[UniqueAddress], - val quarantined: ImmutableLongMap[AssociationState.QuarantinedTimestamp]) { + val quarantined: ImmutableLongMap[AssociationState.QuarantinedTimestamp]) { import AssociationState.QuarantinedTimestamp @@ -225,7 +224,7 @@ private[akka] trait OutboundContext { */ private[remote] object FlushOnShutdown { def props(done: Promise[Done], timeout: FiniteDuration, - inboundContext: InboundContext, associations: Set[Association]): Props = { + inboundContext: InboundContext, associations: Set[Association]): Props = { require(associations.nonEmpty) Props(new FlushOnShutdown(done, timeout, inboundContext, associations)) } @@ -237,7 +236,7 @@ private[remote] object FlushOnShutdown { * INTERNAL API */ private[remote] class FlushOnShutdown(done: Promise[Done], timeout: FiniteDuration, - inboundContext: InboundContext, associations: Set[Association]) extends Actor { + inboundContext: InboundContext, associations: Set[Association]) extends Actor { var remaining = associations.flatMap(_.associationState.uniqueRemoteAddressValue) @@ -288,7 +287,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R override def addresses: Set[Address] = _addresses override def localAddressForRemote(remote: Address): Address = defaultAddress override val log: LoggingAdapter = Logging(system, getClass.getName) - val eventPublisher = new EventPublisher(system, log, remoteSettings.RemoteLifecycleEventsLogLevel) + val eventPublisher = new EventPublisher(system, log, settings.LifecycleEventsLogLevel) private val codec: AkkaPduCodec = AkkaPduProtobufCodec private val killSwitch: SharedKillSwitch = KillSwitches.shared("transportKillSwitch") @@ -296,27 +295,16 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val testStages: CopyOnWriteArrayList[TestManagementApi] = new CopyOnWriteArrayList - // FIXME config - private val systemMessageResendInterval: FiniteDuration = 1.second - private val handshakeRetryInterval: FiniteDuration = 1.second private val handshakeTimeout: FiniteDuration = system.settings.config.getMillisDuration("akka.remote.handshake-timeout").requiring( _ > Duration.Zero, "handshake-timeout must be > 0") - private val injectHandshakeInterval: FiniteDuration = 1.second - private val giveUpSendAfter: FiniteDuration = 60.seconds - private val shutdownFlushTimeout = 1.second - private val remoteDispatcher = system.dispatchers.lookup(remoteSettings.Dispatcher) + private val remoteDispatcher = system.dispatchers.lookup(settings.Dispatcher) - private val largeMessageDestinations = - system.settings.config.getStringList("akka.remote.artery.large-message-destinations").asScala.foldLeft(WildcardIndex[NotUsed]()) { (tree, entry) ⇒ - val segments = entry.split('/').tail - tree.insert(segments, NotUsed) - } // TODO use WildcardIndex.isEmpty when merged from master val largeMessageChannelEnabled = - !largeMessageDestinations.wildcardTree.isEmpty || !largeMessageDestinations.doubleWildcardTree.isEmpty + !settings.LargeMessageDestinations.wildcardTree.isEmpty || !settings.LargeMessageDestinations.doubleWildcardTree.isEmpty private val priorityMessageDestinations = WildcardIndex[NotUsed]() @@ -334,11 +322,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val ordinaryStreamId = 3 private val largeStreamId = 4 - private val taskRunner = new TaskRunner(system, remoteSettings.IdleCpuLevel) + private val taskRunner = new TaskRunner(system, settings.Advanced.IdleCpuLevel) - private val restartTimeout: FiniteDuration = 5.seconds // FIXME config - private val maxRestarts = 5 // FIXME config - private val restartCounter = new RestartCounter(maxRestarts, restartTimeout) + private val restartCounter = new RestartCounter(settings.Advanced.InboundMaxRestarts, settings.Advanced.InboundRestartTimeout) private val envelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) private val largeEnvelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumLargeFrameSize, ArteryTransport.MaximumPooledBuffers) @@ -373,11 +359,11 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R materializer, remoteAddress, controlSubject, - largeMessageDestinations, + settings.LargeMessageDestinations, priorityMessageDestinations, outboundEnvelopePool)) - def remoteSettings: RemoteSettings = provider.remoteSettings + def settings = provider.remoteSettings.Artery override def start(): Unit = { startMediaDriver() @@ -389,22 +375,20 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R topLevelFREvents.loFreq(Transport_TaskRunnerStarted, NoMetaData) val port = - if (remoteSettings.ArteryPort == 0) ArteryTransport.autoSelectPort(remoteSettings.ArteryHostname) - else remoteSettings.ArteryPort + if (settings.Port == 0) ArteryTransport.autoSelectPort(settings.Hostname) + else settings.Port // TODO: Configure materializer properly // TODO: Have a supervisor actor _localAddress = UniqueAddress( - Address(ArteryTransport.ProtocolName, system.name, remoteSettings.ArteryHostname, port), + Address(ArteryTransport.ProtocolName, system.name, settings.Hostname, port), AddressUidExtension(system).longAddressUid) _addresses = Set(_localAddress.address) // 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")) - val materializerSettings = ActorMaterializerSettings( - remoteSettings.config.getConfig("akka.remote.artery.advanced.materializer")) - materializer = ActorMaterializer.systemMaterializer(materializerSettings, "remote", system) + materializer = ActorMaterializer.systemMaterializer(settings.Advanced.MaterializerSettings, "remote", system) messageDispatcher = new MessageDispatcher(system, provider) topLevelFREvents.loFreq(Transport_MaterializerStarted, NoMetaData) @@ -420,16 +404,15 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def startMediaDriver(): Unit = { - if (remoteSettings.EmbeddedMediaDriver) { + if (settings.Advanced.EmbeddedMediaDriver) { val driverContext = new MediaDriver.Context - if (remoteSettings.AeronDirectoryName.nonEmpty) - driverContext.aeronDirectoryName(remoteSettings.AeronDirectoryName) - // FIXME settings from config - driverContext.clientLivenessTimeoutNs(SECONDS.toNanos(20)) - driverContext.imageLivenessTimeoutNs(SECONDS.toNanos(20)) - driverContext.driverTimeoutMs(SECONDS.toNanos(20)) + if (settings.Advanced.AeronDirectoryName.nonEmpty) + driverContext.aeronDirectoryName(settings.Advanced.AeronDirectoryName) + driverContext.clientLivenessTimeoutNs(settings.Advanced.ClientLivenessTimeout.toNanos) + driverContext.imageLivenessTimeoutNs(settings.Advanced.ImageLivenessTimeoutNs.toNanos) + driverContext.driverTimeoutMs(settings.Advanced.DriverTimeout.toMillis) - val idleCpuLevel = remoteSettings.IdleCpuLevel + val idleCpuLevel = settings.Advanced.IdleCpuLevel if (idleCpuLevel == 10) { driverContext .threadingMode(ThreadingMode.DEDICATED) @@ -461,7 +444,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private def aeronDir: String = mediaDriver match { case Some(driver) ⇒ driver.aeronDirectoryName - case None ⇒ remoteSettings.AeronDirectoryName + case None ⇒ settings.Advanced.AeronDirectoryName } private def stopMediaDriver(): Unit = { @@ -469,8 +452,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // this is only for embedded media driver driver.close() try { - // FIXME it should also be configurable to not delete dir - IoUtil.delete(new File(driver.aeronDirectoryName), false) + if (settings.Advanced.DeleteAeronDirectory) { + IoUtil.delete(new File(driver.aeronDirectoryName), false) + } } catch { case NonFatal(e) ⇒ log.warning( @@ -542,7 +526,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private def runInboundControlStream(compression: InboundCompressions): Unit = { val (ctrl, completed) = - if (remoteSettings.TestMode) { + if (settings.Advanced.TestMode) { val (mgmt, (ctrl, completed)) = aeronSource(controlStreamId, envelopePool) .via(inboundFlow(compression)) @@ -617,7 +601,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private def runInboundOrdinaryMessagesStream(compression: InboundCompressions): Unit = { val completed = - if (remoteSettings.TestMode) { + if (settings.Advanced.TestMode) { val (mgmt, c) = aeronSource(ordinaryStreamId, envelopePool) .via(inboundFlow(compression)) .viaMat(inboundTestFlow)(Keep.right) @@ -639,7 +623,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R val disableCompression = NoInboundCompressions // no compression on large message stream for now val completed = - if (remoteSettings.TestMode) { + if (settings.Advanced.TestMode) { val (mgmt, c) = aeronSource(largeStreamId, largeEnvelopePool) .via(inboundLargeFlow(disableCompression)) .viaMat(inboundTestFlow)(Keep.right) @@ -668,7 +652,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R restart() } else { log.error(cause, "{} failed and restarted {} times within {} seconds. Terminating system. {}", - streamName, maxRestarts, restartTimeout.toSeconds, cause.getMessage) + streamName, settings.Advanced.InboundMaxRestarts, settings.Advanced.InboundRestartTimeout.toSeconds, cause.getMessage) system.terminate() } } @@ -681,8 +665,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R if (allAssociations.isEmpty) Future.successful(Done) else { val flushingPromise = Promise[Done]() - system.systemActorOf(FlushOnShutdown.props(flushingPromise, shutdownFlushTimeout, - this, allAssociations).withDispatcher(remoteSettings.Dispatcher), "remoteFlushOnShutdown") + system.systemActorOf(FlushOnShutdown.props(flushingPromise, settings.Advanced.ShutdownFlushTimeout, + this, allAssociations).withDispatcher(settings.Dispatcher), "remoteFlushOnShutdown") flushingPromise.future } implicit val ec = remoteDispatcher @@ -774,14 +758,14 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .mapMaterializedValue { case (_, d) ⇒ d } private def createOutboundSink(streamId: Int, outboundContext: OutboundContext, - bufferPool: EnvelopeBufferPool): Sink[OutboundEnvelope, (ChangeOutboundCompression, Future[Done])] = { + bufferPool: EnvelopeBufferPool): Sink[OutboundEnvelope, (ChangeOutboundCompression, Future[Done])] = { Flow.fromGraph(killSwitch.flow[OutboundEnvelope]) .via(new OutboundHandshake(system, outboundContext, outboundEnvelopePool, handshakeTimeout, - handshakeRetryInterval, injectHandshakeInterval)) + settings.Advanced.HandshakeRetryInterval, settings.Advanced.InjectHandshakeInterval)) .viaMat(createEncoder(bufferPool))(Keep.right) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), streamId, aeron, taskRunner, - envelopePool, giveUpSendAfter, createFlightRecorderEventSink()))(Keep.both) + envelopePool, settings.Advanced.GiveUpSendAfter, createFlightRecorderEventSink()))(Keep.both) } /** @@ -791,9 +775,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R def outboundControlPart1(outboundContext: OutboundContext): Flow[OutboundEnvelope, OutboundEnvelope, SharedKillSwitch] = { Flow.fromGraph(killSwitch.flow[OutboundEnvelope]) .via(new OutboundHandshake(system, outboundContext, outboundEnvelopePool, handshakeTimeout, - handshakeRetryInterval, injectHandshakeInterval)) - .via(new SystemMessageDelivery(outboundContext, system.deadLetters, systemMessageResendInterval, - remoteSettings.SysMsgBufferSize)) + settings.Advanced.HandshakeRetryInterval, settings.Advanced.InjectHandshakeInterval)) + .via(new SystemMessageDelivery(outboundContext, system.deadLetters, settings.Advanced.SystemMessageResendInterval, + settings.Advanced.SysMsgBufferSize)) // FIXME we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages } @@ -807,7 +791,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def createInboundCompressions(inboundContext: InboundContext): InboundCompressions = - if (remoteSettings.ArteryCompressionSettings.enabled) new InboundCompressionsImpl(system, inboundContext) + if (settings.Advanced.Compression.Enabled) new InboundCompressionsImpl(system, inboundContext, settings.Advanced.Compression) else NoInboundCompressions def createEncoder(pool: EnvelopeBufferPool): Flow[OutboundEnvelope, EnvelopeBuffer, ChangeOutboundCompression] = @@ -864,7 +848,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def initializeFlightRecorder(): Option[(FileChannel, File, FlightRecorder)] = { - if (remoteSettings.FlightRecorderEnabled) { + if (settings.Advanced.FlightRecorderEnabled) { // TODO: Figure out where to put it, currently using temporary files val afrFile = File.createTempFile("artery", ".afr") afrFile.deleteOnExit() @@ -920,4 +904,3 @@ private[remote] object ArteryTransport { } } - diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 82dc5bf01f..55db97720e 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -70,15 +70,13 @@ private[remote] class Association( import Association._ private val log = Logging(transport.system, getClass.getName) - private val controlQueueSize = transport.remoteSettings.SysMsgBufferSize + private val controlQueueSize = transport.settings.Advanced.SysMsgBufferSize // FIXME config queue size, and it should perhaps also be possible to use some kind of LinkedQueue // such as agrona.ManyToOneConcurrentLinkedQueue or AbstractNodeQueue for less memory consumption private val queueSize = 3072 private val largeQueueSize = 256 - private val restartTimeout: FiniteDuration = 5.seconds // FIXME config - private val maxRestarts = 5 // FIXME config - private val restartCounter = new RestartCounter(maxRestarts, restartTimeout) + private val restartCounter = new RestartCounter(transport.settings.Advanced.OutboundMaxRestarts, transport.settings.Advanced.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 @@ -339,7 +337,7 @@ private[remote] class Association( controlQueue = wrapper // use new underlying queue immediately for restarts val (queueValue, (control, completed)) = - if (transport.remoteSettings.TestMode) { + if (transport.settings.Advanced.TestMode) { val ((queueValue, mgmt), (control, completed)) = Source.fromGraph(new SendQueue[OutboundEnvelope]) .via(transport.outboundControlPart1(this)) @@ -382,7 +380,7 @@ private[remote] class Association( queue = wrapper // use new underlying queue immediately for restarts val (queueValue, (changeCompression, completed)) = - if (transport.remoteSettings.TestMode) { + if (transport.settings.Advanced.TestMode) { val ((queueValue, mgmt), completed) = Source.fromGraph(new SendQueue[OutboundEnvelope]) .viaMat(transport.outboundTestFlow(this))(Keep.both) .toMat(transport.outbound(this))(Keep.both) @@ -408,7 +406,7 @@ private[remote] class Association( largeQueue = wrapper // use new underlying queue immediately for restarts val (queueValue, completed) = - if (transport.remoteSettings.TestMode) { + if (transport.settings.Advanced.TestMode) { val ((queueValue, mgmt), completed) = Source.fromGraph(new SendQueue[OutboundEnvelope]) .viaMat(transport.outboundTestFlow(this))(Keep.both) .toMat(transport.outboundLarge(this))(Keep.both) @@ -442,7 +440,7 @@ private[remote] class Association( restart(cause) } else { log.error(cause, s"{} to {} failed and restarted {} times within {} seconds. Terminating system. ${cause.getMessage}", - streamName, remoteAddress, maxRestarts, restartTimeout.toSeconds) + streamName, remoteAddress, transport.settings.Advanced.OutboundMaxRestarts, transport.settings.Advanced.OutboundRestartTimeout.toSeconds) transport.system.terminate() } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionSettings.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionSettings.scala deleted file mode 100644 index 82a9c7e752..0000000000 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionSettings.scala +++ /dev/null @@ -1,39 +0,0 @@ -/** - * Copyright (C) 2016 Lightbend Inc. - */ -package akka.remote.artery.compress - -import java.util.concurrent.TimeUnit - -import akka.actor.ActorSystem -import com.typesafe.config.Config - -import scala.concurrent.duration._ - -/** INTERNAL API */ -private[akka] class CompressionSettings(_config: Config) { - val enabled = _config.getBoolean("enabled") - - val debug = _config.getBoolean("debug") - - object actorRefs { - private val c = _config.getConfig("actor-refs") - - val advertisementInterval = c.getDuration("advertisement-interval", TimeUnit.MILLISECONDS).millis - val max = c.getInt("max") - } - object manifests { - private val c = _config.getConfig("manifests") - - val advertisementInterval = c.getDuration("advertisement-interval", TimeUnit.MILLISECONDS).millis - val max = c.getInt("max") - } -} - -/** INTERNAL API */ -private[akka] object CompressionSettings { // TODO make it an extension - def apply(config: Config): CompressionSettings = new CompressionSettings(config) - def apply(system: ActorSystem): CompressionSettings = - new CompressionSettings( - system.settings.config.getConfig("akka.remote.artery.advanced.compression")) -} 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 index 5be9354a32..7b09b10264 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala @@ -11,7 +11,7 @@ import scala.concurrent.duration.{ Duration, FiniteDuration } import akka.actor.{ ActorRef, ActorSystem, Address } import akka.event.{ Logging, NoLogging } -import akka.remote.artery.{ InboundContext, OutboundContext } +import akka.remote.artery.{ ArterySettings, InboundContext, OutboundContext } import akka.util.{ OptionVal, PrettyDuration } import org.agrona.collections.Long2ObjectHashMap @@ -38,15 +38,14 @@ private[remote] trait InboundCompressions { */ private[remote] final class InboundCompressionsImpl( system: ActorSystem, - inboundContext: InboundContext) extends InboundCompressions { - - private val settings = CompressionSettings(system) + inboundContext: InboundContext, + settings: ArterySettings.Compression) extends InboundCompressions { // FIXME we also must remove the ones that won't be used anymore - when quarantine triggers private[this] val _actorRefsIns = new Long2ObjectHashMap[InboundActorRefCompression]() private val createInboundActorRefsForOrigin = new LongFunction[InboundActorRefCompression] { override def apply(originUid: Long): InboundActorRefCompression = { - val actorRefHitters = new TopHeavyHitters[ActorRef](settings.actorRefs.max) + val actorRefHitters = new TopHeavyHitters[ActorRef](settings.ActorRefs.Max) new InboundActorRefCompression(system, settings, originUid, inboundContext, actorRefHitters) } } @@ -56,7 +55,7 @@ private[remote] final class InboundCompressionsImpl( private[this] val _classManifestsIns = new Long2ObjectHashMap[InboundManifestCompression]() private val createInboundManifestsForOrigin = new LongFunction[InboundManifestCompression] { override def apply(originUid: Long): InboundManifestCompression = { - val manifestHitters = new TopHeavyHitters[String](settings.manifests.max) + val manifestHitters = new TopHeavyHitters[String](settings.Manifests.Max) new InboundManifestCompression(system, settings, originUid, inboundContext, manifestHitters) } } @@ -106,7 +105,7 @@ private[remote] final class InboundCompressionsImpl( */ private[remote] final class InboundActorRefCompression( system: ActorSystem, - settings: CompressionSettings, + settings: ArterySettings.Compression, originUid: Long, inboundContext: InboundContext, heavyHitters: TopHeavyHitters[ActorRef]) extends InboundCompression[ActorRef](system, settings, originUid, inboundContext, heavyHitters) { @@ -123,7 +122,7 @@ private[remote] final class InboundActorRefCompression( else super.decompress(tableVersion, idx) scheduleNextTableAdvertisement() - override protected def tableAdvertisementInterval = settings.actorRefs.advertisementInterval + override protected def tableAdvertisementInterval = settings.ActorRefs.AdvertisementInterval override def advertiseCompressionTable(outboundContext: OutboundContext, table: CompressionTable[ActorRef]): Unit = { log.debug(s"Advertise ActorRef compression [$table], from [${inboundContext.localAddress}] to [${outboundContext.remoteAddress}]") @@ -133,13 +132,13 @@ private[remote] final class InboundActorRefCompression( final class InboundManifestCompression( system: ActorSystem, - settings: CompressionSettings, + settings: ArterySettings.Compression, originUid: Long, inboundContext: InboundContext, heavyHitters: TopHeavyHitters[String]) extends InboundCompression[String](system, settings, originUid, inboundContext, heavyHitters) { scheduleNextTableAdvertisement() - override protected def tableAdvertisementInterval = settings.manifests.advertisementInterval + override protected def tableAdvertisementInterval = settings.Manifests.AdvertisementInterval override lazy val log = NoLogging @@ -183,7 +182,7 @@ private[remote] object InboundCompression { */ private[remote] abstract class InboundCompression[T >: Null]( val system: ActorSystem, - val settings: CompressionSettings, + val settings: ArterySettings.Compression, originUid: Long, inboundContext: InboundContext, val heavyHitters: TopHeavyHitters[T]) { From 888ffa5f0698458b21fa74726a8d0d8dc59d6fed Mon Sep 17 00:00:00 2001 From: Konrad Malawski Date: Fri, 2 Sep 2016 08:52:09 +0100 Subject: [PATCH 099/186] +art enable compression for manifests (#21211) (#21333) --- .../akka/remote/artery/ArterySettings.scala | 6 +- .../akka/remote/artery/ArteryTransport.scala | 3 +- .../scala/akka/remote/artery/BufferPool.scala | 6 +- .../scala/akka/remote/artery/Codecs.scala | 2 +- .../artery/compress/CompressionTable.scala | 35 +++++-- .../artery/compress/DecompressionTable.scala | 4 +- .../artery/compress/InboundCompressions.scala | 47 +++++---- .../ActorRefCompressionIntegrationSpec.scala | 66 +++++++++++++ ...ssManifestCompressionIntegrationSpec.scala | 98 +++++++++++++++++++ .../compress/CompressionIntegrationSpec.scala | 84 +++------------- 10 files changed, 247 insertions(+), 104 deletions(-) create mode 100644 akka-remote/src/test/scala/akka/remote/artery/compress/ActorRefCompressionIntegrationSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/compress/ClassManifestCompressionIntegrationSpec.scala diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala index c066790e79..e31a23ad03 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala @@ -81,9 +81,7 @@ private[akka] object ArterySettings { private[akka] final class Compression private[ArterySettings] (config: Config) { import config._ - // Compile time constants final val Enabled = true - final val Debug = false // unlocks additional very verbose debug logging of compression events (on DEBUG log level) object ActorRefs { val config = getConfig("actor-refs") @@ -100,4 +98,8 @@ private[akka] object ArterySettings { val Max = getInt("max") } } + object Compression { + // Compile time constants + final val Debug = false // unlocks additional very verbose debug logging of compression events (on DEBUG log level) + } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index a07a1e12b6..3aa3c6bfd6 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -791,8 +791,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def createInboundCompressions(inboundContext: InboundContext): InboundCompressions = - if (settings.Advanced.Compression.Enabled) new InboundCompressionsImpl(system, inboundContext, settings.Advanced.Compression) - else NoInboundCompressions + new InboundCompressionsImpl(system, inboundContext, settings.Advanced.Compression) def createEncoder(pool: EnvelopeBufferPool): Flow[OutboundEnvelope, EnvelopeBuffer, ChangeOutboundCompression] = Flow.fromGraph(new Encoder(localAddress, system, outboundEnvelopePool, pool)) diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index dc07002d35..6773dfdeb8 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -178,12 +178,14 @@ private[remote] final class HeaderBuilderImpl( override def inboundActorRefCompressionTableVersion: Int = _inboundActorRefCompressionTableVersion override def inboundClassManifestCompressionTableVersion: Int = _inboundClassManifestCompressionTableVersion - def setOutboundActorRefCompression(table: CompressionTable[ActorRef]): Unit = + def setOutboundActorRefCompression(table: CompressionTable[ActorRef]): Unit = { _outboundActorRefCompression = table + } override def outboundActorRefCompression: CompressionTable[ActorRef] = _outboundActorRefCompression - def setOutboundClassManifestCompression(table: CompressionTable[String]): Unit = + def setOutboundClassManifestCompression(table: CompressionTable[String]): Unit = { _outboundClassManifestCompression = table + } def outboundClassManifestCompression: CompressionTable[String] = _outboundClassManifestCompression override def setSenderActorRef(ref: ActorRef): Unit = { diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 5c2243d97b..cd6380fe9f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -199,7 +199,7 @@ private[remote] class Decoder( inboundContext: InboundContext, system: ExtendedActorSystem, resolveActorRefWithLocalAddress: String ⇒ InternalActorRef, - compression: InboundCompressions, // TODO has to do demuxing on remote address It would seem, as decoder does not yet know + compression: InboundCompressions, bufferPool: EnvelopeBufferPool, inEnvelopePool: ObjectPool[ReusableInboundEnvelope]) extends GraphStage[FlowShape[EnvelopeBuffer, InboundEnvelope]] { import Decoder.Tick 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 index d07866e239..244d025c96 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala @@ -4,8 +4,11 @@ package akka.remote.artery.compress +import java.util +import java.util.Comparator + /** INTERNAL API: Versioned compression table to be advertised between systems */ -private[akka] final case class CompressionTable[T](version: Int, map: Map[T, Int]) { +private[artery] final case class CompressionTable[T](version: Int, map: Map[T, Int]) { import CompressionTable.NotCompressedId def compress(value: T): Int = @@ -24,16 +27,36 @@ private[akka] final case class CompressionTable[T](version: Int, map: Map[T, Int require(map.values.sum + map.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: " + map) - val vals = map.toList.sortBy(_._2).iterator.map(_._1) - val dtab = Array.ofDim[Object](map.size).asInstanceOf[Array[T]] - vals.copyToArray(dtab) // TODO HEAVY, AVOID COPYING AND THE MAP ETC!!! - DecompressionTable[T](version, dtab) + val tups = Array.ofDim[(Object, Int)](map.size).asInstanceOf[Array[(T, Int)]] + val ts = Array.ofDim[Object](map.size).asInstanceOf[Array[T]] + + var i = 0 + val mit = map.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](version, ts) } } /** INTERNAL API */ -private[remote] object CompressionTable { +private[artery] 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, 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 index e25ce85489..d6ade2eae8 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala @@ -5,7 +5,7 @@ package akka.remote.artery.compress /** INTERNAL API */ -private[remote] final case class DecompressionTable[T](version: Int, table: Array[T]) { +private[artery] final case class DecompressionTable[T](version: Int, table: Array[T]) { // TODO version maybe better as Long? // OR implement roll-over private[this] val length = table.length @@ -29,7 +29,7 @@ private[remote] final case class DecompressionTable[T](version: Int, table: Arra } /** INTERNAL API */ -private[remote] object DecompressionTable { +private[artery] object DecompressionTable { private[this] val _empty = DecompressionTable(0, Array.empty) def empty[T] = _empty.asInstanceOf[DecompressionTable[T]] } 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 index 7b09b10264..c558d57853 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala @@ -8,13 +8,14 @@ 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, NoLogging } import akka.remote.artery.{ ArterySettings, InboundContext, OutboundContext } import akka.util.{ OptionVal, PrettyDuration } import org.agrona.collections.Long2ObjectHashMap +import scala.annotation.tailrec + /** * INTERNAL API * Decompress and cause compression advertisements. @@ -66,17 +67,23 @@ private[remote] final class InboundCompressionsImpl( override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = actorRefsIn(originUid).decompress(tableVersion, idx) - override def hitActorRef(originUid: Long, address: Address, ref: ActorRef, n: Int): Unit = + 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).increment(address, ref, n) - override def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = + } + + override def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = { actorRefsIn(originUid).confirmAdvertisement(tableVersion) + } // class manifest compression --- override def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] = classManifestsIn(originUid).decompress(tableVersion, idx) - override def hitClassManifest(originUid: Long, address: Address, manifest: String, n: Int): Unit = + 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).increment(address, manifest, n) + } override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = actorRefsIn(originUid).confirmAdvertisement(tableVersion) @@ -114,12 +121,12 @@ private[remote] final class InboundActorRefCompression( /* Since the table is empty here, anything we increment here becomes a heavy hitter immediately. */ def preAllocate(allocations: ActorRef*): Unit = { - allocations foreach { case ref ⇒ increment(null, ref, 100000) } + allocations foreach { ref ⇒ increment(null, ref, 100000) } } override def decompress(tableVersion: Int, idx: Int): OptionVal[ActorRef] = if (idx == 0) OptionVal.Some(system.deadLetters) - else super.decompress(tableVersion, idx) + else super.decompressInternal(tableVersion, idx, 0) scheduleNextTableAdvertisement() override protected def tableAdvertisementInterval = settings.ActorRefs.AdvertisementInterval @@ -140,12 +147,16 @@ final class InboundManifestCompression( scheduleNextTableAdvertisement() override protected def tableAdvertisementInterval = settings.Manifests.AdvertisementInterval - override lazy val log = NoLogging - override def advertiseCompressionTable(outboundContext: OutboundContext, table: CompressionTable[String]): Unit = { - log.debug(s"Advertise ClassManifest compression [$table] to [${outboundContext.remoteAddress}]") + log.debug(s"Advertise {} compression [{}] to [{}]", Logging.simpleName(getClass), table, outboundContext.remoteAddress) 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: Int, idx: Int): OptionVal[String] = + decompressInternal(incomingTableVersion, idx, 0) } /** * INTERNAL API @@ -202,6 +213,9 @@ private[remote] abstract class InboundCompression[T >: Null]( /* ==== COMPRESSION ==== */ + /** Override and specialize if needed, for default compression logic delegate to 3-param overload */ + def decompress(incomingTableVersion: Int, 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), @@ -209,8 +223,10 @@ private[remote] abstract class InboundCompression[T >: Null]( * * @throws UnknownCompressedIdException if given id is not known, this may indicate a bug – such situation should not happen. */ - // not tailrec because we allow special casing in sub-class, however recursion is always at most 1 level deep - def decompress(incomingTableVersion: Int, idx: Int): OptionVal[T] = { + @tailrec final def decompressInternal(incomingTableVersion: Int, 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 @@ -226,14 +242,14 @@ private[remote] abstract class InboundCompression[T >: Null]( if (value != null) OptionVal.Some[T](value) else throw new UnknownCompressedIdException(idx) } else if (incomingTableVersion < activeVersion) { - log.warning("Received value compressed with old table: [{}], current table version is: [{}]", incomingTableVersion, activeVersion) + log.debug("Received value compressed with old table: [{}], current table version is: [{}]", incomingTableVersion, activeVersion) OptionVal.None } else if (incomingTableVersion == current.nextTable.version) { log.debug( "Received first value compressed using the next prepared compression table, flipping to it (version: {})", current.nextTable.version) confirmAdvertisement(incomingTableVersion) - decompress(incomingTableVersion, idx) // recurse, activeTable will not be able to handle this + decompressInternal(incomingTableVersion, idx, attemptCounter + 1) // recurse, activeTable will not be able to handle this } else { // which means that incoming version was > nextTable.version, which likely is a bug log.error( @@ -261,12 +277,8 @@ private[remote] abstract class InboundCompression[T >: Null]( * Add `n` occurance for the given key and call `heavyHittedDetected` if element has become a heavy hitter. * Empty keys are omitted. */ - // TODO not so happy about passing around address here, but in incoming there's no other earlier place to get it? def increment(remoteAddress: Address, value: T, n: Long): Unit = { val count = cms.addObjectAndEstimateCount(value, n) - - // TODO optimise order of these, what is more expensive? - // TODO (now the `previous` is, but if aprox datatype there it would be faster)... Needs pondering. addAndCheckIfheavyHitterDetected(value, count) } @@ -320,6 +332,7 @@ private[remote] abstract class InboundCompression[T >: Null]( */ private[remote] def runNextTableAdvertisement() = { val current = state.get + if (ArterySettings.Compression.Debug) println(s"[compress] runNextTableAdvertisement, state = $current") current.advertisementInProgress match { case None ⇒ inboundContext.association(originUid) match { diff --git a/akka-remote/src/test/scala/akka/remote/artery/compress/ActorRefCompressionIntegrationSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/ActorRefCompressionIntegrationSpec.scala new file mode 100644 index 0000000000..5a202d9bf1 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/ActorRefCompressionIntegrationSpec.scala @@ -0,0 +1,66 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +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 org.scalatest.concurrent.{ Eventually, PatienceConfiguration } + +import scala.concurrent.Await +import scala.concurrent.duration._ + +class ActorRefCompressionIntegrationSpec extends AkkaSpec(CompressionIntegrationSpec.commonConfig) + with ImplicitSender with BeforeAndAfter with Eventually { + import CompressionIntegrationSpec._ + + implicit val t = Timeout(3.seconds) + var systemB = ActorSystem("systemB", configB) + + "Outgoing ActorRef compression table" must { + "compress chatty actor" in { + val messagesToExchange = 10 + + // listen for compression table events + val aProbe = TestProbe() + val b1Probe = TestProbe() + system.eventStream.subscribe(aProbe.ref, classOf[CompressionProtocol.Events.ReceivedActorRefCompressionTable]) + systemB.eventStream.subscribe(b1Probe.ref, classOf[CompressionProtocol.Events.ReceivedActorRefCompressionTable]) + + def voidSel = system.actorSelection(s"artery://systemB@localhost:$portB/user/void") + systemB.actorOf(TestActors.blackholeProps, "void") + + // cause testActor-1 to become a heavy hitter + (1 to messagesToExchange).foreach { i ⇒ voidSel ! "hello" } // does not reply, but a hot receiver should be advertised + + val a1 = aProbe.expectMsgType[Events.ReceivedActorRefCompressionTable](10.seconds) + info("System [A] received: " + a1) + assertCompression[ActorRef](a1.table, 0, _ should ===(system.deadLetters)) + assertCompression[ActorRef](a1.table, 1, _ should ===(testActor)) + } + } + + def assertCompression[T](table: CompressionTable[T], id: Int, assertion: T ⇒ Unit): Unit = { + table.map.find(_._2 == id) + .orElse { throw new AssertionError(s"No key was compressed to the id [$id]! Table was: $table") } + .foreach(i ⇒ assertion(i._1)) + } + + def identify(_system: String, port: Int, name: String): ActorRef = { + val selection = system.actorSelection(s"artery://${_system}@localhost:$port/user/$name").resolveOne(3.seconds) + Await.result(selection, 4.seconds) + } + + 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/ClassManifestCompressionIntegrationSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/ClassManifestCompressionIntegrationSpec.scala new file mode 100644 index 0000000000..919ffdef6c --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/ClassManifestCompressionIntegrationSpec.scala @@ -0,0 +1,98 @@ +/* + * Copyright (C) 2016 Lightbend Inc. + */ + +package akka.remote.artery.compress + +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 org.scalatest.concurrent.{ Eventually, PatienceConfiguration } + +import scala.concurrent.Await +import scala.concurrent.duration._ + +class ClassManifestCompressionIntegrationSpec extends AkkaSpec(CompressionIntegrationSpec.commonConfig) + with ImplicitSender with BeforeAndAfter with Eventually { + import CompressionIntegrationSpec._ + + implicit val t = Timeout(3.seconds) + var systemB = ActorSystem("systemB", configB) + + "Outgoing Manifest compression table" must { + "compress chatty manifest" in { + val messagesToExchange = 10 + + // listen for compression table events + val aProbe = TestProbe() + val b1Probe = TestProbe() + system.eventStream.subscribe(aProbe.ref, classOf[CompressionProtocol.Events.ReceivedClassManifestCompressionTable]) + systemB.eventStream.subscribe(b1Probe.ref, classOf[CompressionProtocol.Events.ReceivedClassManifestCompressionTable]) + systemB.actorOf(TestActors.blackholeProps, "void-2") + + Thread.sleep(1000) + val voidRef = Await.result(system.actorSelection(s"artery://systemB@localhost:$portB/user/void-2").resolveOne(3.second), 3.seconds) + + // cause testActor-1 to become a heavy hitter + (1 to messagesToExchange).foreach { i ⇒ voidRef ! TestMessage("hello") } // does not reply, but a hot receiver should be advertised + + eventually(PatienceConfiguration.Timeout(20.seconds)) { + val a1 = aProbe.expectMsgType[Events.ReceivedClassManifestCompressionTable](10.seconds) + info("System [A] received: " + a1) + assertCompression[String](a1.table, 0, _ should ===("TestMessageManifest")) + } + } + } + + def assertCompression[T](table: CompressionTable[T], id: Int, assertion: T ⇒ Unit): Unit = { + table.map.find(_._2 == id) + .orElse { throw new AssertionError(s"No key was compressed to the id [$id]! Table was: $table") } + .foreach(i ⇒ assertion(i._1)) + } + + def identify(_system: String, port: Int, name: String) = { + val selection = + system.actorSelection(s"artery://${_system}@localhost:$port/user/$name") + val ActorIdentity(1, ref) = Await.result(selection ? Identify(1), 3.seconds) + ref.get + } + + override def afterTermination(): Unit = + shutdownAllActorSystems() + + private def shutdownAllActorSystems(): Unit = { + if (systemB != null) shutdown(systemB) + } +} + +import akka.actor.ExtendedActorSystem +import akka.serialization.SerializerWithStringManifest + +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/CompressionIntegrationSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala index 4fd819062c..1a0d919873 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala @@ -4,16 +4,8 @@ package akka.remote.artery.compress -import akka.actor._ -import akka.remote.artery.compress.CompressionProtocol.Events -import akka.testkit._ -import akka.util.Timeout +import akka.testkit.SocketUtil import com.typesafe.config.ConfigFactory -import org.scalatest.BeforeAndAfter -import akka.pattern.ask - -import scala.concurrent.Await -import scala.concurrent.duration._ object CompressionIntegrationSpec { // need the port before systems are started @@ -23,17 +15,24 @@ object CompressionIntegrationSpec { akka { loglevel = INFO - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.artery.enabled = on - remote.artery.advanced { - compression.debug = on + actor { + provider = "akka.remote.RemoteActorRefProvider" + + serializers { + test-message = "akka.remote.artery.compress.TestMessageSerializer" + } + serialization-bindings { + "akka.remote.artery.compress.TestMessage" = test-message + } } + remote.artery.enabled = on remote.artery.hostname = localhost remote.artery.port = 0 remote.handshake-timeout = 10s remote.artery.advanced.compression { actor-refs.advertisement-interval = 3 seconds + manifests.advertisement-interval = 3 seconds } } @@ -42,62 +41,3 @@ object CompressionIntegrationSpec { val configB = ConfigFactory.parseString(s"akka.remote.artery.port = $portB") .withFallback(commonConfig) } - -class CompressionIntegrationSpec extends AkkaSpec(CompressionIntegrationSpec.commonConfig) - with ImplicitSender with BeforeAndAfter { - import CompressionIntegrationSpec._ - - implicit val t = Timeout(3.seconds) - var systemB: ActorSystem = null - - before { - systemB = ActorSystem("systemB", configB) - } - - "Outgoing compression table" must { - "compress chatty actor" in { - val messagesToExchange = 10 - - // listen for compression table events - val aProbe = TestProbe() - val b1Probe = TestProbe() - system.eventStream.subscribe(aProbe.ref, classOf[CompressionProtocol.Events.Event]) - systemB.eventStream.subscribe(b1Probe.ref, classOf[CompressionProtocol.Events.Event]) - - def voidSel = system.actorSelection(s"artery://systemB@localhost:$portB/user/void") - systemB.actorOf(TestActors.blackholeProps, "void") - - // cause testActor-1 to become a heavy hitter - (1 to messagesToExchange).foreach { i ⇒ voidSel ! "hello" } // does not reply, but a hot receiver should be advertised - - val a1 = aProbe.expectMsgType[Events.ReceivedActorRefCompressionTable](10.seconds) - info("System [A] received: " + a1) - assertCompression[ActorRef](a1.table, 0, _ should ===(system.deadLetters)) - assertCompression[ActorRef](a1.table, 1, _ should ===(testActor)) - } - } - - def assertCompression[T](table: CompressionTable[T], id: Int, assertion: T ⇒ Unit): Unit = { - table.map.find(_._2 == id) - .orElse { throw new AssertionError(s"No key was compressed to the id [$id]! Table was: $table") } - .foreach(i ⇒ assertion(i._1)) - } - - def identify(_system: String, port: Int, name: String) = { - val selection = - system.actorSelection(s"artery://${_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) - } -} From 3d3a3528bf37f07930cd275f3f629178b64039b3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Fri, 2 Sep 2016 16:40:38 +0200 Subject: [PATCH 100/186] Eliminated JVM crashing race on transport stop (#21337) * Eliminated JVM crashing race on transport stop * Fail explicitly if started more than once --- .../akka/remote/artery/ArteryTransport.scala | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 3aa3c6bfd6..16f82393c2 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -6,10 +6,9 @@ package akka.remote.artery import java.io.File import java.net.InetSocketAddress import java.nio.channels.{ DatagramChannel, FileChannel } - import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.TimeUnit -import java.util.concurrent.atomic.AtomicLong +import java.util.concurrent.atomic.{ AtomicLong, AtomicReference } import scala.annotation.tailrec import scala.collection.JavaConverters._ @@ -20,7 +19,6 @@ import scala.util.Failure import scala.util.Success import scala.util.Try import scala.util.control.NonFatal - import akka.Done import akka.NotUsed import akka.actor._ @@ -276,7 +274,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R @volatile private[this] var materializer: Materializer = _ @volatile private[this] var controlSubject: ControlMessageSubject = _ @volatile private[this] var messageDispatcher: MessageDispatcher = _ - @volatile private[this] var mediaDriver: Option[MediaDriver] = None + private[this] val mediaDriver = new AtomicReference[Option[MediaDriver]](None) @volatile private[this] var aeron: Aeron = _ @volatile private[this] var aeronErrorLogTask: Cancellable = _ @@ -438,19 +436,24 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R log.debug("Started embedded media driver in directory [{}]", driver.aeronDirectoryName) topLevelFREvents.loFreq(Transport_MediaDriverStarted, driver.aeronDirectoryName().getBytes("US-ASCII")) Runtime.getRuntime.addShutdownHook(stopMediaDriverShutdownHook) - mediaDriver = Some(driver) + if (!mediaDriver.compareAndSet(None, Some(driver))) { + throw new IllegalStateException("media driver started more than once") + } } } - private def aeronDir: String = mediaDriver match { + private def aeronDir: String = mediaDriver.get match { case Some(driver) ⇒ driver.aeronDirectoryName case None ⇒ settings.Advanced.AeronDirectoryName } private def stopMediaDriver(): Unit = { - mediaDriver.foreach { driver ⇒ + // 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) @@ -461,8 +464,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R "Couldn't delete Aeron embedded media driver files in [{}] due to [{}]", driver.aeronDirectoryName, e.getMessage) } + Try(Runtime.getRuntime.removeShutdownHook(stopMediaDriverShutdownHook)) } - Try(Runtime.getRuntime.removeShutdownHook(stopMediaDriverShutdownHook)) } // TODO: Add FR events @@ -683,7 +686,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R topLevelFREvents.loFreq(Transport_AeronErrorLogTaskStopped, NoMetaData) } if (aeron != null) aeron.close() - if (mediaDriver.isDefined) { + if (mediaDriver.get.isDefined) { stopMediaDriver() topLevelFREvents.loFreq(Transport_MediaFileDeleted, NoMetaData) } From faf941b4c8648ddf5dbcfb110a352363d0e1917e Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 30 Aug 2016 14:37:11 +0200 Subject: [PATCH 101/186] support for parallel lanes, #21207 * for parallel serialziation/deserialization * MergeHub for the outbound lanes * BroadcastHub + filter for the inbound lanes, until we have a PartitionHub * simplify materialization of test stage * add RemoteSendConsistencyWithThreeLanesSpec --- .../akka/remote/artery/CodecBenchmark.scala | 4 +- .../artery/AeronStreamConcistencySpec.scala | 2 +- .../artery/AeronStreamLatencySpec.scala | 2 +- .../artery/AeronStreamMaxThroughputSpec.scala | 2 +- .../akka/remote/artery/LatencySpec.scala | 60 +- .../remote/artery/MaxThroughputSpec.scala | 40 +- .../akka/remote/artery/TestMessage.scala | 65 + .../remote/artery/protobuf/TestMessages.java | 1903 +++++++++++++++++ .../src/test/protobuf/TestMessages.proto | 20 + akka-remote/src/main/resources/reference.conf | 15 + .../akka/remote/RemoteActorRefProvider.scala | 10 +- .../akka/remote/artery/ArterySettings.scala | 38 +- .../akka/remote/artery/ArteryTransport.scala | 211 +- .../akka/remote/artery/Association.scala | 254 ++- .../scala/akka/remote/artery/Codecs.scala | 93 +- .../akka/remote/artery/InboundEnvelope.scala | 43 +- .../scala/akka/remote/artery/TestStage.scala | 202 +- .../akka/remote/artery/AeronSinkSpec.scala | 2 +- .../remote/artery/HandshakeFailureSpec.scala | 3 +- .../artery/LargeMessagesStreamSpec.scala | 10 +- .../RemoteMessageSerializationSpec.scala | 3 +- .../artery/RemoteSendConsistencySpec.scala | 42 +- .../akka/remote/artery/TestContext.scala | 8 +- .../compress/CompressionIntegrationSpec.scala | 2 +- ...dshakeShouldDropCompressionTableSpec.scala | 2 +- 25 files changed, 2653 insertions(+), 383 deletions(-) create mode 100644 akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/TestMessage.scala create mode 100644 akka-remote-tests/src/test/java/akka/remote/artery/protobuf/TestMessages.java create mode 100644 akka-remote-tests/src/test/protobuf/TestMessages.proto 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 index 98974a245b..b2b9cf121a 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -55,12 +55,12 @@ class CodecBenchmark { implicit val system = ActorSystem("CodecBenchmark", config) val systemB = ActorSystem("systemB", system.settings.config) - private val envelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) + 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(ArteryTransport.MaximumFrameSize).order(ByteOrder.LITTLE_ENDIAN) + val envelopeTemplateBuffer = ByteBuffer.allocate(1024 * 1024).order(ByteOrder.LITTLE_ENDIAN) val uniqueLocalAddress = UniqueAddress( system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress, 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 index 3521768011..9103899eb4 100644 --- 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 @@ -69,7 +69,7 @@ abstract class AeronStreamConsistencySpec r } - val pool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) + val pool = new EnvelopeBufferPool(1024 * 1024, 128) lazy implicit val mat = ActorMaterializer()(system) import system.dispatcher 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 index 240a1c1c2f..4b0fd67dd0 100644 --- 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 @@ -86,7 +86,7 @@ abstract class AeronStreamLatencySpec val driver = MediaDriver.launchEmbedded() - val pool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) + val pool = new EnvelopeBufferPool(1024 * 1024, 128) val stats = new AeronStat(AeronStat.mapCounters(new File(driver.aeronDirectoryName, CncFileDescriptor.CNC_FILE))) 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 index 27689d59d3..a1f9a7ee4d 100644 --- 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 @@ -84,7 +84,7 @@ abstract class AeronStreamMaxThroughputSpec val driver = MediaDriver.launchEmbedded() - val pool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) + val pool = new EnvelopeBufferPool(1024 * 1024, 128) val stats = new AeronStat(AeronStat.mapCounters(new File(driver.aeronDirectoryName, CncFileDescriptor.CNC_FILE))) 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 index 896e48124d..c2251ed9f4 100644 --- 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 @@ -30,6 +30,7 @@ object LatencySpec extends MultiNodeConfig { # 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 @@ -86,18 +87,24 @@ object LatencySpec extends MultiNodeConfig { def receive = { case bytes: Array[Byte] ⇒ if (bytes.length != 0) { - if (count == 0) - startTime = System.nanoTime() if (bytes.length != payloadSize) throw new IllegalArgumentException("Invalid message") - reporter.onMessage(1, payloadSize) - count += 1 - val d = System.nanoTime() - sendTimes.get(count - 1) - histogram.recordValue(d) - if (count == totalMessages) { - printTotal(testName, bytes.length, histogram, System.nanoTime() - startTime) - context.stop(self) - } + 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) + histogram.recordValue(d) + if (count == totalMessages) { + printTotal(testName, size, histogram, System.nanoTime() - startTime) + context.stop(self) + } } def printTotal(testName: String, payloadSize: Long, histogram: Histogram, totalDurationNanos: Long): Unit = { @@ -133,7 +140,8 @@ object LatencySpec extends MultiNodeConfig { testName: String, messageRate: Int, // msg/s payloadSize: Int, - repeat: Int) + repeat: Int, + realMessage: Boolean) } @@ -148,6 +156,7 @@ abstract class 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() @@ -183,32 +192,38 @@ abstract class LatencySpec testName = "warmup", messageRate = 10000, payloadSize = 100, - repeat = repeatCount), + repeat = repeatCount, + realMessage), TestSettings( testName = "rate-100-size-100", messageRate = 100, payloadSize = 100, - repeat = repeatCount), + repeat = repeatCount, + realMessage), TestSettings( testName = "rate-1000-size-100", messageRate = 1000, payloadSize = 100, - repeat = repeatCount), + repeat = repeatCount, + realMessage), TestSettings( testName = "rate-10000-size-100", messageRate = 10000, payloadSize = 100, - repeat = repeatCount), + repeat = repeatCount, + realMessage), TestSettings( testName = "rate-20000-size-100", messageRate = 20000, payloadSize = 100, - repeat = repeatCount), + repeat = repeatCount, + realMessage), TestSettings( testName = "rate-1000-size-1k", messageRate = 1000, payloadSize = 1000, - repeat = repeatCount)) + repeat = repeatCount, + realMessage)) def test(testSettings: TestSettings): Unit = { import testSettings._ @@ -259,6 +274,17 @@ abstract class LatencySpec 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 } 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 index 335d3bedbe..0932066320 100644 --- 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 @@ -31,6 +31,7 @@ object MaxThroughputSpec extends MultiNodeConfig { 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 @@ -44,9 +45,11 @@ object MaxThroughputSpec extends MultiNodeConfig { 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 { @@ -85,6 +88,9 @@ object MaxThroughputSpec extends MultiNodeConfig { 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 @@ -194,8 +200,19 @@ object MaxThroughputSpec extends MultiNodeConfig { val batchSize = math.min(remaining, burstSize) var i = 0 while (i < batchSize) { - // target ! payload - target.tell(payload, ActorRef.noSender) + 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 @@ -214,7 +231,8 @@ object MaxThroughputSpec extends MultiNodeConfig { totalMessages: Long, burstSize: Int, payloadSize: Int, - senderReceiverPairs: 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) } @@ -267,6 +285,7 @@ abstract class MaxThroughputSpec 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() @@ -302,31 +321,36 @@ abstract class MaxThroughputSpec totalMessages = adjustedTotalMessages(20000), burstSize = 1000, payloadSize = 100, - senderReceiverPairs = 1), + senderReceiverPairs = 1, + realMessage), TestSettings( testName = "1-to-1", totalMessages = adjustedTotalMessages(50000), burstSize = 1000, payloadSize = 100, - senderReceiverPairs = 1), + senderReceiverPairs = 1, + realMessage), TestSettings( testName = "1-to-1-size-1k", totalMessages = adjustedTotalMessages(20000), burstSize = 1000, payloadSize = 1000, - senderReceiverPairs = 1), + senderReceiverPairs = 1, + realMessage), TestSettings( testName = "1-to-1-size-10k", totalMessages = adjustedTotalMessages(10000), burstSize = 1000, payloadSize = 10000, - senderReceiverPairs = 1), + 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)) + senderReceiverPairs = 5, + realMessage)) def test(testSettings: TestSettings): Unit = { import testSettings._ 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/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/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index bf0fbd141c..99a1116e6a 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -152,6 +152,18 @@ akka { # Level 1 strongly prefer low CPU consumption over low latency. # Level 10 strongly prefer low latency over low CPU consumption. idle-cpu-level = 5 + + # 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 + + # 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 # This setting defines the maximum number of unacknowledged system messages # allowed for a remote system. If this limit is reached the remote system is @@ -161,6 +173,9 @@ akka { # unacknowledged system messages are re-delivered with this interval system-message-resend-interval = 1 second + # The timeout for outbound associations to perform the handshake. + handshake-timeout = 15 s + # incomplete handshake attempt is retried with this interval handshake-retry-interval = 1 second diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 8f1735cdda..914fc409b2 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -455,14 +455,6 @@ private[akka] trait RemoteRef extends ActorRefScope { final def isLocal = false } -/** - * INTERNAL API - */ -private[remote] sealed abstract class MessageDestinationFlag -private[remote] case object RegularDestination extends MessageDestinationFlag -private[remote] case object LargeDestination extends MessageDestinationFlag -private[remote] case object PriorityDestination extends MessageDestinationFlag - /** * INTERNAL API * Remote ActorRef that is used when referencing the Actor on a different node than its "home" node. @@ -488,7 +480,7 @@ private[akka] class RemoteActorRef private[akka] ( @volatile private[remote] var cachedAssociation: artery.Association = null // used by artery to direct messages to separate specialized streams - @volatile private[remote] var cachedMessageDestinationFlag: MessageDestinationFlag = null + @volatile private[remote] var cachedSendQueueIndex: Int = -1 def getChild(name: Iterator[String]): InternalActorRef = { val s = name.toStream diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala index e31a23ad03..365f71daf8 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala @@ -51,25 +51,43 @@ private[akka] final class ArterySettings private (config: Config) { 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 SystemMessageResendInterval = config.getMillisDuration("system-message-resend-interval").requiring(interval ⇒ - interval > 0.seconds, "system-message-resend-interval must be more than zero") + 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 > 0.seconds, "handshake-retry-interval must be more than zero") + interval > Duration.Zero, "handshake-retry-interval must be more than zero") val InjectHandshakeInterval = config.getMillisDuration("inject-handshake-interval").requiring(interval ⇒ - interval > 0.seconds, "inject-handshake-interval must be more than zero") - val GiveUpSendAfter = config.getMillisDuration("give-up-send-after") - val ShutdownFlushTimeout = config.getMillisDuration("shutdown-flush-timeout") - val InboundRestartTimeout = config.getMillisDuration("inbound-restart-timeout") + interval > Duration.Zero, "inject-handshake-interval must be more than zero") + val GiveUpSendAfter = config.getMillisDuration("give-up-send-after").requiring(interval ⇒ + interval > Duration.Zero, "give-up-send-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") + 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 ClientLivenessTimeout = config.getMillisDuration("client-liveness-timeout") - val ImageLivenessTimeoutNs = config.getMillisDuration("image-liveness-timeout") - val DriverTimeout = config.getMillisDuration("driver-timeout") + val ClientLivenessTimeout = config.getMillisDuration("client-liveness-timeout").requiring(interval ⇒ + interval > Duration.Zero, "client-liveness-timeout must be more than zero") + val ImageLivenessTimeoutNs = config.getMillisDuration("image-liveness-timeout").requiring(interval ⇒ + interval > Duration.Zero, "image-liveness-timeout must be more than zero") + 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 Compression = new Compression(getConfig("compression")) + + final val MaximumFrameSize = 1024 * 1024 + final val MaximumPooledBuffers = 128 + final val MaximumLargeFrameSize = MaximumFrameSize * 5 + final val InboundBroadcastHubBufferSize = MaximumPooledBuffers / 2 } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 16f82393c2..40bbfd709f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -68,6 +68,7 @@ import io.aeron.exceptions.ConductorServiceTimeoutException import org.agrona.ErrorHandler import org.agrona.IoUtil import org.agrona.concurrent.BackoffIdleStrategy +import akka.stream.scaladsl.BroadcastHub /** * INTERNAL API @@ -293,10 +294,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val testStages: CopyOnWriteArrayList[TestManagementApi] = new CopyOnWriteArrayList - private val handshakeTimeout: FiniteDuration = - system.settings.config.getMillisDuration("akka.remote.handshake-timeout").requiring( - _ > Duration.Zero, - "handshake-timeout must be > 0") + private val inboundLanes = settings.Advanced.InboundLanes private val remoteDispatcher = system.dispatchers.lookup(settings.Dispatcher) @@ -317,15 +315,15 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private def outboundChannel(a: Address) = s"aeron:udp?endpoint=${a.host.get}:${a.port.get}" private val controlStreamId = 1 - private val ordinaryStreamId = 3 - private val largeStreamId = 4 + 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 envelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) - private val largeEnvelopePool = new EnvelopeBufferPool(ArteryTransport.MaximumLargeFrameSize, ArteryTransport.MaximumPooledBuffers) + private val envelopeBufferPool = new EnvelopeBufferPool(settings.Advanced.MaximumFrameSize, settings.Advanced.MaximumPooledBuffers) + private val largeEnvelopeBufferPool = new EnvelopeBufferPool(settings.Advanced.MaximumLargeFrameSize, settings.Advanced.MaximumPooledBuffers) private val inboundEnvelopePool = ReusableInboundEnvelope.createObjectPool(capacity = 16) // FIXME capacity of outboundEnvelopePool should probably be derived from the sendQueue capacity @@ -528,22 +526,14 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundControlStream(compression: InboundCompressions): Unit = { - val (ctrl, completed) = - if (settings.Advanced.TestMode) { - val (mgmt, (ctrl, completed)) = - aeronSource(controlStreamId, envelopePool) - .via(inboundFlow(compression)) - .viaMat(inboundTestFlow)(Keep.right) - .toMat(inboundControlSink)(Keep.both) - .run()(materializer) - testStages.add(mgmt) - (ctrl, completed) - } else { - aeronSource(controlStreamId, envelopePool) - .via(inboundFlow(compression)) - .toMat(inboundControlSink)(Keep.right) - .run()(materializer) - } + val (testMgmt, ctrl, completed) = + aeronSource(controlStreamId, envelopeBufferPool) + .via(inboundFlow(compression)) + .toMat(inboundControlSink)(Keep.right) + .run()(materializer) + + if (settings.Advanced.TestMode) + testStages.add(testMgmt) controlSubject = ctrl @@ -604,19 +594,54 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private def runInboundOrdinaryMessagesStream(compression: InboundCompressions): Unit = { val completed = - if (settings.Advanced.TestMode) { - val (mgmt, c) = aeronSource(ordinaryStreamId, envelopePool) + if (inboundLanes == 1) { + val (testMgmt, completed) = aeronSource(ordinaryStreamId, envelopeBufferPool) .via(inboundFlow(compression)) - .viaMat(inboundTestFlow)(Keep.right) - .toMat(inboundSink)(Keep.both) + .toMat(inboundSink(envelopeBufferPool))(Keep.right) .run()(materializer) - testStages.add(mgmt) - c + + if (settings.Advanced.TestMode) + testStages.add(testMgmt) + + completed + } else { - aeronSource(ordinaryStreamId, envelopePool) + val source = aeronSource(ordinaryStreamId, envelopeBufferPool) .via(inboundFlow(compression)) - .toMat(inboundSink)(Keep.right) - .run()(materializer) + .map(env ⇒ (env.recipient, env)) + + val broadcastHub = source.runWith(BroadcastHub.sink(bufferSize = settings.Advanced.InboundBroadcastHubBufferSize))(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 values: Vector[(TestManagementApi, 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) + + val (testMgmtValues, completedValues) = values.unzip + + if (settings.Advanced.TestMode) + testMgmtValues.foreach(testStages.add) + + import system.dispatcher + val completed = Future.sequence(completedValues).map(_ ⇒ Done) + + completed } attachStreamRestart("Inbound message stream", completed, () ⇒ runInboundOrdinaryMessagesStream(compression)) @@ -625,21 +650,13 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private def runInboundLargeMessagesStream(): Unit = { val disableCompression = NoInboundCompressions // no compression on large message stream for now - val completed = - if (settings.Advanced.TestMode) { - val (mgmt, c) = aeronSource(largeStreamId, largeEnvelopePool) - .via(inboundLargeFlow(disableCompression)) - .viaMat(inboundTestFlow)(Keep.right) - .toMat(inboundSink)(Keep.both) - .run()(materializer) - testStages.add(mgmt) - c - } else { - aeronSource(largeStreamId, largeEnvelopePool) - .via(inboundLargeFlow(disableCompression)) - .toMat(inboundSink)(Keep.right) - .run()(materializer) - } + val (testMgmt, completed) = aeronSource(largeStreamId, largeEnvelopeBufferPool) + .via(inboundLargeFlow(disableCompression)) + .toMat(inboundSink(largeEnvelopeBufferPool))(Keep.right) + .run()(materializer) + + if (settings.Advanced.TestMode) + testStages.add(testMgmt) attachStreamRestart("Inbound large message stream", completed, () ⇒ runInboundLargeMessagesStream()) } @@ -753,44 +770,59 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R association(remoteAddress).quarantine(reason = "", uid.map(_.toLong)) } - def outbound(outboundContext: OutboundContext): Sink[OutboundEnvelope, (ChangeOutboundCompression, Future[Done])] = - createOutboundSink(ordinaryStreamId, outboundContext, envelopePool) - def outboundLarge(outboundContext: OutboundContext): Sink[OutboundEnvelope, Future[Done]] = - createOutboundSink(largeStreamId, outboundContext, largeEnvelopePool) + 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])] = { - Flow.fromGraph(killSwitch.flow[OutboundEnvelope]) - .via(new OutboundHandshake(system, outboundContext, outboundEnvelopePool, handshakeTimeout, - settings.Advanced.HandshakeRetryInterval, settings.Advanced.InjectHandshakeInterval)) - .viaMat(createEncoder(bufferPool))(Keep.right) - .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), streamId, aeron, taskRunner, - envelopePool, settings.Advanced.GiveUpSendAfter, createFlightRecorderEventSink()))(Keep.both) + outboundLane(outboundContext, bufferPool) + .toMat(aeronSink(outboundContext, streamId))(Keep.both) } - /** - * The outbound stream is defined as two parts to be able to add test stage in-between. - * System messages must not be dropped before the SystemMessageDelivery stage. - */ - def outboundControlPart1(outboundContext: OutboundContext): Flow[OutboundEnvelope, OutboundEnvelope, SharedKillSwitch] = { + def aeronSink(outboundContext: OutboundContext): Sink[EnvelopeBuffer, Future[Done]] = + aeronSink(outboundContext, ordinaryStreamId) + + private def aeronSink(outboundContext: OutboundContext, streamId: Int): Sink[EnvelopeBuffer, Future[Done]] = { + Sink.fromGraph(new AeronSink(outboundChannel(outboundContext.remoteAddress), streamId, aeron, taskRunner, + envelopeBufferPool, settings.Advanced.GiveUpSendAfter, 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, handshakeTimeout, + .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, (TestManagementApi, 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)) - - // FIXME we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages - } - - def outboundControlPart2(outboundContext: OutboundContext): Sink[OutboundEnvelope, (OutboundControlIngress, Future[Done])] = { - Flow[OutboundEnvelope] - .viaMat(new OutboundControlJunction(outboundContext, outboundEnvelopePool))(Keep.right) - .via(createEncoder(envelopePool)) + // note that System messages must not be dropped before the SystemMessageDelivery stage + .viaMat(outboundTestFlow(outboundContext))(Keep.right) + .viaMat(new OutboundControlJunction(outboundContext, outboundEnvelopePool))(Keep.both) + .via(createEncoder(envelopeBufferPool)) .toMat(new AeronSink(outboundChannel(outboundContext.remoteAddress), controlStreamId, aeron, taskRunner, - envelopePool, Duration.Inf, createFlightRecorderEventSink()))(Keep.both) + envelopeBufferPool, Duration.Inf, createFlightRecorderEventSink()))(Keep.both) + .mapMaterializedValue { + case ((a, b), c) ⇒ (a, b, c) + } + + // 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 = @@ -819,34 +851,41 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R inboundEnvelopePool)) } - def decoder(compression: InboundCompressions): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = - createDecoder(compression, envelopePool) + def createDeserializer(bufferPool: EnvelopeBufferPool): Flow[InboundEnvelope, InboundEnvelope, NotUsed] = + Flow.fromGraph(new Deserializer(this, system, bufferPool)) - def inboundSink: Sink[InboundEnvelope, Future[Done]] = + def inboundSink(bufferPool: EnvelopeBufferPool): Sink[InboundEnvelope, (TestManagementApi, Future[Done])] = Flow[InboundEnvelope] + .via(createDeserializer(bufferPool)) + .viaMat(new InboundTestStage(this, settings.Advanced.TestMode))(Keep.right) .via(new InboundHandshake(this, inControlStream = false)) .via(new InboundQuarantineCheck(this)) - .toMat(messageDispatcherSink)(Keep.right) + .toMat(messageDispatcherSink)(Keep.both) def inboundFlow(compression: InboundCompressions): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { Flow[EnvelopeBuffer] .via(killSwitch.flow) - .via(decoder(compression)) + .via(createDecoder(compression, envelopeBufferPool)) } def inboundLargeFlow(compression: InboundCompressions): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { Flow[EnvelopeBuffer] .via(killSwitch.flow) - .via(createDecoder(compression, largeEnvelopePool)) + .via(createDecoder(compression, largeEnvelopeBufferPool)) } - def inboundControlSink: Sink[InboundEnvelope, (ControlMessageSubject, Future[Done])] = { + def inboundControlSink: Sink[InboundEnvelope, (TestManagementApi, ControlMessageSubject, Future[Done])] = { Flow[InboundEnvelope] + .via(createDeserializer(envelopeBufferPool)) + .viaMat(new InboundTestStage(this, settings.Advanced.TestMode))(Keep.right) .via(new InboundHandshake(this, inControlStream = true)) .via(new InboundQuarantineCheck(this)) - .viaMat(new InboundControlJunction)(Keep.right) + .viaMat(new InboundControlJunction)(Keep.both) .via(new SystemMessageAcker(this)) .toMat(messageDispatcherSink)(Keep.both) + .mapMaterializedValue { + case ((a, b), c) ⇒ (a, b, c) + } } private def initializeFlightRecorder(): Option[(FileChannel, File, FlightRecorder)] = { @@ -861,11 +900,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R None } - def inboundTestFlow: Flow[InboundEnvelope, InboundEnvelope, TestManagementApi] = - Flow.fromGraph(new InboundTestStage(this)) - - def outboundTestFlow(association: Association): Flow[OutboundEnvelope, OutboundEnvelope, TestManagementApi] = - Flow.fromGraph(new OutboundTestStage(association)) + def outboundTestFlow(outboundContext: OutboundContext): Flow[OutboundEnvelope, OutboundEnvelope, TestManagementApi] = + Flow.fromGraph(new OutboundTestStage(outboundContext, settings.Advanced.TestMode)) /** INTERNAL API: for testing only. */ private[remote] def triggerCompressionAdvertisements(actorRef: Boolean, manifest: Boolean) = { @@ -888,9 +924,6 @@ private[remote] object ArteryTransport { val ProtocolName = "artery" val Version = 0 - val MaximumFrameSize = 1024 * 1024 - val MaximumPooledBuffers = 256 - val MaximumLargeFrameSize = MaximumFrameSize * 5 /** * Internal API diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 55db97720e..552ce77b65 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -42,6 +42,7 @@ import akka.util.{ Unsafe, WildcardIndex } import akka.util.OptionVal import org.agrona.concurrent.ManyToOneConcurrentArrayQueue import akka.remote.artery.compress.CompressionProtocol._ +import akka.stream.scaladsl.MergeHub /** * INTERNAL API @@ -50,6 +51,10 @@ private[remote] object Association { final case class QueueWrapper(queue: Queue[OutboundEnvelope]) extends SendQueue.ProducerApi[OutboundEnvelope] { override def offer(message: OutboundEnvelope): Boolean = queue.offer(message) } + + final val ControlQueueIndex = 0 + final val LargeQueueIndex = 1 + final val OrdinaryQueueIndex = 2 } /** @@ -70,11 +75,6 @@ private[remote] class Association( import Association._ private val log = Logging(transport.system, getClass.getName) - private val controlQueueSize = transport.settings.Advanced.SysMsgBufferSize - // FIXME config queue size, and it should perhaps also be possible to use some kind of LinkedQueue - // such as agrona.ManyToOneConcurrentLinkedQueue or AbstractNodeQueue for less memory consumption - private val queueSize = 3072 - private val largeQueueSize = 256 private val restartCounter = new RestartCounter(transport.settings.Advanced.OutboundMaxRestarts, transport.settings.Advanced.OutboundRestartTimeout) @@ -85,30 +85,57 @@ private[remote] class Association( def createQueue(capacity: Int): Queue[OutboundEnvelope] = new ManyToOneConcurrentArrayQueue[OutboundEnvelope](capacity) - @volatile private[this] var queue: SendQueue.ProducerApi[OutboundEnvelope] = QueueWrapper(createQueue(queueSize)) - @volatile private[this] var largeQueue: SendQueue.ProducerApi[OutboundEnvelope] = QueueWrapper(createQueue(largeQueueSize)) - @volatile private[this] var controlQueue: SendQueue.ProducerApi[OutboundEnvelope] = QueueWrapper(createQueue(controlQueueSize)) + private val outboundLanes = transport.settings.Advanced.OutboundLanes + private val controlQueueSize = transport.settings.Advanced.SysMsgBufferSize + // FIXME config queue size, and it should perhaps also be possible to use some kind of LinkedQueue + // such as agrona.ManyToOneConcurrentLinkedQueue or AbstractNodeQueue for less memory consumption + private val queueSize = 3072 + private val largeQueueSize = 256 + + private[this] val queues: Array[SendQueue.ProducerApi[OutboundEnvelope]] = Array.ofDim(2 + outboundLanes) + queues(ControlQueueIndex) = QueueWrapper(createQueue(controlQueueSize)) // control stream + queues(LargeQueueIndex) = QueueWrapper(createQueue(largeQueueSize)) // large messages stream + (0 until outboundLanes).foreach { i ⇒ + queues(OrdinaryQueueIndex + i) = QueueWrapper(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: OutboundControlIngress = _ @volatile private[this] var materializing = new CountDownLatch(1) - @volatile private[this] var changeOutboundCompression: Option[ChangeOutboundCompression] = None + @volatile private[this] var changeOutboundCompression: Option[Vector[ChangeOutboundCompression]] = None - def changeActorRefCompression(table: CompressionTable[ActorRef]): Future[Done] = + def changeActorRefCompression(table: CompressionTable[ActorRef]): Future[Done] = { + import transport.system.dispatcher changeOutboundCompression match { - case Some(c) ⇒ c.changeActorRefCompression(table) - case None ⇒ Future.failed(new ChangeOutboundCompressionFailed) + case Some(c) ⇒ + if (c.size == 1) c.head.changeActorRefCompression(table) + else Future.sequence(c.map(_.changeActorRefCompression(table))).map(_ ⇒ Done) + case None ⇒ Future.failed(new ChangeOutboundCompressionFailed) } + } - def changeClassManifestCompression(table: CompressionTable[String]): Future[Done] = + def changeClassManifestCompression(table: CompressionTable[String]): Future[Done] = { + import transport.system.dispatcher changeOutboundCompression match { - case Some(c) ⇒ c.changeClassManifestCompression(table) - case None ⇒ Future.failed(new ChangeOutboundCompressionFailed) + case Some(c) ⇒ + if (c.size == 1) c.head.changeClassManifestCompression(table) + else Future.sequence(c.map(_.changeClassManifestCompression(table))).map(_ ⇒ Done) + case None ⇒ Future.failed(new ChangeOutboundCompressionFailed) } + } - def clearCompression(): Future[Done] = + def clearCompression(): Future[Done] = { + import transport.system.dispatcher changeOutboundCompression match { - case Some(c) ⇒ c.clearCompression() - case None ⇒ Future.failed(new ChangeOutboundCompressionFailed) + case Some(c) ⇒ + if (c.size == 1) c.head.clearCompression() + else Future.sequence(c.map(_.clearCompression())).map(_ ⇒ Done) + case None ⇒ Future.failed(new ChangeOutboundCompressionFailed) } + } private val _testStages: CopyOnWriteArrayList[TestManagementApi] = new CopyOnWriteArrayList @@ -201,6 +228,9 @@ private[remote] class Association( def createOutboundEnvelope(): OutboundEnvelope = outboundEnvelopePool.acquire().init(recipient, message.asInstanceOf[AnyRef], sender) + // volatile read to see latest queue array + val unused = queuesVisibility + // allow ActorSelectionMessage to pass through quarantine, to be able to establish interaction with new system // FIXME where is that ActorSelectionMessage check in old remoting? if (message.isInstanceOf[ActorSelectionMessage] || !associationState.isQuarantined() || message == ClearSystemMessageDelivery) { @@ -219,9 +249,11 @@ private[remote] class Association( val outboundEnvelope1 = createOutboundEnvelope() if (!controlQueue.offer(outboundEnvelope1)) transport.system.deadLetters ! outboundEnvelope1 - val outboundEnvelope2 = createOutboundEnvelope() - if (!queue.offer(outboundEnvelope2)) - transport.system.deadLetters ! outboundEnvelope2 + (0 until outboundLanes).foreach { i ⇒ + val outboundEnvelope2 = createOutboundEnvelope() + if (!queues(OrdinaryQueueIndex + i).offer(outboundEnvelope2)) + transport.system.deadLetters ! outboundEnvelope2 + } case _ ⇒ val outboundEnvelope = createOutboundEnvelope() val queue = selectQueue(recipient) @@ -233,30 +265,35 @@ private[remote] class Association( log.debug("Dropping message to quarantined system {}", remoteAddress) } - @tailrec private def selectQueue(recipient: OptionVal[RemoteActorRef]): ProducerApi[OutboundEnvelope] = { recipient match { case OptionVal.Some(r) ⇒ - r.cachedMessageDestinationFlag match { - case RegularDestination ⇒ queue - case PriorityDestination ⇒ controlQueue - case LargeDestination ⇒ largeQueue - case null ⇒ + val queueIndex = 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 - if (priorityMessageDestinations.find(elements).isDefined) { - log.debug("Using priority message stream for {}", r.path) - r.cachedMessageDestinationFlag = PriorityDestination - } else if (transport.largeMessageChannelEnabled && largeMessageDestinations.find(elements).isDefined) { - log.debug("Using large message stream for {}", r.path) - r.cachedMessageDestinationFlag = LargeDestination - } else { - r.cachedMessageDestinationFlag = RegularDestination - } - selectQueue(recipient) + 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 ⇒ queue + queues(queueIndex) + + case OptionVal.None ⇒ + queues(OrdinaryQueueIndex) } } @@ -333,29 +370,22 @@ private[remote] class Association( // using CountDownLatch to make sure that materialization is completed before accessing outboundControlIngress materializing = new CountDownLatch(1) - val wrapper = getOrCreateQueueWrapper(controlQueue, queueSize) - controlQueue = wrapper // use new underlying queue immediately for restarts + 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 (queueValue, (control, completed)) = - if (transport.settings.Advanced.TestMode) { - val ((queueValue, mgmt), (control, completed)) = - Source.fromGraph(new SendQueue[OutboundEnvelope]) - .via(transport.outboundControlPart1(this)) - .viaMat(transport.outboundTestFlow(this))(Keep.both) - .toMat(transport.outboundControlPart2(this))(Keep.both) - .run()(materializer) - _testStages.add(mgmt) - (queueValue, (control, completed)) - } else { - Source.fromGraph(new SendQueue[OutboundEnvelope]) - .via(transport.outboundControlPart1(this)) - .toMat(transport.outboundControlPart2(this))(Keep.both) - .run()(materializer) - } + val (queueValue, (testMgmt, control, completed)) = + Source.fromGraph(new SendQueue[OutboundEnvelope]) + .toMat(transport.outboundControl(this))(Keep.both) + .run()(materializer) + + if (transport.settings.Advanced.TestMode) + _testStages.add(testMgmt) queueValue.inject(wrapper.queue) // replace with the materialized value, still same underlying queue - controlQueue = queueValue + queues(ControlQueueIndex) = queueValue + queuesVisibility = true // volatile write for visibility of the queues array _outboundControlIngress = control materializing.countDown() attachStreamRestart("Outbound control stream", completed, cause ⇒ { @@ -367,61 +397,103 @@ private[remote] class Association( }) } - private def getOrCreateQueueWrapper(q: SendQueue.ProducerApi[OutboundEnvelope], capacity: Int): QueueWrapper = - q match { + 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 QueueWrapper(createQueue(capacity)) } + } private def runOutboundOrdinaryMessagesStream(): Unit = { - val wrapper = getOrCreateQueueWrapper(queue, queueSize) - queue = wrapper // use new underlying queue immediately for restarts + if (outboundLanes == 1) { + 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 (queueValue, (changeCompression, completed)) = - if (transport.settings.Advanced.TestMode) { - val ((queueValue, mgmt), completed) = Source.fromGraph(new SendQueue[OutboundEnvelope]) + val ((queueValue, testMgmt), (changeCompression, completed)) = + Source.fromGraph(new SendQueue[OutboundEnvelope]) .viaMat(transport.outboundTestFlow(this))(Keep.both) .toMat(transport.outbound(this))(Keep.both) .run()(materializer) - _testStages.add(mgmt) - (queueValue, completed) - } else { - Source.fromGraph(new SendQueue[OutboundEnvelope]) - .toMat(transport.outbound(this))(Keep.both) - .run()(materializer) + + if (transport.settings.Advanced.TestMode) + _testStages.add(testMgmt) + + 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 = Some(Vector(changeCompression)) + + attachStreamRestart("Outbound message stream", completed, _ ⇒ runOutboundOrdinaryMessagesStream()) + + } else { + 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 lane = Source.fromGraph(new SendQueue[OutboundEnvelope]) + .viaMat(transport.outboundTestFlow(this))(Keep.both) + .viaMat(transport.outboundLane(this))(Keep.both) + .watchTermination()(Keep.both) + .mapMaterializedValue { + case (((q, m), c), w) ⇒ ((q, m), (c, w)) + } + + val (mergeHub, aeronSinkCompleted) = MergeHub.source[EnvelopeBuffer].toMat(transport.aeronSink(this))(Keep.both).run()(materializer) + + val values: Vector[((SendQueue.QueueValue[OutboundEnvelope], TestManagementApi), (Encoder.ChangeOutboundCompression, Future[Done]))] = + (0 until outboundLanes).map { _ ⇒ + lane.to(mergeHub).run()(materializer) + }(collection.breakOut) + + val (a, b) = values.unzip + val (queueValues, testMgmtValues) = a.unzip + val (changeCompressionValues, laneCompletedValues) = b.unzip + + if (transport.settings.Advanced.TestMode) + testMgmtValues.foreach(_testStages.add) + + import transport.system.dispatcher + val completed = Future.sequence(laneCompletedValues).flatMap(_ ⇒ aeronSinkCompleted) + + 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 - queueValue.inject(wrapper.queue) - // replace with the materialized value, still same underlying queue - queue = queueValue - changeOutboundCompression = Some(changeCompression) + changeOutboundCompression = Some(changeCompressionValues) - attachStreamRestart("Outbound message stream", completed, _ ⇒ runOutboundOrdinaryMessagesStream()) + attachStreamRestart("Outbound message stream", completed, _ ⇒ runOutboundOrdinaryMessagesStream()) + } } private def runOutboundLargeMessagesStream(): Unit = { - val wrapper = getOrCreateQueueWrapper(largeQueue, largeQueueSize) - largeQueue = wrapper // use new underlying queue immediately for restarts + 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 (queueValue, completed) = - if (transport.settings.Advanced.TestMode) { - val ((queueValue, mgmt), completed) = Source.fromGraph(new SendQueue[OutboundEnvelope]) - .viaMat(transport.outboundTestFlow(this))(Keep.both) - .toMat(transport.outboundLarge(this))(Keep.both) - .run()(materializer) - _testStages.add(mgmt) - (queueValue, completed) - } else { - Source.fromGraph(new SendQueue[OutboundEnvelope]) - .toMat(transport.outboundLarge(this))(Keep.both) - .run()(materializer) - } + val ((queueValue, testMgmt), completed) = Source.fromGraph(new SendQueue[OutboundEnvelope]) + .viaMat(transport.outboundTestFlow(this))(Keep.both) + .toMat(transport.outboundLarge(this))(Keep.both) + .run()(materializer) + + if (transport.settings.Advanced.TestMode) + _testStages.add(testMgmt) queueValue.inject(wrapper.queue) // replace with the materialized value, still same underlying queue - largeQueue = queueValue + queues(LargeQueueIndex) = queueValue + queuesVisibility = true // volatile write for visibility of the queues array attachStreamRestart("Outbound large message stream", completed, _ ⇒ runOutboundLargeMessagesStream()) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index cd6380fe9f..10916fbf93 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -21,6 +21,7 @@ import akka.remote.artery.compress.CompressionTable import akka.Done import akka.stream.stage.GraphStageWithMaterializedValue import scala.concurrent.Promise +import java.util.concurrent.atomic.AtomicInteger /** * INTERNAL API @@ -34,6 +35,7 @@ private[remote] object Encoder { private[remote] class ChangeOutboundCompressionFailed extends RuntimeException( "Change of outbound compression table failed (will be retried), because materialization did not complete yet") + } /** @@ -212,7 +214,6 @@ private[remote] class Decoder( import Decoder.RetryResolveRemoteDeployedRecipient private val localAddress = inboundContext.localAddress.address private val headerBuilder = HeaderBuilder.in(compression) - private val serialization = SerializationExtension(system) private val retryResolveRemoteDeployedRecipientInterval = 50.millis private val retryResolveRemoteDeployedRecipientAttempts = 20 @@ -284,35 +285,24 @@ private[remote] class Decoder( // --- end of hit refs and manifests for heavy-hitter counting } - try { - val deserializedMessage = MessageSerializer.deserializeForArtery( - system, originUid, serialization, headerBuilder.serializer, classManifest, envelope) + val decoded = inEnvelopePool.acquire().init( + recipient, + localAddress, // FIXME: Is this needed anymore? What should we do here? + sender, + originUid, + headerBuilder.serializer, + classManifest, + envelope, + association) - val decoded = inEnvelopePool.acquire().init( - recipient, - localAddress, // FIXME: Is this needed anymore? What should we do here? - deserializedMessage, - sender, // FIXME: No need for an option, decode simply to deadLetters instead - originUid, - 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 - scheduleOnce(RetryResolveRemoteDeployedRecipient( - retryResolveRemoteDeployedRecipientAttempts, - headerBuilder.recipientActorRefPath.get, decoded), retryResolveRemoteDeployedRecipientInterval) // FIXME IS THIS SAFE? - } else - push(out, decoded) - } catch { - case NonFatal(e) ⇒ - log.warning( - "Failed to deserialize message with serializer id [{}] and manifest [{}]. {}", - headerBuilder.serializer, classManifest, e.getMessage) - pull(in) - } finally { - bufferPool.release(envelope) - } + 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 + scheduleOnce(RetryResolveRemoteDeployedRecipient( + retryResolveRemoteDeployedRecipientAttempts, + headerBuilder.recipientActorRefPath.get, decoded), retryResolveRemoteDeployedRecipientInterval) // FIXME IS THIS SAFE? + } else + push(out, decoded) } private def resolveRecipient(path: String): OptionVal[InternalActorRef] = { @@ -369,3 +359,48 @@ private[remote] class Decoder( } } +/** + * 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 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) + + push(out, envelope.withMessage(deserializedMessage)) + } 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) + + setHandlers(in, out, this) + } +} + diff --git a/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala b/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala index 2c20cb287d..bd15681ef4 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala @@ -11,7 +11,10 @@ import akka.actor.ActorRef /** * INTERNAL API */ -private[akka] object InboundEnvelope { +private[remote] object InboundEnvelope { + /** + * Only used in tests + */ def apply( recipient: OptionVal[InternalActorRef], recipientAddress: Address, @@ -20,7 +23,8 @@ private[akka] object InboundEnvelope { originUid: Long, association: OptionVal[OutboundContext]): InboundEnvelope = { val env = new ReusableInboundEnvelope - env.init(recipient, recipientAddress, message, sender, originUid, association) + env.init(recipient, recipientAddress, sender, originUid, -1, "", null, association) + .withMessage(message) } } @@ -28,23 +32,29 @@ private[akka] object InboundEnvelope { /** * INTERNAL API */ -private[akka] trait InboundEnvelope { +private[remote] trait InboundEnvelope { def recipient: OptionVal[InternalActorRef] def recipientAddress: Address - def message: AnyRef def sender: OptionVal[ActorRef] def originUid: Long def association: OptionVal[OutboundContext] + def serializer: Int + def classManifest: String + def message: AnyRef + def envelopeBuffer: EnvelopeBuffer + def withMessage(message: AnyRef): InboundEnvelope + def releaseEnvelopeBuffer(): InboundEnvelope + def withRecipient(ref: InternalActorRef): InboundEnvelope } /** * INTERNAL API */ -private[akka] object ReusableInboundEnvelope { +private[remote] object ReusableInboundEnvelope { def createObjectPool(capacity: Int) = new ObjectPool[ReusableInboundEnvelope]( capacity, create = () ⇒ new ReusableInboundEnvelope, clear = inEnvelope ⇒ inEnvelope.asInstanceOf[ReusableInboundEnvelope].clear()) @@ -56,23 +66,34 @@ private[akka] object ReusableInboundEnvelope { private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { private var _recipient: OptionVal[InternalActorRef] = OptionVal.None private var _recipientAddress: Address = null - private var _message: AnyRef = null 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 _message: AnyRef = null + private var _envelopeBuffer: EnvelopeBuffer = null override def recipient: OptionVal[InternalActorRef] = _recipient override def recipientAddress: Address = _recipientAddress - override def message: AnyRef = _message 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 withMessage(message: AnyRef): InboundEnvelope = { _message = message this } + def releaseEnvelopeBuffer(): InboundEnvelope = { + _envelopeBuffer = null + this + } + def withRecipient(ref: InternalActorRef): InboundEnvelope = { _recipient = OptionVal(ref) this @@ -90,15 +111,19 @@ private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { def init( recipient: OptionVal[InternalActorRef], recipientAddress: Address, - message: AnyRef, sender: OptionVal[ActorRef], originUid: Long, + serializer: Int, + classManifest: String, + envelopeBuffer: EnvelopeBuffer, association: OptionVal[OutboundContext]): InboundEnvelope = { _recipient = recipient _recipientAddress = recipientAddress - _message = message _sender = sender _originUid = originUid + _serializer = serializer + _classManifest = classManifest + _envelopeBuffer = envelopeBuffer _association = association this } diff --git a/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala b/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala index b354f32bda..45e975e0e3 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala @@ -24,6 +24,7 @@ import akka.stream.stage.InHandler import akka.stream.stage.OutHandler import akka.stream.stage.TimerGraphStageLogic import akka.util.OptionVal +import akka.stream.stage.GraphStageLogic /** * INTERNAL API @@ -49,6 +50,11 @@ private[remote] class TestManagementApiImpl(stopped: Future[Done], callback: Asy } } +private[remote] class DisabledTestManagementApi extends TestManagementApi { + override def send(command: Any)(implicit ec: ExecutionContext): Future[Done] = + Future.failed(new RuntimeException("TestStage is disabled, enable with MultiNodeConfig.testTransport")) +} + /** * INTERNAL API */ @@ -57,61 +63,70 @@ private[remote] final case class TestManagementMessage(command: Any, done: Promi /** * INTERNAL API */ -private[remote] class OutboundTestStage(outboundContext: OutboundContext) +private[remote] class OutboundTestStage(outboundContext: OutboundContext, enabled: Boolean) extends GraphStageWithMaterializedValue[FlowShape[OutboundEnvelope, OutboundEnvelope], TestManagementApi] { 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 createLogicAndMaterializedValue(inheritedAttributes: Attributes) = { - val stoppedPromise = Promise[Done]() + if (enabled) { + val stoppedPromise = Promise[Done]() - // FIXME see issue #20503 related to CallbackWrapper, we might implement this in a better way - val logic = new TimerGraphStageLogic(shape) with CallbackWrapper[TestManagementMessage] with InHandler with OutHandler with StageLogging { + // FIXME see issue #20503 related to CallbackWrapper, we might implement this in a better way + val logic = new TimerGraphStageLogic(shape) with CallbackWrapper[TestManagementMessage] with InHandler with OutHandler with StageLogging { - private var blackhole = Set.empty[Address] + private var blackhole = Set.empty[Address] - private val callback = getAsyncCallback[TestManagementMessage] { - case TestManagementMessage(command, done) ⇒ - command match { - case SetThrottle(address, Direction.Send | Direction.Both, Blackhole) ⇒ - log.info("blackhole outbound messages to {}", address) - blackhole += address - case SetThrottle(address, Direction.Send | Direction.Both, Unthrottled) ⇒ - log.info("accept outbound messages to {}", address) - blackhole -= address - case _ ⇒ // not interested - } - done.success(Done) + private val callback = getAsyncCallback[TestManagementMessage] { + case TestManagementMessage(command, done) ⇒ + command match { + case SetThrottle(address, Direction.Send | Direction.Both, Blackhole) ⇒ + log.info("blackhole outbound messages to {}", address) + blackhole += address + case SetThrottle(address, Direction.Send | Direction.Both, Unthrottled) ⇒ + log.info("accept outbound messages to {}", address) + blackhole -= address + case _ ⇒ // not interested + } + done.success(Done) + } + + override def preStart(): Unit = { + initCallback(callback.invoke) + } + + override def postStop(): Unit = stoppedPromise.success(Done) + + // InHandler + override def onPush(): Unit = { + val env = grab(in) + if (blackhole(outboundContext.remoteAddress)) { + log.debug( + "dropping outbound message [{}] to [{}] because of blackhole", + env.message.getClass.getName, outboundContext.remoteAddress) + pull(in) // drop message + } else + push(out, env) + } + + // OutHandler + override def onPull(): Unit = pull(in) + + setHandlers(in, out, this) } - override def preStart(): Unit = { - initCallback(callback.invoke) + val managementApi: TestManagementApi = new TestManagementApiImpl(stoppedPromise.future, logic) + + (logic, managementApi) + } else { + val logic = 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) } - - override def postStop(): Unit = stoppedPromise.success(Done) - - // InHandler - override def onPush(): Unit = { - val env = grab(in) - if (blackhole(outboundContext.remoteAddress)) { - log.debug( - "dropping outbound message [{}] to [{}] because of blackhole", - env.message.getClass.getName, outboundContext.remoteAddress) - pull(in) // drop message - } else - push(out, env) - } - - // OutHandler - override def onPull(): Unit = pull(in) - - setHandlers(in, out, this) + (logic, new DisabledTestManagementApi) } - - val managementApi: TestManagementApi = new TestManagementApiImpl(stoppedPromise.future, logic) - - (logic, managementApi) } } @@ -119,67 +134,76 @@ private[remote] class OutboundTestStage(outboundContext: OutboundContext) /** * INTERNAL API */ -private[remote] class InboundTestStage(inboundContext: InboundContext) +private[remote] class InboundTestStage(inboundContext: InboundContext, enabled: Boolean) extends GraphStageWithMaterializedValue[FlowShape[InboundEnvelope, InboundEnvelope], TestManagementApi] { 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 createLogicAndMaterializedValue(inheritedAttributes: Attributes) = { - val stoppedPromise = Promise[Done]() + if (enabled) { + val stoppedPromise = Promise[Done]() - // FIXME see issue #20503 related to CallbackWrapper, we might implement this in a better way - val logic = new TimerGraphStageLogic(shape) with CallbackWrapper[TestManagementMessage] with InHandler with OutHandler with StageLogging { + // FIXME see issue #20503 related to CallbackWrapper, we might implement this in a better way + val logic = new TimerGraphStageLogic(shape) with CallbackWrapper[TestManagementMessage] with InHandler with OutHandler with StageLogging { - private var blackhole = Set.empty[Address] + private var blackhole = Set.empty[Address] - private val callback = getAsyncCallback[TestManagementMessage] { - case TestManagementMessage(command, done) ⇒ - command match { - case SetThrottle(address, Direction.Receive | Direction.Both, Blackhole) ⇒ - log.info("blackhole inbound messages from {}", address) - blackhole += address - case SetThrottle(address, Direction.Receive | Direction.Both, Unthrottled) ⇒ - log.info("accept inbound messages from {}", address) - blackhole -= address - case _ ⇒ // not interested - } - done.success(Done) - } - - override def preStart(): Unit = { - initCallback(callback.invoke) - } - - override def postStop(): Unit = stoppedPromise.success(Done) - - // 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 (blackhole(association.remoteAddress)) { - log.debug( - "dropping inbound message [{}] from [{}] with UID [{}] because of blackhole", - env.message.getClass.getName, association.remoteAddress, env.originUid) - pull(in) // drop message - } else - push(out, env) + private val callback = getAsyncCallback[TestManagementMessage] { + case TestManagementMessage(command, done) ⇒ + command match { + case SetThrottle(address, Direction.Receive | Direction.Both, Blackhole) ⇒ + log.info("blackhole inbound messages from {}", address) + blackhole += address + case SetThrottle(address, Direction.Receive | Direction.Both, Unthrottled) ⇒ + log.info("accept inbound messages from {}", address) + blackhole -= address + case _ ⇒ // not interested + } + done.success(Done) } + + override def preStart(): Unit = { + initCallback(callback.invoke) + } + + override def postStop(): Unit = stoppedPromise.success(Done) + + // 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 (blackhole(association.remoteAddress)) { + log.debug( + "dropping inbound message [{}] from [{}] with UID [{}] because of blackhole", + env.message.getClass.getName, association.remoteAddress, env.originUid) + pull(in) // drop message + } else + push(out, env) + } + } + + // OutHandler + override def onPull(): Unit = pull(in) + + setHandlers(in, out, this) } - // OutHandler - override def onPull(): Unit = pull(in) + val managementApi: TestManagementApi = new TestManagementApiImpl(stoppedPromise.future, logic) - setHandlers(in, out, this) + (logic, managementApi) + } else { + val logic = 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) + } + (logic, new DisabledTestManagementApi) } - - val managementApi: TestManagementApi = new TestManagementApiImpl(stoppedPromise.future, logic) - - (logic, managementApi) } } diff --git a/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala index 368537ee55..ead929855d 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala @@ -39,7 +39,7 @@ class AeronSinkSpec extends AkkaSpec with ImplicitSender { r } - val pool = new EnvelopeBufferPool(ArteryTransport.MaximumFrameSize, ArteryTransport.MaximumPooledBuffers) + val pool = new EnvelopeBufferPool(1034 * 1024, 128) val matSettings = ActorMaterializerSettings(system).withFuzzing(true) implicit val mat = ActorMaterializer(matSettings)(system) diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala index ce8126368a..a246e865b4 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala @@ -10,6 +10,7 @@ import akka.testkit.{ AkkaSpec, ImplicitSender } import akka.testkit.SocketUtil import akka.testkit.TestActors import com.typesafe.config.ConfigFactory +import akka.testkit.TestProbe object HandshakeFailureSpec { @@ -22,7 +23,7 @@ object HandshakeFailureSpec { remote.artery.enabled = on remote.artery.hostname = localhost remote.artery.port = 0 - remote.handshake-timeout = 2s + remote.artery.advanced.handshake-timeout = 2s } """) diff --git a/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala index bb584ea8f2..6312a8bc28 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala @@ -4,7 +4,7 @@ package akka.remote.artery import akka.actor.{ Actor, ActorRef, ActorSelection, Props, RootActorPath } -import akka.remote.{ LargeDestination, RARP, RegularDestination, RemoteActorRef } +import akka.remote.{ RARP, RemoteActorRef } import akka.testkit.TestProbe import akka.util.ByteString @@ -51,7 +51,7 @@ class LargeMessagesStreamSpec extends ArteryMultiNodeSpec( senderProbeA.expectMsg(Pong(0)) // flag should be cached now - regularRemote.asInstanceOf[RemoteActorRef].cachedMessageDestinationFlag should ===(RegularDestination) + regularRemote.asInstanceOf[RemoteActorRef].cachedSendQueueIndex should be >= (Association.OrdinaryQueueIndex) } @@ -75,7 +75,7 @@ class LargeMessagesStreamSpec extends ArteryMultiNodeSpec( senderProbeA.expectMsg(Pong(0)) // flag should be cached now - largeRemote.asInstanceOf[RemoteActorRef].cachedMessageDestinationFlag should ===(LargeDestination) + largeRemote.asInstanceOf[RemoteActorRef].cachedSendQueueIndex should ===(Association.LargeQueueIndex) } @@ -112,8 +112,8 @@ class LargeMessagesStreamSpec extends ArteryMultiNodeSpec( remoteProbe.expectMsg(10.seconds, Pong(largeBytes)) // cached flags should be set now - largeRemote.asInstanceOf[RemoteActorRef].cachedMessageDestinationFlag should ===(LargeDestination) - regularRemote.asInstanceOf[RemoteActorRef].cachedMessageDestinationFlag should ===(RegularDestination) + largeRemote.asInstanceOf[RemoteActorRef].cachedSendQueueIndex should ===(Association.LargeQueueIndex) + regularRemote.asInstanceOf[RemoteActorRef].cachedSendQueueIndex should be >= (Association.OrdinaryQueueIndex) } } diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala index 77fdca9792..654a31221b 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala @@ -21,7 +21,6 @@ object RemoteMessageSerializationSpec { case s if sender().path == another.path ⇒ one ! s } } - val maxPayloadBytes = ArteryTransport.MaximumFrameSize } class RemoteMessageSerializationSpec extends ArteryMultiNodeSpec(""" @@ -31,6 +30,8 @@ class RemoteMessageSerializationSpec extends ArteryMultiNodeSpec(""" import RemoteMessageSerializationSpec._ + val maxPayloadBytes = RARP(system).provider.remoteSettings.Artery.Advanced.MaximumFrameSize + val remoteSystem = newRemoteSystem() val remotePort = port(remoteSystem) diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala index 8149919ee6..da8b7bea11 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala @@ -12,6 +12,8 @@ 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 { @@ -26,7 +28,15 @@ object RemoteSendConsistencySpec { } -class RemoteSendConsistencySpec extends AkkaSpec(RemoteSendConsistencySpec.config) with ImplicitSender { +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 @@ -78,18 +88,24 @@ class RemoteSendConsistencySpec extends AkkaSpec(RemoteSendConsistencySpec.confi } "be able to send messages concurrently preserving order" in { - val actorOnSystemB = systemB.actorOf(Props(new Actor { - def receive = { - case i: Int ⇒ sender() ! i - } - }), "echo2") + systemB.actorOf(TestActors.echoActorProps, "echoA") + systemB.actorOf(TestActors.echoActorProps, "echoB") + systemB.actorOf(TestActors.echoActorProps, "echoC") - val remoteRef = { - system.actorSelection(rootB / "user" / "echo2") ! Identify(None) + 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 } - val senderProps = Props(new Actor { + def senderProps(remoteRef: ActorRef) = Props(new Actor { var counter = 1000 remoteRef ! counter @@ -106,10 +122,10 @@ class RemoteSendConsistencySpec extends AkkaSpec(RemoteSendConsistencySpec.confi } }).withDeploy(Deploy.local) - system.actorOf(senderProps) - system.actorOf(senderProps) - system.actorOf(senderProps) - system.actorOf(senderProps) + system.actorOf(senderProps(remoteRefA)) + system.actorOf(senderProps(remoteRefB)) + system.actorOf(senderProps(remoteRefC)) + system.actorOf(senderProps(remoteRefA)) within(10.seconds) { expectMsg("success") diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala index 04233b14ef..5ae7d99c58 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -20,7 +20,7 @@ import akka.util.OptionVal import akka.actor.InternalActorRef import akka.dispatch.ExecutionContexts -private[akka] class TestInboundContext( +private[remote] class TestInboundContext( override val localAddress: UniqueAddress, val controlSubject: TestControlMessageSubject = new TestControlMessageSubject, val controlProbe: Option[ActorRef] = None, @@ -61,7 +61,7 @@ private[akka] class TestInboundContext( new TestOutboundContext(localAddress, remoteAddress, controlSubject, controlProbe) } -private[akka] class TestOutboundContext( +private[remote] class TestOutboundContext( override val localAddress: UniqueAddress, override val remoteAddress: Address, override val controlSubject: TestControlMessageSubject, @@ -96,7 +96,7 @@ private[akka] class TestOutboundContext( } -private[akka] class TestControlMessageSubject extends ControlMessageSubject { +private[remote] class TestControlMessageSubject extends ControlMessageSubject { private val observers = new CopyOnWriteArrayList[ControlMessageObserver] @@ -119,7 +119,7 @@ private[akka] class TestControlMessageSubject extends ControlMessageSubject { } -private[akka] class ManualReplyInboundContext( +private[remote] class ManualReplyInboundContext( replyProbe: ActorRef, localAddress: UniqueAddress, controlSubject: TestControlMessageSubject) extends TestInboundContext(localAddress, controlSubject) { 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 index 1a0d919873..e3d479c7fa 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala @@ -28,7 +28,7 @@ object CompressionIntegrationSpec { remote.artery.enabled = on remote.artery.hostname = localhost remote.artery.port = 0 - remote.handshake-timeout = 10s + remote.artery.advanced.handshake-timeout = 10s remote.artery.advanced.compression { actor-refs.advertisement-interval = 3 seconds 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 index 74ac512e67..f2eff03ea0 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala @@ -30,7 +30,7 @@ object HandshakeShouldDropCompressionTableSpec { remote.artery.enabled = on remote.artery.hostname = localhost remote.artery.port = 0 - remote.handshake-timeout = 10s + remote.artery.advanced.handshake-timeout = 10s remote.artery.advanced.compression { actor-refs { From 432086b3f47b61e15884c8a494bfd421b0d5eccd Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 5 Sep 2016 12:09:59 +0200 Subject: [PATCH 102/186] improve deadLetters and logging when send queue overflow (#21355) --- .../akka/remote/RemoteActorRefProvider.scala | 12 ++++++---- .../akka/remote/artery/Association.scala | 24 +++++++++++++++---- 2 files changed, 26 insertions(+), 10 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 914fc409b2..46b195cd4e 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -89,18 +89,20 @@ 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.!(env.message)(env.sender.orNull) + super.!(DeadLetter(env.message, env.sender.getOrElse(_provider.deadLetters), + env.recipient.getOrElse(_provider.deadLetters))) case DeadLetter(env: OutboundEnvelope, _, _) ⇒ - super.!(env.message)(env.sender.orNull) + super.!(DeadLetter(env.message, env.sender.getOrElse(_provider.deadLetters), + env.recipient.getOrElse(_provider.deadLetters))) case _ ⇒ super.!(message)(sender) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 552ce77b65..8094e22c02 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -43,6 +43,7 @@ import akka.util.OptionVal import org.agrona.concurrent.ManyToOneConcurrentArrayQueue import akka.remote.artery.compress.CompressionProtocol._ import akka.stream.scaladsl.MergeHub +import akka.actor.DeadLetter /** * INTERNAL API @@ -144,6 +145,8 @@ private[remote] class Association( _testStages.asScala.toList } + private def deadletters = transport.system.deadLetters + def outboundControlIngress: OutboundControlIngress = { if (_outboundControlIngress ne null) _outboundControlIngress @@ -231,6 +234,14 @@ private[remote] class Association( // volatile read to see latest queue array val unused = queuesVisibility + def dropped(qSize: Int, env: OutboundEnvelope): Unit = { + log.debug( + "Dropping message [{}] from [{}] to [{}] due to overflow of send queue, size [{}]", + message.getClass, sender.getOrElse(deadletters), recipient.getOrElse(recipient), qSize) + // FIXME AFR + deadletters ! env + } + // allow ActorSelectionMessage to pass through quarantine, to be able to establish interaction with new system // FIXME where is that ActorSelectionMessage check in old remoting? if (message.isInstanceOf[ActorSelectionMessage] || !associationState.isQuarantined() || message == ClearSystemMessageDelivery) { @@ -239,7 +250,7 @@ private[remote] class Association( val outboundEnvelope = createOutboundEnvelope() if (!controlQueue.offer(createOutboundEnvelope())) { quarantine(reason = s"Due to overflow of control queue, size [$controlQueueSize]") - transport.system.deadLetters ! outboundEnvelope + dropped(controlQueueSize, outboundEnvelope) } case _: DaemonMsgCreate ⇒ // DaemonMsgCreate is not a SystemMessage, but must be sent over the control stream because @@ -248,21 +259,24 @@ private[remote] class Association( // destination) before the first ordinary message arrives. val outboundEnvelope1 = createOutboundEnvelope() if (!controlQueue.offer(outboundEnvelope1)) - transport.system.deadLetters ! outboundEnvelope1 + dropped(controlQueueSize, outboundEnvelope1) (0 until outboundLanes).foreach { i ⇒ val outboundEnvelope2 = createOutboundEnvelope() if (!queues(OrdinaryQueueIndex + i).offer(outboundEnvelope2)) - transport.system.deadLetters ! outboundEnvelope2 + dropped(queueSize, outboundEnvelope2) } case _ ⇒ val outboundEnvelope = createOutboundEnvelope() val queue = selectQueue(recipient) val offerOk = queue.offer(outboundEnvelope) if (!offerOk) - transport.system.deadLetters ! outboundEnvelope + dropped(queueSize, outboundEnvelope) + } } else if (log.isDebugEnabled) - log.debug("Dropping message to quarantined system {}", remoteAddress) + log.debug( + "Dropping message [{}] from [{}] to [{}] due to quarantined system [{}]", + message.getClass, sender.getOrElse(deadletters), recipient.getOrElse(recipient), remoteAddress) } private def selectQueue(recipient: OptionVal[RemoteActorRef]): ProducerApi[OutboundEnvelope] = { From 9d8981067465b9b573d51ca685b49028f8ab181a Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 2 Sep 2016 08:52:20 +0200 Subject: [PATCH 103/186] make restart materialization of outbound streams lazy, #21347 * Materialize on first message instead, otherwise handshake attempts to non-existing nodes will continue forever * also fix HandshakeFailureSpec --- .../akka/remote/artery/Association.scala | 65 ++++++++++++++----- .../remote/artery/HandshakeFailureSpec.scala | 5 +- 2 files changed, 50 insertions(+), 20 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 8094e22c02..d8557b81af 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -44,15 +44,30 @@ import org.agrona.concurrent.ManyToOneConcurrentArrayQueue import akka.remote.artery.compress.CompressionProtocol._ import akka.stream.scaladsl.MergeHub import akka.actor.DeadLetter +import java.util.concurrent.atomic.AtomicBoolean /** * INTERNAL API */ private[remote] object Association { - final case class QueueWrapper(queue: Queue[OutboundEnvelope]) extends SendQueue.ProducerApi[OutboundEnvelope] { + 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) } + final case class LazyQueueWrapper(queue: Queue[OutboundEnvelope], materialize: () ⇒ Unit) extends QueueWrapper { + private val onlyOnce = new AtomicBoolean + + override def offer(message: OutboundEnvelope): Boolean = { + if (onlyOnce.compareAndSet(false, true)) + materialize() + queue.offer(message) + } + } + final val ControlQueueIndex = 0 final val LargeQueueIndex = 1 final val OrdinaryQueueIndex = 2 @@ -94,10 +109,10 @@ private[remote] class Association( private val largeQueueSize = 256 private[this] val queues: Array[SendQueue.ProducerApi[OutboundEnvelope]] = Array.ofDim(2 + outboundLanes) - queues(ControlQueueIndex) = QueueWrapper(createQueue(controlQueueSize)) // control stream - queues(LargeQueueIndex) = QueueWrapper(createQueue(largeQueueSize)) // large messages stream + queues(ControlQueueIndex) = QueueWrapperImpl(createQueue(controlQueueSize)) // control stream + queues(LargeQueueIndex) = QueueWrapperImpl(createQueue(largeQueueSize)) // large messages stream (0 until outboundLanes).foreach { i ⇒ - queues(OrdinaryQueueIndex + i) = QueueWrapper(createQueue(queueSize)) // ordinary messages stream + queues(OrdinaryQueueIndex + i) = QueueWrapperImpl(createQueue(queueSize)) // ordinary messages stream } @volatile private[this] var queuesVisibility = false @@ -402,13 +417,8 @@ private[remote] class Association( queuesVisibility = true // volatile write for visibility of the queues array _outboundControlIngress = control materializing.countDown() - attachStreamRestart("Outbound control stream", completed, cause ⇒ { - runOutboundControlStream() - cause match { - case _: HandshakeTimeoutException ⇒ // ok, quarantine not possible without UID - case _ ⇒ quarantine("Outbound control stream restarted") - } - }) + attachStreamRestart("Outbound control stream", ControlQueueIndex, controlQueueSize, + completed, () ⇒ runOutboundControlStream()) } private def getOrCreateQueueWrapper(queueIndex: Int, capacity: Int): QueueWrapper = { @@ -417,7 +427,7 @@ private[remote] class Association( case existing: QueueWrapper ⇒ existing case _ ⇒ // use new queue for restarts - QueueWrapper(createQueue(capacity)) + QueueWrapperImpl(createQueue(capacity)) } } @@ -443,7 +453,8 @@ private[remote] class Association( queuesVisibility = true // volatile write for visibility of the queues array changeOutboundCompression = Some(Vector(changeCompression)) - attachStreamRestart("Outbound message stream", completed, _ ⇒ runOutboundOrdinaryMessagesStream()) + attachStreamRestart("Outbound message stream", OrdinaryQueueIndex, queueSize, + completed, () ⇒ runOutboundOrdinaryMessagesStream()) } else { val wrappers = (0 until outboundLanes).map { i ⇒ @@ -487,7 +498,8 @@ private[remote] class Association( changeOutboundCompression = Some(changeCompressionValues) - attachStreamRestart("Outbound message stream", completed, _ ⇒ runOutboundOrdinaryMessagesStream()) + attachStreamRestart("Outbound message stream", OrdinaryQueueIndex, queueSize, + completed, () ⇒ runOutboundOrdinaryMessagesStream()) } } @@ -508,10 +520,19 @@ private[remote] class Association( // replace with the materialized value, still same underlying queue queues(LargeQueueIndex) = queueValue queuesVisibility = true // volatile write for visibility of the queues array - attachStreamRestart("Outbound large message stream", completed, _ ⇒ runOutboundLargeMessagesStream()) + attachStreamRestart("Outbound large message stream", LargeQueueIndex, largeQueueSize, + completed, () ⇒ runOutboundLargeMessagesStream()) } - private def attachStreamRestart(streamName: String, streamCompleted: Future[Done], restart: Throwable ⇒ Unit): Unit = { + private def attachStreamRestart(streamName: String, queueIndex: Int, queueCapacity: Int, + streamCompleted: Future[Done], restart: () ⇒ Unit): Unit = { + + def lazyRestart(): Unit = { + // 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 _ if transport.isShutdown ⇒ // don't restart after shutdown @@ -519,14 +540,22 @@ private[remote] class Association( case cause: GaveUpSendingException ⇒ log.debug("{} to {} failed. Restarting it. {}", streamName, remoteAddress, cause.getMessage) // restart unconditionally, without counting restarts - restart(cause) + lazyRestart() case cause ⇒ if (restartCounter.restart()) { log.error(cause, "{} to {} failed. Restarting it. {}", streamName, remoteAddress, cause.getMessage) - restart(cause) + lazyRestart() } else { log.error(cause, s"{} to {} failed and restarted {} times within {} seconds. Terminating system. ${cause.getMessage}", streamName, remoteAddress, transport.settings.Advanced.OutboundMaxRestarts, transport.settings.Advanced.OutboundRestartTimeout.toSeconds) + if (queueIndex == ControlQueueIndex) { + cause match { + case _: HandshakeTimeoutException ⇒ // ok, quarantine not possible without UID + case _ ⇒ quarantine("Outbound control stream restarted") + } + } + + // FIXME is this the right thing to do for outbound? transport.system.terminate() } } diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala index a246e865b4..bef3092935 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala @@ -49,8 +49,9 @@ class HandshakeFailureSpec extends AkkaSpec(HandshakeFailureSpec.commonConfig) w within(10.seconds) { awaitAssert { - sel ! "hello2" - expectMsg(1.second, "hello2") + val probe = TestProbe() + sel.tell("hello2", probe.ref) + probe.expectMsg(1.second, "hello2") } } From 32e852596fe6465d7384520233c295c4b43c4d4a Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 1 Sep 2016 16:26:11 +0200 Subject: [PATCH 104/186] harden receive of messages using wrong compression table --- .../scala/akka/remote/artery/Codecs.scala | 111 ++++++++++-------- .../artery/compress/InboundCompressions.scala | 8 +- 2 files changed, 65 insertions(+), 54 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 10916fbf93..04081b6ebe 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -248,61 +248,70 @@ private[remote] class Decoder( OptionVal.None } - val sender: OptionVal[InternalActorRef] = headerBuilder.senderActorRef(originUid) match { - case OptionVal.Some(ref) ⇒ - OptionVal(ref.asInstanceOf[InternalActorRef]) - case OptionVal.None if headerBuilder.senderActorRefPath.isDefined ⇒ - OptionVal(resolveActorRefWithLocalAddress(headerBuilder.senderActorRefPath.get)) - case _ ⇒ - OptionVal.None - } - - val classManifest = headerBuilder.manifest(originUid) - - 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) + if (recipient.isEmpty && headerBuilder.recipientActorRefPath.isEmpty && !headerBuilder.isNoRecipient) { + log.debug("Dropping message for unknown recipient. 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 sender: OptionVal[InternalActorRef] = headerBuilder.senderActorRef(originUid) match { + case OptionVal.Some(ref) ⇒ + OptionVal(ref.asInstanceOf[InternalActorRef]) + case OptionVal.None if headerBuilder.senderActorRefPath.isDefined ⇒ + OptionVal(resolveActorRefWithLocalAddress(headerBuilder.senderActorRefPath.get)) 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?") + OptionVal.None } - // --- end of hit refs and manifests for heavy-hitter counting + + val classManifest = headerBuilder.manifest(originUid) + + 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, + localAddress, // FIXME: Is this needed anymore? What should we do here? + sender, + originUid, + headerBuilder.serializer, + classManifest, + 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 + scheduleOnce(RetryResolveRemoteDeployedRecipient( + retryResolveRemoteDeployedRecipientAttempts, + headerBuilder.recipientActorRefPath.get, decoded), retryResolveRemoteDeployedRecipientInterval) // FIXME IS THIS SAFE? + } else + push(out, decoded) } - - val decoded = inEnvelopePool.acquire().init( - recipient, - localAddress, // FIXME: Is this needed anymore? What should we do here? - sender, - originUid, - headerBuilder.serializer, - classManifest, - 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 - scheduleOnce(RetryResolveRemoteDeployedRecipient( - retryResolveRemoteDeployedRecipientAttempts, - headerBuilder.recipientActorRefPath.get, decoded), retryResolveRemoteDeployedRecipientInterval) // FIXME IS THIS SAFE? - } else - push(out, decoded) } private def resolveRecipient(path: String): OptionVal[InternalActorRef] = { 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 index c558d57853..ddcfd0bda5 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala @@ -251,10 +251,12 @@ private[remote] abstract class InboundCompression[T >: Null]( confirmAdvertisement(incomingTableVersion) decompressInternal(incomingTableVersion, idx, attemptCounter + 1) // recurse, activeTable will not be able to handle this } else { - // which means that incoming version was > nextTable.version, which likely is a bug - log.error( + // 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 is using compression table version higher than the highest allocated table on this node. " + - "This should not happen! State: activeTable: {}, nextTable: {}, incoming tableVersion: {}", + "It was probably sent with compression table built for previous incarnation of this system. " + + "State: activeTable: {}, nextTable: {}, incoming tableVersion: {}", activeVersion, current.nextTable.version, incomingTableVersion) OptionVal.None } From b59f03fa34208d49ae2317b511cc694d52601f77 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 5 Sep 2016 14:03:52 +0200 Subject: [PATCH 105/186] fix failing SystemMessageDeliverySpec, #21331 --- .../src/test/scala/akka/remote/artery/TestContext.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala index 5ae7d99c58..9a350984ef 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -126,12 +126,12 @@ private[remote] class ManualReplyInboundContext( private var lastReply: Option[(Address, ControlMessage)] = None - override def sendControl(to: Address, message: ControlMessage) = { + override def sendControl(to: Address, message: ControlMessage): Unit = synchronized { lastReply = Some((to, message)) replyProbe ! message } - def deliverLastReply(): Unit = { + def deliverLastReply(): Unit = synchronized { lastReply.foreach { case (to, message) ⇒ super.sendControl(to, message) } lastReply = None } From 9287a28702cd6b2d84965655ac109c7743ea0e00 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Tue, 6 Sep 2016 11:50:10 +0200 Subject: [PATCH 106/186] Artery transport shutdown improvements (#21357) * Make sure streams have stopped before shutting down aeron etc * Log completion failures rather than failing shutdown --- .../akka/remote/artery/ArteryTransport.scala | 34 +++++++++++++++++-- .../akka/remote/artery/Association.scala | 26 ++++++++++++-- 2 files changed, 55 insertions(+), 5 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 40bbfd709f..ba5fd1f4b1 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -290,6 +290,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val codec: AkkaPduCodec = AkkaPduProtobufCodec private val killSwitch: SharedKillSwitch = KillSwitches.shared("transportKillSwitch") + private[this] val streamCompletions = new AtomicReference(Map.empty[String, Future[Done]]) @volatile private[this] var _shutdown = false private val testStages: CopyOnWriteArrayList[TestManagementApi] = new CopyOnWriteArrayList @@ -663,8 +664,11 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private def attachStreamRestart(streamName: String, streamCompleted: Future[Done], restart: () ⇒ Unit): Unit = { implicit val ec = materializer.executionContext + updateStreamCompletion(streamName, streamCompleted.recover { case _ ⇒ Done }) streamCompleted.onFailure { - case _ if isShutdown ⇒ // don't restart after shutdown + 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 ⇒ if (restartCounter.restart()) { @@ -690,9 +694,12 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R flushingPromise.future } implicit val ec = remoteDispatcher - flushing.recover { case _ ⇒ Done }.map { _ ⇒ - killSwitch.shutdown() + for { + _ ← flushing.recover { case _ ⇒ Done } + _ = killSwitch.shutdown() + _ ← streamsCompleted + } yield { topLevelFREvents.loFreq(Transport_KillSwitchPulled, NoMetaData) if (taskRunner != null) { taskRunner.stop() @@ -718,6 +725,27 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } } + // set the future that completes when the current stream for a given name completes + @tailrec + private def updateStreamCompletion(streamName: String, streamCompleted: Future[Done]): Unit = { + val prev = streamCompletions.get() + if (!streamCompletions.compareAndSet(prev, prev + (streamName → streamCompleted))) { + updateStreamCompletion(streamName, streamCompleted) + } + } + + /** + * 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 = remoteDispatcher + for { + _ ← Future.traverse(associationRegistry.allAssociations)(_.streamsCompleted) + _ ← Future.sequence(streamCompletions.get().valuesIterator) + } yield Done + } + private[remote] def isShutdown: Boolean = _shutdown override def managementCommand(cmd: Any): Future[Boolean] = { diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index d8557b81af..cd361840d0 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -122,6 +122,7 @@ private[remote] class Association( @volatile private[this] var _outboundControlIngress: OutboundControlIngress = _ @volatile private[this] var materializing = new CountDownLatch(1) @volatile private[this] var changeOutboundCompression: Option[Vector[ChangeOutboundCompression]] = None + private[this] val streamCompletions = new AtomicReference(Map.empty[String, Future[Done]]) def changeActorRefCompression(table: CompressionTable[ActorRef]): Future[Done] = { import transport.system.dispatcher @@ -534,8 +535,11 @@ private[remote] class Association( } implicit val ec = materializer.executionContext + updateStreamCompletion(streamName, streamCompleted.recover { case _ ⇒ Done }) streamCompleted.onFailure { - case _ if transport.isShutdown ⇒ // don't restart after shutdown + 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) case _: AbruptTerminationException ⇒ // ActorSystem shutdown case cause: GaveUpSendingException ⇒ log.debug("{} to {} failed. Restarting it. {}", streamName, remoteAddress, cause.getMessage) @@ -546,7 +550,7 @@ private[remote] class Association( 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}", + log.error(cause, s"{} to {} failed and restarted {} times within {} seconds. Terminating system. {cause.getMessage}", streamName, remoteAddress, transport.settings.Advanced.OutboundMaxRestarts, transport.settings.Advanced.OutboundRestartTimeout.toSeconds) if (queueIndex == ControlQueueIndex) { cause match { @@ -561,6 +565,24 @@ private[remote] class Association( } } + // set the future that completes when the current stream for a given name completes + @tailrec + private def updateStreamCompletion(streamName: String, streamCompleted: Future[Done]): Unit = { + val prev = streamCompletions.get() + if (!streamCompletions.compareAndSet(prev, prev + (streamName → streamCompleted))) { + updateStreamCompletion(streamName, streamCompleted) + } + } + + /** + * 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(streamCompletions.get().values).map(_ ⇒ Done) + } + override def toString: String = s"Association($localAddress -> $remoteAddress with $associationState)" From 294947a9a2bffdd67724a376c3721b5d53b761c1 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 6 Sep 2016 08:04:02 +0200 Subject: [PATCH 107/186] fix glitch in lazy restart, #21347 --- .../akka/remote/artery/Association.scala | 104 +++++++++++------- .../akka/remote/artery/LateConnectSpec.scala | 65 +++++++++++ 2 files changed, 128 insertions(+), 41 deletions(-) create mode 100644 akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index cd361840d0..db52ef48a5 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -61,9 +61,13 @@ private[remote] object Association { final case class LazyQueueWrapper(queue: Queue[OutboundEnvelope], materialize: () ⇒ Unit) extends QueueWrapper { private val onlyOnce = new AtomicBoolean - override def offer(message: OutboundEnvelope): Boolean = { + def runMaterialize(): Unit = { if (onlyOnce.compareAndSet(false, true)) materialize() + } + + override def offer(message: OutboundEnvelope): Boolean = { + runMaterialize() queue.offer(message) } } @@ -119,39 +123,48 @@ private[remote] class Association( private def controlQueue: SendQueue.ProducerApi[OutboundEnvelope] = queues(ControlQueueIndex) private def largeQueue: SendQueue.ProducerApi[OutboundEnvelope] = queues(LargeQueueIndex) - @volatile private[this] var _outboundControlIngress: OutboundControlIngress = _ + @volatile private[this] var _outboundControlIngress: OptionVal[OutboundControlIngress] = OptionVal.None @volatile private[this] var materializing = new CountDownLatch(1) - @volatile private[this] var changeOutboundCompression: Option[Vector[ChangeOutboundCompression]] = None + @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[artery] 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) + } private[this] val streamCompletions = new AtomicReference(Map.empty[String, Future[Done]]) - def changeActorRefCompression(table: CompressionTable[ActorRef]): Future[Done] = { + private[artery] def changeClassManifestCompression(table: CompressionTable[String]): Future[Done] = { import transport.system.dispatcher - changeOutboundCompression match { - case Some(c) ⇒ - if (c.size == 1) c.head.changeActorRefCompression(table) - else Future.sequence(c.map(_.changeActorRefCompression(table))).map(_ ⇒ Done) - case None ⇒ Future.failed(new ChangeOutboundCompressionFailed) - } + 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) } - def changeClassManifestCompression(table: CompressionTable[String]): Future[Done] = { + private[artery] def clearCompression(): Future[Done] = { import transport.system.dispatcher - changeOutboundCompression match { - case Some(c) ⇒ - if (c.size == 1) c.head.changeClassManifestCompression(table) - else Future.sequence(c.map(_.changeClassManifestCompression(table))).map(_ ⇒ Done) - case None ⇒ Future.failed(new ChangeOutboundCompressionFailed) - } + 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) } - def clearCompression(): Future[Done] = { + private def timeoutAfter[T](f: Future[T], timeout: FiniteDuration, e: ⇒ Throwable): Future[T] = { + import akka.pattern.after import transport.system.dispatcher - changeOutboundCompression match { - case Some(c) ⇒ - if (c.size == 1) c.head.clearCompression() - else Future.sequence(c.map(_.clearCompression())).map(_ ⇒ Done) - case None ⇒ Future.failed(new ChangeOutboundCompressionFailed) - } + val f2 = after(timeout, transport.system.scheduler)(Future.failed(e)) + Future.firstCompletedOf(List(f, f2)) } private val _testStages: CopyOnWriteArrayList[TestManagementApi] = new CopyOnWriteArrayList @@ -164,14 +177,19 @@ private[remote] class Association( private def deadletters = transport.system.deadLetters def outboundControlIngress: OutboundControlIngress = { - if (_outboundControlIngress ne null) - _outboundControlIngress - else { - // materialization not completed yet - materializing.await(10, TimeUnit.SECONDS) - if (_outboundControlIngress eq null) - throw new IllegalStateException("outboundControlIngress not initialized yet") - _outboundControlIngress + _outboundControlIngress match { + case OptionVal.Some(o) ⇒ o + case OptionVal.None ⇒ + controlQueue match { + case w: LazyQueueWrapper ⇒ w.runMaterialize() + case _ ⇒ + } + // materialization not completed yet + materializing.await(10, TimeUnit.SECONDS) + _outboundControlIngress match { + case OptionVal.Some(o) ⇒ o + case OptionVal.None ⇒ throw new IllegalStateException("outboundControlIngress not initialized yet") + } } } @@ -416,7 +434,7 @@ private[remote] class Association( // replace with the materialized value, still same underlying queue queues(ControlQueueIndex) = queueValue queuesVisibility = true // volatile write for visibility of the queues array - _outboundControlIngress = control + _outboundControlIngress = OptionVal.Some(control) materializing.countDown() attachStreamRestart("Outbound control stream", ControlQueueIndex, controlQueueSize, completed, () ⇒ runOutboundControlStream()) @@ -452,7 +470,7 @@ private[remote] class Association( // replace with the materialized value, still same underlying queue queues(queueIndex) = queueValue queuesVisibility = true // volatile write for visibility of the queues array - changeOutboundCompression = Some(Vector(changeCompression)) + changeOutboundCompression = Vector(changeCompression) attachStreamRestart("Outbound message stream", OrdinaryQueueIndex, queueSize, completed, () ⇒ runOutboundOrdinaryMessagesStream()) @@ -497,7 +515,7 @@ private[remote] class Association( } queuesVisibility = true // volatile write for visibility of the queues array - changeOutboundCompression = Some(changeCompressionValues) + changeOutboundCompression = changeCompressionValues attachStreamRestart("Outbound message stream", OrdinaryQueueIndex, queueSize, completed, () ⇒ runOutboundOrdinaryMessagesStream()) @@ -529,6 +547,9 @@ private[remote] class Association( streamCompleted: Future[Done], restart: () ⇒ Unit): Unit = { def lazyRestart(): Unit = { + changeOutboundCompression = Vector.empty + if (queueIndex == ControlQueueIndex) + _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 @@ -546,18 +567,19 @@ private[remote] class Association( // 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, transport.settings.Advanced.OutboundMaxRestarts, transport.settings.Advanced.OutboundRestartTimeout.toSeconds) - if (queueIndex == ControlQueueIndex) { - cause match { - case _: HandshakeTimeoutException ⇒ // ok, quarantine not possible without UID - case _ ⇒ quarantine("Outbound control stream restarted") - } - } // FIXME is this the right thing to do for outbound? transport.system.terminate() 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..0a208b1712 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala @@ -0,0 +1,65 @@ +/** + * 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.hostname = localhost + remote.artery.port = 0 + remote.artery.advanced.handshake-timeout = 3s + } + """) + +} + +class LateConnectSpec extends AkkaSpec(LateConnectSpec.config) with ImplicitSender { + + val portB = SocketUtil.temporaryServerAddress("localhost", udp = true).getPort + val configB = ConfigFactory.parseString(s"akka.remote.artery.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"artery://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) + +} From edf1c83839d0e3803ab2f0c64403a428b908d02e Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 7 Sep 2016 08:27:33 +0200 Subject: [PATCH 108/186] abort streams on shutdown, #21388 * otherwise AeronSink will continue sending outstanding messages before completing * this was noticed by RemoteDeathWatchSpec couldn't shutdown, since it was trying to send to unknown --- .../main/scala/akka/remote/artery/ArteryTransport.scala | 8 +++++++- .../src/main/scala/akka/remote/artery/Association.scala | 1 + 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index ba5fd1f4b1..bc8d27df53 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -69,6 +69,7 @@ import org.agrona.ErrorHandler import org.agrona.IoUtil import org.agrona.concurrent.BackoffIdleStrategy import akka.stream.scaladsl.BroadcastHub +import scala.util.control.NoStackTrace /** * INTERNAL API @@ -242,6 +243,7 @@ private[remote] class FlushOnShutdown(done: Promise[Done], timeout: FiniteDurati val timeoutTask = context.system.scheduler.scheduleOnce(timeout, self, FlushOnShutdown.Timeout)(context.dispatcher) override def preStart(): Unit = { + // FIXME shall we also try to flush the ordinary message stream, not only control stream? val msg = ActorSystemTerminating(inboundContext.localAddress) associations.foreach { a ⇒ a.send(msg, OptionVal.Some(self), OptionVal.None) } } @@ -268,6 +270,7 @@ private[remote] class FlushOnShutdown(done: Promise[Done], timeout: FiniteDurati private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: RemoteActorRefProvider) extends RemoteTransport(_system, _provider) with InboundContext { import FlightRecorderEvents._ + import ArteryTransport.ShutdownSignal // these vars are initialized once in the start method @volatile private[this] var _localAddress: UniqueAddress = _ @@ -666,6 +669,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R implicit val ec = materializer.executionContext updateStreamCompletion(streamName, streamCompleted.recover { case _ ⇒ Done }) streamCompleted.onFailure { + case ShutdownSignal ⇒ // shutdown as expected 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) @@ -697,7 +701,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R for { _ ← flushing.recover { case _ ⇒ Done } - _ = killSwitch.shutdown() + _ = killSwitch.abort(ShutdownSignal) _ ← streamsCompleted } yield { topLevelFREvents.loFreq(Transport_KillSwitchPulled, NoMetaData) @@ -966,4 +970,6 @@ private[remote] object ArteryTransport { port } + object ShutdownSignal extends RuntimeException with NoStackTrace + } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index cd361840d0..0456ceb62b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -537,6 +537,7 @@ private[remote] class Association( implicit val ec = materializer.executionContext updateStreamCompletion(streamName, streamCompleted.recover { case _ ⇒ Done }) streamCompleted.onFailure { + case ArteryTransport.ShutdownSignal ⇒ // shutdown as expected 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) From 9fd359042a2264e6645a7d3b3ab014fb164282a2 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 6 Sep 2016 16:57:30 +0200 Subject: [PATCH 109/186] add missing killSwitch for parallel outbound lanes, #21381 * it caused the shutdown to stall, since the part after MergeHub was never stopped * tear down parts upstream and downstream of the hub toghether --- .../akka/remote/artery/ArteryTransport.scala | 8 ++++++++ .../scala/akka/remote/artery/Association.scala | 16 +++++++++++++++- 2 files changed, 23 insertions(+), 1 deletion(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index ba5fd1f4b1..a4d02673aa 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -607,7 +607,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R completed } else { + val hubKillSwitch = KillSwitches.shared("hubKillSwitch") val source = aeronSource(ordinaryStreamId, envelopeBufferPool) + .via(hubKillSwitch.flow) .via(inboundFlow(compression)) .map(env ⇒ (env.recipient, env)) @@ -642,6 +644,12 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R 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) + } + completed } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index cd361840d0..92608b00e4 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -45,6 +45,9 @@ import akka.remote.artery.compress.CompressionProtocol._ import akka.stream.scaladsl.MergeHub import akka.actor.DeadLetter import java.util.concurrent.atomic.AtomicBoolean +import akka.stream.KillSwitches +import scala.util.Failure +import scala.util.Success /** * INTERNAL API @@ -465,7 +468,10 @@ private[remote] class Association( wrapper }.toVector + val laneKillSwitch = KillSwitches.shared("outboundLaneKillSwitch") + val lane = Source.fromGraph(new SendQueue[OutboundEnvelope]) + .via(laneKillSwitch.flow) .viaMat(transport.outboundTestFlow(this))(Keep.both) .viaMat(transport.outboundLane(this))(Keep.both) .watchTermination()(Keep.both) @@ -473,7 +479,9 @@ private[remote] class Association( case (((q, m), c), w) ⇒ ((q, m), (c, w)) } - val (mergeHub, aeronSinkCompleted) = MergeHub.source[EnvelopeBuffer].toMat(transport.aeronSink(this))(Keep.both).run()(materializer) + val (mergeHub, aeronSinkCompleted) = MergeHub.source[EnvelopeBuffer] + .via(laneKillSwitch.flow) + .toMat(transport.aeronSink(this))(Keep.both).run()(materializer) val values: Vector[((SendQueue.QueueValue[OutboundEnvelope], TestManagementApi), (Encoder.ChangeOutboundCompression, Future[Done]))] = (0 until outboundLanes).map { _ ⇒ @@ -490,6 +498,12 @@ private[remote] class Association( 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 ⇒ laneKillSwitch.abort(reason) + } + (laneCompletedValues :+ aeronSinkCompleted).foreach(_.onSuccess { case _ ⇒ laneKillSwitch.shutdown() }) + queueValues.zip(wrappers).zipWithIndex.foreach { case ((q, w), i) ⇒ q.inject(w.queue) From f4b82ce62be9da96defb104a51cfa69e895bf019 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 6 Sep 2016 14:32:42 +0200 Subject: [PATCH 110/186] handle fatal Aeron exceptions, #20561 * handle ConductorServiceTimeoutException and DriverTimeoutException * shutdown things properly, in the right order, and without overwhelming the logs with exceptions --- .../akka/remote/artery/ArteryTransport.scala | 71 ++++++--- project/AkkaBuild.scala | 142 +++++++++--------- project/Doc.scala | 3 +- project/Release.scala | 12 +- 4 files changed, 130 insertions(+), 98 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index fb9957a401..41c7c74332 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -70,6 +70,8 @@ import org.agrona.IoUtil import org.agrona.concurrent.BackoffIdleStrategy import akka.stream.scaladsl.BroadcastHub import scala.util.control.NoStackTrace +import io.aeron.exceptions.DriverTimeoutException +import java.util.concurrent.atomic.AtomicBoolean /** * INTERNAL API @@ -245,21 +247,27 @@ private[remote] class FlushOnShutdown(done: Promise[Done], timeout: FiniteDurati override def preStart(): Unit = { // FIXME shall we also try to flush the ordinary message stream, not only control stream? val msg = ActorSystemTerminating(inboundContext.localAddress) - associations.foreach { a ⇒ a.send(msg, OptionVal.Some(self), OptionVal.None) } + try { + associations.foreach { a ⇒ a.send(msg, OptionVal.Some(self), OptionVal.None) } + } catch { + case NonFatal(e) ⇒ + // send may throw + done.tryFailure(e) + throw e + } } - override def postStop(): Unit = + override def postStop(): Unit = { timeoutTask.cancel() + done.trySuccess(Done) + } def receive = { case ActorSystemTerminatingAck(from) ⇒ remaining -= from - if (remaining.isEmpty) { - done.trySuccess(Done) + if (remaining.isEmpty) context.stop(self) - } case FlushOnShutdown.Timeout ⇒ - done.trySuccess(Done) context.stop(self) } } @@ -271,6 +279,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R extends RemoteTransport(_system, _provider) with InboundContext { import FlightRecorderEvents._ import ArteryTransport.ShutdownSignal + import ArteryTransport.AeronTerminated // these vars are initialized once in the start method @volatile private[this] var _localAddress: UniqueAddress = _ @@ -474,6 +483,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R 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) @@ -487,17 +498,37 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // FIXME we should call FragmentAssembler.freeSessionBuffer when image is unavailable } }) + ctx.errorHandler(new ErrorHandler { + private val fatalErrorOccured = new AtomicBoolean + override def onError(cause: Throwable): Unit = { cause match { - case e: ConductorServiceTimeoutException ⇒ - // Timeout between service calls - log.error(cause, s"Aeron ServiceTimeoutException, ${cause.getMessage}") - + 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) @@ -713,10 +744,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R _ ← streamsCompleted } yield { topLevelFREvents.loFreq(Transport_KillSwitchPulled, NoMetaData) - if (taskRunner != null) { - taskRunner.stop() - topLevelFREvents.loFreq(Transport_Stopped, NoMetaData) - } + taskRunner.stop() + topLevelFREvents.loFreq(Transport_Stopped, NoMetaData) + if (aeronErrorLogTask != null) { aeronErrorLogTask.cancel() topLevelFREvents.loFreq(Transport_AeronErrorLogTaskStopped, NoMetaData) @@ -965,12 +995,11 @@ private[remote] object ArteryTransport { val Version = 0 - /** - * Internal API - * - * @return A port that is hopefully available - */ - private[remote] def autoSelectPort(hostname: String): Int = { + class AeronTerminated(e: Throwable) extends RuntimeException(e) + + object ShutdownSignal extends RuntimeException with NoStackTrace + + def autoSelectPort(hostname: String): Int = { val socket = DatagramChannel.open().socket() socket.bind(new InetSocketAddress(hostname, 0)) val port = socket.getLocalPort @@ -978,6 +1007,4 @@ private[remote] object ArteryTransport { port } - object ShutdownSignal extends RuntimeException with NoStackTrace - } diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index e5453761fb..220bdbdd06 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -82,7 +82,7 @@ object AkkaBuild extends Build { protobuf, remote, remoteTests, -// samples, +// samples, // FIXME temporary in artery-dev branch slf4j, stream, streamTestkit, @@ -98,13 +98,14 @@ object AkkaBuild extends Build { aggregate = aggregatedProjects ).settings(rootSettings: _*) - lazy val akkaScalaNightly = Project( - id = "akka-scala-nightly", - base = file("akka-scala-nightly"), - // remove dependencies that we have to build ourselves (Scala STM) - // samples don't work with dbuild right now - aggregate = aggregatedProjects diff List(agent, docs, samples) - ).disablePlugins(ValidatePullRequest, MimaPlugin) +// FIXME temporary in artery-dev branch +// lazy val akkaScalaNightly = Project( +// id = "akka-scala-nightly", +// base = file("akka-scala-nightly"), +// // remove dependencies that we have to build ourselves (Scala STM) +// // samples don't work with dbuild right now +// aggregate = aggregatedProjects diff List(agent, docs, samples) +// ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val actor = Project( id = "akka-actor", @@ -371,68 +372,69 @@ object AkkaBuild extends Build { lazy val samplesSettings = parentSettings ++ ActivatorDist.settings - lazy val samples = Project( - id = "akka-samples", - base = file("akka-samples"), - // FIXME osgiDiningHakkersSampleMavenTest temporarily removed from aggregate due to #16703 - aggregate = if (!Sample.CliOptions.aggregateSamples) Nil else - Seq(sampleCamelJava, sampleCamelScala, sampleClusterJava, sampleClusterScala, sampleFsmScala, sampleFsmJavaLambda, - sampleMainJava, sampleMainScala, sampleMainJavaLambda, sampleMultiNodeScala, - samplePersistenceJava, samplePersistenceScala, samplePersistenceJavaLambda, - sampleRemoteJava, sampleRemoteScala, sampleSupervisionJavaLambda, - sampleDistributedDataScala, sampleDistributedDataJava) - ) - .settings(samplesSettings: _*) - .disablePlugins(MimaPlugin) - - lazy val sampleCamelJava = Sample.project("akka-sample-camel-java") - lazy val sampleCamelScala = Sample.project("akka-sample-camel-scala") - - lazy val sampleClusterJava = Sample.project("akka-sample-cluster-java") - lazy val sampleClusterScala = Sample.project("akka-sample-cluster-scala") - - lazy val sampleFsmScala = Sample.project("akka-sample-fsm-scala") - lazy val sampleFsmJavaLambda = Sample.project("akka-sample-fsm-java-lambda") - - lazy val sampleMainJava = Sample.project("akka-sample-main-java") - lazy val sampleMainScala = Sample.project("akka-sample-main-scala") - lazy val sampleMainJavaLambda = Sample.project("akka-sample-main-java-lambda") - - lazy val sampleMultiNodeScala = Sample.project("akka-sample-multi-node-scala") - - lazy val samplePersistenceJava = Sample.project("akka-sample-persistence-java") - lazy val samplePersistenceScala = Sample.project("akka-sample-persistence-scala") - lazy val samplePersistenceJavaLambda = Sample.project("akka-sample-persistence-java-lambda") - - lazy val sampleRemoteJava = Sample.project("akka-sample-remote-java") - lazy val sampleRemoteScala = Sample.project("akka-sample-remote-scala") - - lazy val sampleSupervisionJavaLambda = Sample.project("akka-sample-supervision-java-lambda") - - lazy val sampleDistributedDataScala = Sample.project("akka-sample-distributed-data-scala") - lazy val sampleDistributedDataJava = Sample.project("akka-sample-distributed-data-java") - - lazy val osgiDiningHakkersSampleMavenTest = Project( - id = "akka-sample-osgi-dining-hakkers-maven-test", - base = file("akka-samples/akka-sample-osgi-dining-hakkers-maven-test") - ) - .settings( - publishArtifact := false, - // force publication of artifacts to local maven repo, so latest versions can be used when running maven tests - compile in Compile <<= - (publishM2 in actor, publishM2 in testkit, publishM2 in remote, publishM2 in cluster, publishM2 in osgi, - publishM2 in slf4j, publishM2 in persistence, compile in Compile) map - ((_, _, _, _, _, _, _, c) => c), - test in Test ~= { x => { - def executeMvnCommands(failureMessage: String, commands: String*) = { - if ({List("sh", "-c", commands.mkString("cd akka-samples/akka-sample-osgi-dining-hakkers; mvn ", " ", "")) !} != 0) - throw new Exception(failureMessage) - } - executeMvnCommands("Osgi sample Dining hakkers test failed", "clean", "install") - }} - ) - .disablePlugins(ValidatePullRequest, MimaPlugin) - .settings(dontPublishSettings: _*) +// FIXME temporary in artery-dev branch +// lazy val samples = Project( +// id = "akka-samples", +// base = file("akka-samples"), +// // FIXME osgiDiningHakkersSampleMavenTest temporarily removed from aggregate due to #16703 +// aggregate = if (!Sample.CliOptions.aggregateSamples) Nil else +// Seq(sampleCamelJava, sampleCamelScala, sampleClusterJava, sampleClusterScala, sampleFsmScala, sampleFsmJavaLambda, +// sampleMainJava, sampleMainScala, sampleMainJavaLambda, sampleMultiNodeScala, +// samplePersistenceJava, samplePersistenceScala, samplePersistenceJavaLambda, +// sampleRemoteJava, sampleRemoteScala, sampleSupervisionJavaLambda, +// sampleDistributedDataScala, sampleDistributedDataJava) +// ) +// .settings(samplesSettings: _*) +// .disablePlugins(MimaPlugin) +// +// lazy val sampleCamelJava = Sample.project("akka-sample-camel-java") +// lazy val sampleCamelScala = Sample.project("akka-sample-camel-scala") +// +// lazy val sampleClusterJava = Sample.project("akka-sample-cluster-java") +// lazy val sampleClusterScala = Sample.project("akka-sample-cluster-scala") +// +// lazy val sampleFsmScala = Sample.project("akka-sample-fsm-scala") +// lazy val sampleFsmJavaLambda = Sample.project("akka-sample-fsm-java-lambda") +// +// lazy val sampleMainJava = Sample.project("akka-sample-main-java") +// lazy val sampleMainScala = Sample.project("akka-sample-main-scala") +// lazy val sampleMainJavaLambda = Sample.project("akka-sample-main-java-lambda") +// +// lazy val sampleMultiNodeScala = Sample.project("akka-sample-multi-node-scala") +// +// lazy val samplePersistenceJava = Sample.project("akka-sample-persistence-java") +// lazy val samplePersistenceScala = Sample.project("akka-sample-persistence-scala") +// lazy val samplePersistenceJavaLambda = Sample.project("akka-sample-persistence-java-lambda") +// +// lazy val sampleRemoteJava = Sample.project("akka-sample-remote-java") +// lazy val sampleRemoteScala = Sample.project("akka-sample-remote-scala") +// +// lazy val sampleSupervisionJavaLambda = Sample.project("akka-sample-supervision-java-lambda") +// +// lazy val sampleDistributedDataScala = Sample.project("akka-sample-distributed-data-scala") +// lazy val sampleDistributedDataJava = Sample.project("akka-sample-distributed-data-java") +// +// lazy val osgiDiningHakkersSampleMavenTest = Project( +// id = "akka-sample-osgi-dining-hakkers-maven-test", +// base = file("akka-samples/akka-sample-osgi-dining-hakkers-maven-test") +// ) +// .settings( +// publishArtifact := false, +// // force publication of artifacts to local maven repo, so latest versions can be used when running maven tests +// compile in Compile <<= +// (publishM2 in actor, publishM2 in testkit, publishM2 in remote, publishM2 in cluster, publishM2 in osgi, +// publishM2 in slf4j, publishM2 in persistence, compile in Compile) map +// ((_, _, _, _, _, _, _, c) => c), +// test in Test ~= { x => { +// def executeMvnCommands(failureMessage: String, commands: String*) = { +// if ({List("sh", "-c", commands.mkString("cd akka-samples/akka-sample-osgi-dining-hakkers; mvn ", " ", "")) !} != 0) +// throw new Exception(failureMessage) +// } +// executeMvnCommands("Osgi sample Dining hakkers test failed", "clean", "install") +// }} +// ) +// .disablePlugins(ValidatePullRequest, MimaPlugin) +// .settings(dontPublishSettings: _*) val dontPublishSettings = Seq( publishSigned := (), diff --git a/project/Doc.scala b/project/Doc.scala index 2294bd2315..cb8b3b9b74 100644 --- a/project/Doc.scala +++ b/project/Doc.scala @@ -118,9 +118,10 @@ object UnidocRoot extends AutoPlugin { )) } + // FIXME temporary removal of samples in artery-dev branch override lazy val projectSettings = CliOptions.genjavadocEnabled.ifTrue(scalaJavaUnidocSettings).getOrElse(scalaUnidocSettings) ++ - settings(Seq(AkkaBuild.samples), Seq(AkkaBuild.remoteTests, AkkaBuild.benchJmh, AkkaBuild.parsing, AkkaBuild.protobuf, AkkaBuild.osgiDiningHakkersSampleMavenTest, AkkaBuild.akkaScalaNightly)) + settings(Seq(), Seq(AkkaBuild.remoteTests, AkkaBuild.benchJmh, AkkaBuild.parsing, AkkaBuild.protobuf)) } /** diff --git a/project/Release.scala b/project/Release.scala index 33c898faed..dd2d72830e 100644 --- a/project/Release.scala +++ b/project/Release.scala @@ -33,7 +33,8 @@ object Release { val (state2, Seq(api, japi)) = extracted.runTask(unidoc in Compile, state1) val (state3, docs) = extracted.runTask(generate in Sphinx, state2) val (state4, _) = extracted.runTask(Dist.dist, state3) - val (state5, activatorDist) = extracted.runTask(ActivatorDist.activatorDist in LocalProject(AkkaBuild.samples.id), state4) +// FIXME temporary in artery-dev branch +// val (state5, activatorDist) = extracted.runTask(ActivatorDist.activatorDist in LocalProject(AkkaBuild.samples.id), state4) IO.delete(release) IO.createDirectory(release) @@ -47,10 +48,11 @@ object Release { for (f <- (dist * "akka_*.zip").get) IO.copyFile(f, release / "downloads" / f.name) - for (f <- (activatorDist * "*.zip").get) - IO.copyFile(f, release / "downloads" / f.name) - - state5 +// FIXME temporary in artery-dev branch +// for (f <- (activatorDist * "*.zip").get) +// IO.copyFile(f, release / "downloads" / f.name) +// state5 + state4 } def uploadReleaseCommand = Command.command("uploadRelease") { state => From 90193907fe081c05836902d9188363124cb3ae11 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Wed, 7 Sep 2016 16:41:03 +0200 Subject: [PATCH 111/186] Make cluster tests run with artery #21204 --- .../scala/akka/cluster/NodeChurnSpec.scala | 4 + .../scala/akka/cluster/QuickRestartSpec.scala | 40 +- .../cluster/RestartFirstSeedNodeSpec.scala | 10 +- .../scala/akka/cluster/RestartNode3Spec.scala | 10 +- .../scala/akka/cluster/StressSpec.scala | 387 +++++++++--------- .../scala/akka/cluster/SunnyWeatherSpec.scala | 45 +- .../SurviveNetworkInstabilitySpec.scala | 23 +- .../UnreachableNodeJoinsAgainSpec.scala | 27 +- .../src/main/scala/akka/remote/Remoting.scala | 4 +- 9 files changed, 286 insertions(+), 264 deletions(-) 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..5bc53ba4c7 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.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..adada79382 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.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..652923602c 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.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) { 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 06ae2bc94e..47d2fff641 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala @@ -1129,199 +1129,200 @@ abstract class StressSpec "A cluster under stress" must { - "log settings" taggedAs LongRunningTest in { - if (infolog) { - log.info("StressSpec JVM:\n{}", jvmInfo) - runOn(roles.head) { - log.info("StressSpec settings:\n{}", settings) - } - } - enterBarrier("after-" + step) - } - - "join seed nodes" taggedAs LongRunningTest in within(30 seconds) { - - val otherNodesJoiningSeedNodes = roles.slice(numberOfSeedNodes, numberOfSeedNodes + numberOfNodesJoiningToSeedNodesInitially) - val size = seedNodes.size + otherNodesJoiningSeedNodes.size - - createResultAggregator("join seed nodes", expectedResults = size, includeInHistory = true) - - runOn((seedNodes ++ otherNodesJoiningSeedNodes): _*) { - reportResult { - cluster.joinSeedNodes(seedNodes.toIndexedSeq map address) - awaitMembersUp(size, timeout = remainingOrDefault) - } - } - - awaitClusterResult() - - nbrUsedRoles += size - enterBarrier("after-" + step) - } - - "start routers that are running while nodes are joining" taggedAs LongRunningTest in { - runOn(roles.take(3): _*) { - system.actorOf( - Props(classOf[Master], settings, settings.workBatchInterval, false).withDeploy(Deploy.local), - name = masterName) ! Begin - } - } - - "join nodes one-by-one to small cluster" taggedAs LongRunningTest in { - joinOneByOne(numberOfNodesJoiningOneByOneSmall) - enterBarrier("after-" + step) - } - - "join several nodes to one node" taggedAs LongRunningTest in { - joinSeveral(numberOfNodesJoiningToOneNode, toSeedNodes = false) - nbrUsedRoles += numberOfNodesJoiningToOneNode - enterBarrier("after-" + step) - } - - "join several nodes to seed nodes" taggedAs LongRunningTest in { - if (numberOfNodesJoiningToSeedNodes > 0) { - joinSeveral(numberOfNodesJoiningToSeedNodes, toSeedNodes = true) - nbrUsedRoles += numberOfNodesJoiningToSeedNodes - } - enterBarrier("after-" + step) - } - - "join nodes one-by-one to large cluster" taggedAs LongRunningTest in { - joinOneByOne(numberOfNodesJoiningOneByOneLarge) - enterBarrier("after-" + step) - } - - "end routers that are running while nodes are joining" taggedAs LongRunningTest in within(30.seconds) { - if (exerciseActors) { - runOn(roles.take(3): _*) { - master match { - case Some(m) ⇒ - m.tell(End, testActor) - val workResult = awaitWorkResult(m) - workResult.retryCount should ===(0) - workResult.sendCount should be > (0L) - workResult.ackCount should be > (0L) - case None ⇒ fail("master not running") - } - } - } - enterBarrier("after-" + step) - } - - "use routers with normal throughput" taggedAs LongRunningTest in { - if (exerciseActors) { - exerciseRouters("use routers with normal throughput", normalThroughputDuration, - batchInterval = workBatchInterval, expectDroppedMessages = false, tree = false) - } - enterBarrier("after-" + step) - } - - "use routers with high throughput" taggedAs LongRunningTest in { - if (exerciseActors) { - exerciseRouters("use routers with high throughput", highThroughputDuration, - batchInterval = Duration.Zero, expectDroppedMessages = false, tree = false) - } - enterBarrier("after-" + step) - } - - "use many actors with normal throughput" taggedAs LongRunningTest in { - if (exerciseActors) { - exerciseRouters("use many actors with normal throughput", normalThroughputDuration, - batchInterval = workBatchInterval, expectDroppedMessages = false, tree = true) - } - enterBarrier("after-" + step) - } - - "use many actors with high throughput" taggedAs LongRunningTest in { - if (exerciseActors) { - exerciseRouters("use many actors with high throughput", highThroughputDuration, - batchInterval = Duration.Zero, expectDroppedMessages = false, tree = true) - } - enterBarrier("after-" + step) - } - - "exercise join/remove/join/remove" taggedAs LongRunningTest in { - exerciseJoinRemove("exercise join/remove", joinRemoveDuration) - enterBarrier("after-" + step) - } - - "exercise supervision" taggedAs LongRunningTest in { - if (exerciseActors) { - exerciseSupervision("exercise supervision", supervisionDuration, supervisionOneIteration) - } - enterBarrier("after-" + step) - } - - "gossip when idle" taggedAs LongRunningTest in { - idleGossip("idle gossip") - enterBarrier("after-" + step) - } - - "start routers that are running while nodes are removed" taggedAs LongRunningTest in { - if (exerciseActors) { - runOn(roles.take(3): _*) { - system.actorOf( - Props(classOf[Master], settings, settings.workBatchInterval, false).withDeploy(Deploy.local), - name = masterName) ! Begin - } - } - enterBarrier("after-" + step) - } - - "leave nodes one-by-one from large cluster" taggedAs LongRunningTest in { - removeOneByOne(numberOfNodesLeavingOneByOneLarge, shutdown = false) - enterBarrier("after-" + step) - } - - "shutdown nodes one-by-one from large cluster" taggedAs LongRunningTest in { - removeOneByOne(numberOfNodesShutdownOneByOneLarge, shutdown = true) - enterBarrier("after-" + step) - } - - "leave several nodes" taggedAs LongRunningTest in { - removeSeveral(numberOfNodesLeaving, shutdown = false) - nbrUsedRoles -= numberOfNodesLeaving - enterBarrier("after-" + step) - } - - "shutdown several nodes" taggedAs LongRunningTest in { - removeSeveral(numberOfNodesShutdown, shutdown = true) - nbrUsedRoles -= numberOfNodesShutdown - enterBarrier("after-" + step) - } - - "shutdown nodes one-by-one from small cluster" taggedAs LongRunningTest in { - removeOneByOne(numberOfNodesShutdownOneByOneSmall, shutdown = true) - enterBarrier("after-" + step) - } - - "leave nodes one-by-one from small cluster" taggedAs LongRunningTest in { - removeOneByOne(numberOfNodesLeavingOneByOneSmall, shutdown = false) - enterBarrier("after-" + step) - } - - "end routers that are running while nodes are removed" taggedAs LongRunningTest in within(30.seconds) { - if (exerciseActors) { - runOn(roles.take(3): _*) { - master match { - case Some(m) ⇒ - m.tell(End, testActor) - val workResult = awaitWorkResult(m) - workResult.sendCount should be > (0L) - workResult.ackCount should be > (0L) - case None ⇒ fail("master not running") - } - } - } - enterBarrier("after-" + step) - } - - "log jvm info" taggedAs LongRunningTest in { - if (infolog) { - log.info("StressSpec JVM:\n{}", jvmInfo) - } - enterBarrier("after-" + step) - } + "TODO work with artery" in (pending) + // "log settings" taggedAs LongRunningTest in { + // if (infolog) { + // log.info("StressSpec JVM:\n{}", jvmInfo) + // runOn(roles.head) { + // log.info("StressSpec settings:\n{}", settings) + // } + // } + // enterBarrier("after-" + step) + // } + // + // "join seed nodes" taggedAs LongRunningTest in within(30 seconds) { + // + // val otherNodesJoiningSeedNodes = roles.slice(numberOfSeedNodes, numberOfSeedNodes + numberOfNodesJoiningToSeedNodesInitially) + // val size = seedNodes.size + otherNodesJoiningSeedNodes.size + // + // createResultAggregator("join seed nodes", expectedResults = size, includeInHistory = true) + // + // runOn((seedNodes ++ otherNodesJoiningSeedNodes): _*) { + // reportResult { + // cluster.joinSeedNodes(seedNodes.toIndexedSeq map address) + // awaitMembersUp(size, timeout = remainingOrDefault) + // } + // } + // + // awaitClusterResult() + // + // nbrUsedRoles += size + // enterBarrier("after-" + step) + // } + // + // "start routers that are running while nodes are joining" taggedAs LongRunningTest in { + // runOn(roles.take(3): _*) { + // system.actorOf( + // Props(classOf[Master], settings, settings.workBatchInterval, false).withDeploy(Deploy.local), + // name = masterName) ! Begin + // } + // } + // + // "join nodes one-by-one to small cluster" taggedAs LongRunningTest in { + // joinOneByOne(numberOfNodesJoiningOneByOneSmall) + // enterBarrier("after-" + step) + // } + // + // "join several nodes to one node" taggedAs LongRunningTest in { + // joinSeveral(numberOfNodesJoiningToOneNode, toSeedNodes = false) + // nbrUsedRoles += numberOfNodesJoiningToOneNode + // enterBarrier("after-" + step) + // } + // + // "join several nodes to seed nodes" taggedAs LongRunningTest in { + // if (numberOfNodesJoiningToSeedNodes > 0) { + // joinSeveral(numberOfNodesJoiningToSeedNodes, toSeedNodes = true) + // nbrUsedRoles += numberOfNodesJoiningToSeedNodes + // } + // enterBarrier("after-" + step) + // } + // + // "join nodes one-by-one to large cluster" taggedAs LongRunningTest in { + // joinOneByOne(numberOfNodesJoiningOneByOneLarge) + // enterBarrier("after-" + step) + // } + // + // "end routers that are running while nodes are joining" taggedAs LongRunningTest in within(30.seconds) { + // if (exerciseActors) { + // runOn(roles.take(3): _*) { + // master match { + // case Some(m) ⇒ + // m.tell(End, testActor) + // val workResult = awaitWorkResult(m) + // workResult.retryCount should ===(0) + // workResult.sendCount should be > (0L) + // workResult.ackCount should be > (0L) + // case None ⇒ fail("master not running") + // } + // } + // } + // enterBarrier("after-" + step) + // } + // + // "use routers with normal throughput" taggedAs LongRunningTest in { + // if (exerciseActors) { + // exerciseRouters("use routers with normal throughput", normalThroughputDuration, + // batchInterval = workBatchInterval, expectDroppedMessages = false, tree = false) + // } + // enterBarrier("after-" + step) + // } + // + // "use routers with high throughput" taggedAs LongRunningTest in { + // if (exerciseActors) { + // exerciseRouters("use routers with high throughput", highThroughputDuration, + // batchInterval = Duration.Zero, expectDroppedMessages = false, tree = false) + // } + // enterBarrier("after-" + step) + // } + // + // "use many actors with normal throughput" taggedAs LongRunningTest in { + // if (exerciseActors) { + // exerciseRouters("use many actors with normal throughput", normalThroughputDuration, + // batchInterval = workBatchInterval, expectDroppedMessages = false, tree = true) + // } + // enterBarrier("after-" + step) + // } + // + // "use many actors with high throughput" taggedAs LongRunningTest in { + // if (exerciseActors) { + // exerciseRouters("use many actors with high throughput", highThroughputDuration, + // batchInterval = Duration.Zero, expectDroppedMessages = false, tree = true) + // } + // enterBarrier("after-" + step) + // } + // + // "exercise join/remove/join/remove" taggedAs LongRunningTest in { + // exerciseJoinRemove("exercise join/remove", joinRemoveDuration) + // enterBarrier("after-" + step) + // } + // + // "exercise supervision" taggedAs LongRunningTest in { + // if (exerciseActors) { + // exerciseSupervision("exercise supervision", supervisionDuration, supervisionOneIteration) + // } + // enterBarrier("after-" + step) + // } + // + // "gossip when idle" taggedAs LongRunningTest in { + // idleGossip("idle gossip") + // enterBarrier("after-" + step) + // } + // + // "start routers that are running while nodes are removed" taggedAs LongRunningTest in { + // if (exerciseActors) { + // runOn(roles.take(3): _*) { + // system.actorOf( + // Props(classOf[Master], settings, settings.workBatchInterval, false).withDeploy(Deploy.local), + // name = masterName) ! Begin + // } + // } + // enterBarrier("after-" + step) + // } + // + // "leave nodes one-by-one from large cluster" taggedAs LongRunningTest in { + // removeOneByOne(numberOfNodesLeavingOneByOneLarge, shutdown = false) + // enterBarrier("after-" + step) + // } + // + // "shutdown nodes one-by-one from large cluster" taggedAs LongRunningTest in { + // removeOneByOne(numberOfNodesShutdownOneByOneLarge, shutdown = true) + // enterBarrier("after-" + step) + // } + // + // "leave several nodes" taggedAs LongRunningTest in { + // removeSeveral(numberOfNodesLeaving, shutdown = false) + // nbrUsedRoles -= numberOfNodesLeaving + // enterBarrier("after-" + step) + // } + // + // "shutdown several nodes" taggedAs LongRunningTest in { + // removeSeveral(numberOfNodesShutdown, shutdown = true) + // nbrUsedRoles -= numberOfNodesShutdown + // enterBarrier("after-" + step) + // } + // + // "shutdown nodes one-by-one from small cluster" taggedAs LongRunningTest in { + // removeOneByOne(numberOfNodesShutdownOneByOneSmall, shutdown = true) + // enterBarrier("after-" + step) + // } + // + // "leave nodes one-by-one from small cluster" taggedAs LongRunningTest in { + // removeOneByOne(numberOfNodesLeavingOneByOneSmall, shutdown = false) + // enterBarrier("after-" + step) + // } + // + // "end routers that are running while nodes are removed" taggedAs LongRunningTest in within(30.seconds) { + // if (exerciseActors) { + // runOn(roles.take(3): _*) { + // master match { + // case Some(m) ⇒ + // m.tell(End, testActor) + // val workResult = awaitWorkResult(m) + // workResult.sendCount should be > (0L) + // workResult.ackCount should be > (0L) + // case None ⇒ fail("master not running") + // } + // } + // } + // enterBarrier("after-" + step) + // } + // + // "log jvm info" taggedAs LongRunningTest in { + // if (infolog) { + // log.info("StressSpec JVM:\n{}", jvmInfo) + // } + // enterBarrier("after-" + step) + // } } } 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 7ab3a00bfb..2934581077 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala @@ -14,14 +14,16 @@ import scala.collection.immutable.SortedSet import akka.actor.Props import akka.actor.Actor -class SunnyWeatherMultiJvmSpec(artery: Boolean) extends MultiNodeConfig { +object SunnyWeatherMultiJvmSpec extends MultiNodeConfig { val first = role("first") val second = role("second") val third = role("third") val fourth = role("fourth") val fifth = role("fifth") - val common = + // Note that this test uses default configuration, + // not MultiNodeClusterSpec.clusterConfig + commonConfig(ConfigFactory.parseString( """ akka { actor.provider = cluster @@ -30,43 +32,20 @@ class SunnyWeatherMultiJvmSpec(artery: Boolean) extends MultiNodeConfig { remote.log-remote-lifecycle-events = off cluster.failure-detector.monitored-by-nr-of-members = 3 } - """ - - val arteryConfig = - """ - akka.remote.artery { - enabled = on - } - """ - - // Note that this test uses default configuration, - // not MultiNodeClusterSpec.clusterConfig - commonConfig( - if (artery) ConfigFactory.parseString(arteryConfig).withFallback(ConfigFactory.parseString(common)) - else ConfigFactory.parseString(common)) + """)) } -class SunnyWeatherRemotingMultiJvmNode1 extends SunnyWeatherRemotingSpec -class SunnyWeatherRemotingMultiJvmNode2 extends SunnyWeatherRemotingSpec -class SunnyWeatherRemotingMultiJvmNode3 extends SunnyWeatherRemotingSpec -class SunnyWeatherRemotingMultiJvmNode4 extends SunnyWeatherRemotingSpec -class SunnyWeatherRemotingMultiJvmNode5 extends SunnyWeatherRemotingSpec +class SunnyWeatherMultiJvmNode1 extends SunnyWeatherSpec +class SunnyWeatherMultiJvmNode2 extends SunnyWeatherSpec +class SunnyWeatherMultiJvmNode3 extends SunnyWeatherSpec +class SunnyWeatherMultiJvmNode4 extends SunnyWeatherSpec +class SunnyWeatherMultiJvmNode5 extends SunnyWeatherSpec -class SunnyWeatherArteryMultiJvmNode1 extends SunnyWeatherArterySpec -class SunnyWeatherArteryMultiJvmNode2 extends SunnyWeatherArterySpec -class SunnyWeatherArteryMultiJvmNode3 extends SunnyWeatherArterySpec -class SunnyWeatherArteryMultiJvmNode4 extends SunnyWeatherArterySpec -class SunnyWeatherArteryMultiJvmNode5 extends SunnyWeatherArterySpec - -abstract class SunnyWeatherRemotingSpec extends SunnyWeatherSpec(new SunnyWeatherMultiJvmSpec(artery = false)) -abstract class SunnyWeatherArterySpec extends SunnyWeatherSpec(new SunnyWeatherMultiJvmSpec(artery = true)) - -abstract class SunnyWeatherSpec(multiNodeConfig: SunnyWeatherMultiJvmSpec) - extends MultiNodeSpec(multiNodeConfig) +abstract class SunnyWeatherSpec extends MultiNodeSpec(SunnyWeatherMultiJvmSpec) with MultiNodeClusterSpec { - import multiNodeConfig._ + import SunnyWeatherMultiJvmSpec._ import ClusterEvent._ 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..09392afe35 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,21 @@ abstract class SurviveNetworkInstabilitySpec } runOn(side2: _*) { - val expected = ((side2 ++ side1) map address).toSet - clusterView.members.map(_.address) should ===(expected) - assertUnreachable(side1: _*) + if (RARP(system).provider.remoteSettings.Artery.Enabled) { + // with artery the other side stays quarantined + val expected = (side2 map address).toSet + clusterView.members.map(_.address) should ===(expected) + + } else { + // with the old remoting 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..a6a0aacb3a 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 { + 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-remote/src/main/scala/akka/remote/Remoting.scala b/akka-remote/src/main/scala/akka/remote/Remoting.scala index 35817c99d0..bd59bc4f7c 100644 --- a/akka-remote/src/main/scala/akka/remote/Remoting.scala +++ b/akka-remote/src/main/scala/akka/remote/Remoting.scala @@ -36,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 From 11fceb41213c5dba79f2b14f3fd0b1cbcfb5363b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Endre=20S=C3=A1ndor=20Varga?= Date: Fri, 2 Sep 2016 18:09:43 +0200 Subject: [PATCH 112/186] 21203: Cache/memoize ActorRef resolution --- .../akka/remote/artery/CodecBenchmark.scala | 4 +- .../akka/util/LruBoundedCacheBench.scala | 78 ++++++ .../akka/remote/artery/ArteryTransport.scala | 5 +- .../scala/akka/remote/artery/BufferPool.scala | 22 +- .../scala/akka/remote/artery/Codecs.scala | 38 ++- .../akka/remote/artery/LruBoundedCache.scala | 221 +++++++++++++++++ .../remote/artery/LruBoundedCacheSpec.scala | 225 ++++++++++++++++++ 7 files changed, 575 insertions(+), 18 deletions(-) create mode 100644 akka-bench-jmh/src/main/scala/akka/util/LruBoundedCacheBench.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/LruBoundedCache.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/LruBoundedCacheSpec.scala 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 index b2b9cf121a..76c52c01a5 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -167,7 +167,7 @@ class CodecBenchmark { val decoder: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = Flow.fromGraph(new Decoder(inboundContext, system.asInstanceOf[ExtendedActorSystem], - resolveActorRefWithLocalAddress, NoInboundCompressions, envelopePool, inboundEnvelopePool)) + uniqueLocalAddress, NoInboundCompressions, envelopePool, inboundEnvelopePool)) Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) .map { _ => @@ -208,7 +208,7 @@ class CodecBenchmark { val decoder: Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = Flow.fromGraph(new Decoder(inboundContext, system.asInstanceOf[ExtendedActorSystem], - resolveActorRefWithLocalAddress, NoInboundCompressions, envelopePool, inboundEnvelopePool)) + uniqueLocalAddress, NoInboundCompressions, envelopePool, inboundEnvelopePool)) Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) .map(msg ⇒ outboundEnvelopePool.acquire().init(OptionVal.None, payload, OptionVal.Some(remoteRefB))) 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..cc89de28d9 --- /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 + 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-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 41c7c74332..8ddb7e17a7 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -914,10 +914,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } def createDecoder(compression: InboundCompressions, bufferPool: EnvelopeBufferPool): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { - val resolveActorRefWithLocalAddress: String ⇒ InternalActorRef = { - recipient ⇒ provider.resolveActorRefWithLocalAddress(recipient, localAddress.address) - } - Flow.fromGraph(new Decoder(this, system, resolveActorRefWithLocalAddress, compression, bufferPool, + Flow.fromGraph(new Decoder(this, system, localAddress, compression, bufferPool, inboundEnvelopePool)) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index 6773dfdeb8..745417daa1 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -7,7 +7,7 @@ package akka.remote.artery import java.nio.charset.Charset import java.nio.{ ByteBuffer, ByteOrder } -import akka.actor.{ ActorRef, Address } +import akka.actor.{ ActorPath, ChildActorPath, ActorRef, Address } import akka.remote.artery.compress.CompressionProtocol._ import akka.remote.artery.compress.{ CompressionTable, InboundCompressions } import akka.serialization.Serialization @@ -147,6 +147,21 @@ private[remote] sealed trait HeaderBuilder { def manifest(originUid: Long): String } +/** + * 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 */ @@ -154,6 +169,9 @@ private[remote] final class HeaderBuilderImpl( inboundCompression: InboundCompressions, var _outboundActorRefCompression: CompressionTable[ActorRef], var _outboundClassManifestCompression: CompressionTable[String]) extends HeaderBuilder { + + private[this] val toSerializationFormat: SerializationFormatCache = new SerializationFormatCache + // Fields only available for EnvelopeBuffer var _version: Int = _ var _uid: Long = _ @@ -215,7 +233,7 @@ private[remote] final class HeaderBuilderImpl( def setRecipientActorRef(ref: ActorRef): Unit = { _recipientActorRefIdx = outboundActorRefCompression.compress(ref) if (_recipientActorRefIdx == -1) { - _recipientActorRef = ref.path.toSerializationFormat + _recipientActorRef = toSerializationFormat.getOrCompute(ref) } } def recipientActorRef(originUid: Long): OptionVal[ActorRef] = diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 04081b6ebe..b9ded54aed 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -6,7 +6,7 @@ package akka.remote.artery import scala.concurrent.duration._ import scala.util.control.NonFatal import akka.actor._ -import akka.remote.{ MessageSerializer, OversizedPayloadException, UniqueAddress } +import akka.remote.{ MessageSerializer, OversizedPayloadException, RemoteActorRefProvider, UniqueAddress } import akka.remote.artery.SystemMessageDelivery.SystemMessageEnvelope import akka.serialization.{ Serialization, SerializationExtension } import akka.stream._ @@ -16,10 +16,12 @@ 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 java.util.concurrent.atomic.AtomicInteger @@ -194,16 +196,30 @@ private[remote] object Decoder { private object Tick } +/** + * INTERNAL API + */ +private[akka] final class ActorRefResolveCache(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, - resolveActorRefWithLocalAddress: String ⇒ InternalActorRef, - compression: InboundCompressions, - bufferPool: EnvelopeBufferPool, - inEnvelopePool: ObjectPool[ReusableInboundEnvelope]) extends GraphStage[FlowShape[EnvelopeBuffer, InboundEnvelope]] { + 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") @@ -214,6 +230,8 @@ private[remote] class Decoder( import Decoder.RetryResolveRemoteDeployedRecipient private val localAddress = inboundContext.localAddress.address private val headerBuilder = HeaderBuilder.in(compression) + private val actorRefResolver: ActorRefResolveCache = + new ActorRefResolveCache(system.provider.asInstanceOf[RemoteActorRefProvider], uniqueLocalAddress) private val retryResolveRemoteDeployedRecipientInterval = 50.millis private val retryResolveRemoteDeployedRecipientAttempts = 20 @@ -260,7 +278,7 @@ private[remote] class Decoder( case OptionVal.Some(ref) ⇒ OptionVal(ref.asInstanceOf[InternalActorRef]) case OptionVal.None if headerBuilder.senderActorRefPath.isDefined ⇒ - OptionVal(resolveActorRefWithLocalAddress(headerBuilder.senderActorRefPath.get)) + OptionVal(actorRefResolver.getOrCompute(headerBuilder.senderActorRefPath.get)) case _ ⇒ OptionVal.None } @@ -315,7 +333,7 @@ private[remote] class Decoder( } private def resolveRecipient(path: String): OptionVal[InternalActorRef] = { - resolveActorRefWithLocalAddress(path) match { + actorRefResolver.getOrCompute(path) match { case empty: EmptyLocalActorRef ⇒ val pathElements = empty.path.elements // FIXME remote deployment corner case, please fix @patriknw (see also below, in onTimer) @@ -354,7 +372,7 @@ private[remote] class Decoder( attemptsLeft - 1, recipientPath, inboundEnvelope), retryResolveRemoteDeployedRecipientInterval) else { - val recipient = resolveActorRefWithLocalAddress(recipientPath) + val recipient = actorRefResolver.getOrCompute(recipientPath) // FIXME only retry for the first message, need to keep them in a cache push(out, inboundEnvelope.withRecipient(recipient)) } 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..40e382348f --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/LruBoundedCache.scala @@ -0,0 +1,221 @@ +/** + * Copyright (C) 2016 Lightbend Inc. + */ +package akka.remote.artery + +import akka.util.{ OptionVal, Unsafe } + +import scala.annotation.tailrec +import scala.reflect.ClassTag + +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/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 + } + + } + + } + +} From 8513ac9752b06cd0f4dfcc156272dda057dd0cd3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Endre=20S=C3=A1ndor=20Varga?= Date: Fri, 2 Sep 2016 13:50:36 +0200 Subject: [PATCH 113/186] #21206: Improve AFR resilience to write-after-close --- .../remote/artery/FlightRecorderBench.scala | 2 +- .../akka/remote/artery/ArteryTransport.scala | 4 +- .../akka/remote/artery/FlightRecorder.scala | 76 +++++++++++++------ 3 files changed, 54 insertions(+), 28 deletions(-) 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 index be6cdfcb67..9660405291 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/FlightRecorderBench.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/FlightRecorderBench.scala @@ -43,7 +43,7 @@ class FlightRecorderBench { @Benchmark @OperationsPerInvocation(10000000) - def flight_recorder_wrtites(): Unit = { + def flight_recorder_writes(): Unit = { val latch = new CountDownLatch(writers) (1 to writers).foreach { _ => val sink = recorder.createEventSink() diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index a07a1e12b6..cd9ca5f7eb 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -334,7 +334,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // times a factor (for reasonable number of outbound streams) private val outboundEnvelopePool = ReusableOutboundEnvelope.createObjectPool(capacity = 3072 * 2) - val (afrFileChannel, afrFlie, flightRecorder) = initializeFlightRecorder() match { + val (afrFileChannel, afrFile, flightRecorder) = initializeFlightRecorder() match { case None ⇒ (None, None, None) case Some((c, f, r)) ⇒ (Some(c), Some(f), Some(r)) } @@ -693,7 +693,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R afrFileChannel.foreach(_.force(true)) afrFileChannel.foreach(_.close()) // TODO: Be smarter about this in tests and make it always-on-for prod - afrFlie.foreach(_.delete()) + afrFile.foreach(_.delete()) Done } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala index f86248cb08..75e6f517ec 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala @@ -4,12 +4,12 @@ package akka.remote.artery import java.io.{ File, RandomAccessFile } -import java.nio.{ ByteBuffer, ByteOrder } import java.nio.channels.FileChannel import java.nio.file.StandardOpenOption -import java.util.concurrent.atomic.AtomicBoolean +import java.nio.{ ByteBuffer, ByteOrder } +import java.util.concurrent.atomic.AtomicReference +import java.util.concurrent.{ CountDownLatch, TimeUnit } -import akka.util.ByteString import org.agrona.BitUtil import org.agrona.concurrent.MappedResizeableBuffer @@ -232,7 +232,15 @@ private[remote] object FlightRecorder { /** * INTERNAL API */ -private[akka] class FlightRecorder(val fileChannel: FileChannel) extends AtomicBoolean { +private[akka] sealed trait FlightRecorderStatus +case object Running extends FlightRecorderStatus +case object ShutDown extends FlightRecorderStatus +final case class SnapshotInProgress(latch: CountDownLatch) extends FlightRecorderStatus + +/** + * INTERNAL API + */ +private[akka] class FlightRecorder(val fileChannel: FileChannel) extends AtomicReference[FlightRecorderStatus](Running) { import FlightRecorder._ private[this] val globalSection = new MappedResizeableBuffer(fileChannel, 0, GlobalSectionSize) @@ -274,7 +282,9 @@ private[akka] class FlightRecorder(val fileChannel: FileChannel) extends AtomicB // 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. ¯\_(ツ)_/¯ ") - if (!get && compareAndSet(false, true)) { + 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 @@ -291,13 +301,18 @@ private[akka] class FlightRecorder(val fileChannel: FileChannel) extends AtomicB loFreqLogs.markSnapshot(previousLog) alertLogs.markSnapshot(previousLog) fileChannel.force(true) - set(false) + 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() @@ -314,17 +329,22 @@ private[akka] class FlightRecorder(val fileChannel: FileChannel) extends AtomicB startHiFreqBatch() override def alert(code: Int, metadata: Array[Byte]): Unit = { - clock.updateWallClock() - prepareRichRecord(alertRecordBuffer, code, metadata) - alertLogs.write(currentLog, alertRecordBuffer) - flushHiFreqBatch() - snapshot() + 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 = { - clock.updateHighSpeedClock() - prepareRichRecord(loFreqRecordBuffer, code, metadata) - loFreqLogs.write(currentLog, loFreqRecordBuffer) + 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 = { @@ -344,12 +364,15 @@ private[akka] class FlightRecorder(val fileChannel: FileChannel) extends AtomicB // FIXME: Try to save as many bytes here as possible! We will see crazy throughput here override def hiFreq(code: Long, param: Long): Unit = { - hiFreqBatchedEntries += 1 - hiFreqBatchBuffer.putLong(code) - hiFreqBatchBuffer.putLong(param) + 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() + // If batch is full, time to flush + if (!hiFreqBatchBuffer.hasRemaining) flushHiFreqBatch() + } } private def startHiFreqBatch(): Unit = { @@ -368,12 +391,15 @@ private[akka] class FlightRecorder(val fileChannel: FileChannel) extends AtomicB } override def flushHiFreqBatch(): Unit = { - if (hiFreqBatchedEntries > 0) { - hiFreqBatchBuffer.putLong(HiFreqEntryCountFieldOffset, hiFreqBatchedEntries) - hiFreqBatchedEntries = 0 - hiFreqBatchBuffer.position(0) - hiFreqLogs.write(currentLog, hiFreqBatchBuffer) - startHiFreqBatch() + 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() + } } } From b127f1fdcf3af63865be0a042d60a9bd00363b76 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Thu, 8 Sep 2016 16:12:29 +0200 Subject: [PATCH 114/186] Rename protocol from "artery" to "akka" (#21405) --- .../artery/LiteralEncodingBenchmark.scala | 2 +- .../akka/remote/artery/ArteryTransport.scala | 2 +- .../remote/artery/HandshakeFailureSpec.scala | 2 +- .../remote/artery/HandshakeRetrySpec.scala | 2 +- .../artery/InboundControlJunctionSpec.scala | 4 +- .../remote/artery/InboundHandshakeSpec.scala | 4 +- .../akka/remote/artery/LateConnectSpec.scala | 2 +- .../artery/OutboundControlJunctionSpec.scala | 4 +- .../remote/artery/OutboundHandshakeSpec.scala | 4 +- .../remote/artery/RemoteActorForSpec.scala | 8 ++-- .../artery/RemoteActorRefProviderSpec.scala | 4 +- .../artery/RemoteActorSelectionSpec.scala | 4 +- .../remote/artery/RemoteConnectionSpec.scala | 6 +-- .../remote/artery/RemoteDeathWatchSpec.scala | 8 ++-- .../remote/artery/RemoteDeploymentSpec.scala | 4 +- .../RemoteMessageSerializationSpec.scala | 2 +- .../akka/remote/artery/RemoteRouterSpec.scala | 48 +++++++++---------- .../artery/SystemMessageAckerSpec.scala | 6 +-- .../ActorRefCompressionIntegrationSpec.scala | 4 +- ...ssManifestCompressionIntegrationSpec.scala | 4 +- ...dshakeShouldDropCompressionTableSpec.scala | 4 +- .../src/main/resources/application.conf | 4 +- 22 files changed, 66 insertions(+), 66 deletions(-) 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 index 105dee92a3..1cba3a4fa6 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/LiteralEncodingBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/LiteralEncodingBenchmark.scala @@ -18,7 +18,7 @@ import org.openjdk.jmh.annotations._ class LiteralEncodingBenchmark { private val UsAscii = Charset.forName("US-ASCII") - private val str = "artery://SomeSystem@host12:1234/user/foo" + 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) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 8ddb7e17a7..90efe937dd 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -988,7 +988,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R */ private[remote] object ArteryTransport { - val ProtocolName = "artery" + val ProtocolName = "akka" val Version = 0 diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala index bef3092935..9910d72653 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala @@ -40,7 +40,7 @@ class HandshakeFailureSpec extends AkkaSpec(HandshakeFailureSpec.commonConfig) w "Artery handshake" must { "allow for timeout and later connect" in { - def sel = system.actorSelection(s"artery://systemB@localhost:$portB/user/echo") + def sel = system.actorSelection(s"akka://systemB@localhost:$portB/user/echo") sel ! "hello" expectNoMsg(3.seconds) // longer than handshake-timeout diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala index 0edc30d3eb..fa003aa628 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala @@ -40,7 +40,7 @@ class HandshakeRetrySpec extends AkkaSpec(HandshakeRetrySpec.commonConfig) with "Artery handshake" must { "be retried during handshake-timeout (no message loss)" in { - def sel = system.actorSelection(s"artery://systemB@localhost:$portB/user/echo") + def sel = system.actorSelection(s"akka://systemB@localhost:$portB/user/echo") sel ! "hello" expectNoMsg(1.second) diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala index 203169817c..ab85391a90 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala @@ -31,8 +31,8 @@ class InboundControlJunctionSpec extends AkkaSpec with ImplicitSender { val matSettings = ActorMaterializerSettings(system).withFuzzing(true) implicit val mat = ActorMaterializer(matSettings)(system) - val addressA = UniqueAddress(Address("artery", "sysA", "hostA", 1001), 1) - val addressB = UniqueAddress(Address("artery", "sysB", "hostB", 1002), 2) + val addressA = UniqueAddress(Address("akka", "sysA", "hostA", 1001), 1) + val addressB = UniqueAddress(Address("akka", "sysB", "hostB", 1002), 2) "Control messages" must { diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala index ac7b62529b..befdc927e0 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala @@ -35,8 +35,8 @@ class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { val matSettings = ActorMaterializerSettings(system).withFuzzing(true) implicit val mat = ActorMaterializer(matSettings)(system) - val addressA = UniqueAddress(Address("artery", "sysA", "hostA", 1001), 1) - val addressB = UniqueAddress(Address("artery", "sysB", "hostB", 1002), 2) + 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 diff --git a/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala index 0a208b1712..edf74c2736 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala @@ -41,7 +41,7 @@ class LateConnectSpec extends AkkaSpec(LateConnectSpec.config) with ImplicitSend "be established after initial lazy restart" in { system.actorOf(TestActors.echoActorProps, "echoA") - val echoB = system.actorSelection(s"artery://systemB@localhost:$portB/user/echoB") + 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 diff --git a/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala index c4b5569a0e..58192b5099 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundControlJunctionSpec.scala @@ -30,8 +30,8 @@ class OutboundControlJunctionSpec extends AkkaSpec with ImplicitSender { val matSettings = ActorMaterializerSettings(system).withFuzzing(true) implicit val mat = ActorMaterializer(matSettings)(system) - val addressA = UniqueAddress(Address("artery", "sysA", "hostA", 1001), 1) - val addressB = UniqueAddress(Address("artery", "sysB", "hostB", 1002), 2) + 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) diff --git a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala index 6066d042fc..3b044e654c 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala @@ -28,8 +28,8 @@ class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { val matSettings = ActorMaterializerSettings(system).withFuzzing(true) implicit val mat = ActorMaterializer(matSettings)(system) - val addressA = UniqueAddress(Address("artery", "sysA", "hostA", 1001), 1) - val addressB = UniqueAddress(Address("artery", "sysB", "hostB", 1002), 2) + 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) diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorForSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorForSpec.scala index 69c4640c7c..4ddd4f4ea5 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorForSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorForSpec.scala @@ -25,7 +25,7 @@ class RemoteActorForSpec extends ArteryMultiNodeSpec("akka.loglevel=INFO") with "support remote look-ups" in { remoteSystem.actorOf(TestActors.echoActorProps, "remote-look-ups") - val remoteRef = localSystem.actorFor(s"artery://${remoteSystem.name}@localhost:$remotePort/user/remote-look-ups") + val remoteRef = localSystem.actorFor(s"akka://${remoteSystem.name}@localhost:$remotePort/user/remote-look-ups") remoteRef ! "ping" expectMsg("ping") } @@ -33,13 +33,13 @@ class RemoteActorForSpec extends ArteryMultiNodeSpec("akka.loglevel=INFO") with // 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("artery://nonexistingsystem@localhost:12346/user/echo") ! "ping" + localSystem.actorFor("akka://nonexistingsystem@localhost:12346/user/echo") ! "ping" } } "support ask" in { remoteSystem.actorOf(TestActors.echoActorProps, "support-ask") - val remoteRef = localSystem.actorFor(s"artery://${remoteSystem.name}@localhost:$remotePort/user/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") @@ -47,7 +47,7 @@ class RemoteActorForSpec extends ArteryMultiNodeSpec("akka.loglevel=INFO") with "send dead letters on remote if actor does not exist" in { EventFilter.warning(pattern = "dead.*buh", occurrences = 1).intercept { - localSystem.actorFor(s"artery://${remoteSystem.name}@localhost:$remotePort/dead-letters-on-remote") ! "buh" + localSystem.actorFor(s"akka://${remoteSystem.name}@localhost:$remotePort/dead-letters-on-remote") ! "buh" }(remoteSystem) } diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala index 5410f0e0d2..96f7cef146 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala @@ -19,12 +19,12 @@ class RemoteActorRefProviderSpec extends ArteryMultiNodeSpec { "RemoteActorRefProvider" must { "resolve local actor selection" in { - val sel = system.actorSelection(s"artery://${system.name}@${addressA.host.get}:${addressA.port.get}/user/echo") + 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"artery://${systemB.name}@${addressB.host.get}:${addressB.port.get}/user/echo") + 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) } diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala index 8be763ad6f..d8e0c1e115 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala @@ -48,8 +48,8 @@ class RemoteActorSelectionSpec extends ArteryMultiNodeSpec with ImplicitSender { akka { remote.artery.port = $port actor.deployment { - /looker2/child.remote = "artery://$remoteSysName@localhost:$remotePort" - /looker2/child/grandchild.remote = "artery://$localSysName@localhost:$localPort" + /looker2/child.remote = "akka://$remoteSysName@localhost:$remotePort" + /looker2/child/grandchild.remote = "akka://$localSysName@localhost:$localPort" } } """ diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteConnectionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteConnectionSpec.scala index 168e67ad0a..7ff1fad35c 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteConnectionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteConnectionSpec.scala @@ -29,7 +29,7 @@ class RemoteConnectionSpec extends ArteryMultiNodeSpec("akka.remote.retry-gate-c val remotePort = temporaryServerAddress(udp = true).getPort // try to talk to it before it is up - val selection = localSystem.actorSelection(s"artery://$nextGeneratedSystemName@localhost:$remotePort/user/echo") + val selection = localSystem.actorSelection(s"akka://$nextGeneratedSystemName@localhost:$remotePort/user/echo") selection.tell("ping", localProbe.ref) localProbe.expectNoMsg(1.seconds) @@ -60,7 +60,7 @@ class RemoteConnectionSpec extends ArteryMultiNodeSpec("akka.remote.retry-gate-c val remotePort = temporaryServerAddress(udp = true).getPort // try to talk to remote before it is up - val selection = localSystem.actorSelection(s"artery://$nextGeneratedSystemName@localhost:$remotePort/user/echo") + val selection = localSystem.actorSelection(s"akka://$nextGeneratedSystemName@localhost:$remotePort/user/echo") selection.tell("ping", localProbe.ref) localProbe.expectNoMsg(1.seconds) @@ -71,7 +71,7 @@ class RemoteConnectionSpec extends ArteryMultiNodeSpec("akka.remote.retry-gate-c localProbe.expectNoMsg(2.seconds) val otherProbe = new TestProbe(remoteSystem) val otherSender = otherProbe.ref - val thisSelection = remoteSystem.actorSelection(s"artery://${localSystem.name}@localhost:$localPort/user/echo") + val thisSelection = remoteSystem.actorSelection(s"akka://${localSystem.name}@localhost:$localPort/user/echo") within(5.seconds) { awaitAssert { thisSelection.tell("ping", otherSender) diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala index 8b8ec6a37e..4a7d049b0e 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala @@ -22,7 +22,7 @@ object RemoteDeathWatchSpec { actor { provider = remote deployment { - /watchers.remote = "artery://other@localhost:$otherPort" + /watchers.remote = "akka://other@localhost:$otherPort" } } remote.watch-failure-detector.acceptable-heartbeat-pause = 3s @@ -54,7 +54,7 @@ class RemoteDeathWatchSpec extends AkkaSpec(RemoteDeathWatchSpec.config) with Im // pick an unused port val port = SocketUtil.temporaryServerAddress("localhost", udp = true).getPort // simulate de-serialized ActorRef - val ref = rarp.resolveActorRef(s"artery://OtherSystem@localhost:$port/user/foo/bar#1752527294") + 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 { @@ -73,7 +73,7 @@ class RemoteDeathWatchSpec extends AkkaSpec(RemoteDeathWatchSpec.config) with Im } "receive Terminated when watched node is unknown host" in { - val path = RootActorPath(Address("artery", system.name, "unknownhost", 2552)) / "user" / "subject" + val path = RootActorPath(Address("akka", system.name, "unknownhost", 2552)) / "user" / "subject" system.actorOf(Props(new Actor { context.watch(context.actorFor(path)) def receive = { @@ -85,7 +85,7 @@ class RemoteDeathWatchSpec extends AkkaSpec(RemoteDeathWatchSpec.config) with Im } "receive ActorIdentity(None) when identified node is unknown host" in { - val path = RootActorPath(Address("artery", system.name, "unknownhost2", 2552)) / "user" / "subject" + 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/RemoteDeploymentSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala index c4bbf74cb4..7189141f3d 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala @@ -46,7 +46,7 @@ class RemoteDeploymentSpec extends AkkaSpec(""" val conf = ConfigFactory.parseString( s""" akka.actor.deployment { - /blub.remote = "artery://${system.name}@localhost:$port" + /blub.remote = "akka://${system.name}@localhost:$port" } """).withFallback(system.settings.config) @@ -62,7 +62,7 @@ class RemoteDeploymentSpec extends AkkaSpec(""" "create and supervise children on remote node" in { val senderProbe = TestProbe()(masterSystem) val r = masterSystem.actorOf(Props[Echo1], "blub") - r.path.toString should ===(s"artery://${system.name}@localhost:${port}/remote/artery/${masterSystem.name}@localhost:${masterPort}/user/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) diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala index 654a31221b..c84387077d 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteMessageSerializationSpec.scala @@ -92,7 +92,7 @@ class RemoteMessageSerializationSpec extends ArteryMultiNodeSpec(""" case x ⇒ sender() ! x } }), bigBounceId) - val bigBounceHere = localSystem.actorFor(s"artery://${remoteSystem.name}@localhost:$remotePort/user/$bigBounceId") + val bigBounceHere = localSystem.actorFor(s"akka://${remoteSystem.name}@localhost:$remotePort/user/$bigBounceId") val eventForwarder = localSystem.actorOf(Props(new Actor { def receive = { diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala index d55b842c53..00fbf374d6 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala @@ -52,7 +52,7 @@ class RemoteRouterSpec extends AkkaSpec(""" /blub { router = round-robin-pool nr-of-instances = 2 - target.nodes = ["artery://${sysName}@localhost:${port}"] + target.nodes = ["akka://${sysName}@localhost:${port}"] } /elastic-blub { router = round-robin-pool @@ -60,10 +60,10 @@ class RemoteRouterSpec extends AkkaSpec(""" lower-bound = 2 upper-bound = 3 } - target.nodes = ["artery://${sysName}@localhost:${port}"] + target.nodes = ["akka://${sysName}@localhost:${port}"] } /remote-blub { - remote = "artery://${sysName}@localhost:${port}" + remote = "akka://${sysName}@localhost:${port}" router = round-robin-pool nr-of-instances = 2 } @@ -71,12 +71,12 @@ class RemoteRouterSpec extends AkkaSpec(""" remote = "akka://MasterRemoteRouterSpec" router = round-robin-pool nr-of-instances = 2 - target.nodes = ["artery://${sysName}@localhost:${port}"] + target.nodes = ["akka://${sysName}@localhost:${port}"] } /local-blub2 { router = round-robin-pool nr-of-instances = 4 - target.nodes = ["artery://${sysName}@localhost:${port}"] + target.nodes = ["akka://${sysName}@localhost:${port}"] } } }""").withFallback(system.settings.config) @@ -105,7 +105,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"artery://${sysName}@localhost:${port}")) + children foreach (_.address.toString should ===(s"akka://${sysName}@localhost:${port}")) masterSystem.stop(router) } @@ -113,12 +113,12 @@ class RemoteRouterSpec extends AkkaSpec(""" val probe = TestProbe()(masterSystem) val router = masterSystem.actorOf(new RemoteRouterConfig( RoundRobinPool(2), - Seq(Address("artery", sysName, "localhost", port))).props(echoActorProps), "blub2") + 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"artery://${sysName}@localhost:${port}")) + children foreach (_.address.toString should ===(s"akka://${sysName}@localhost:${port}")) masterSystem.stop(router) } @@ -129,81 +129,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"artery://${sysName}@localhost:${port}")) + 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"artery://${sysName}@localhost:${port}") + 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"artery://${sysName}@localhost:${port}")) + 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"artery://${sysName}@localhost:${port}")))), "remote-blub2") - router.path.address.toString should ===(s"artery://${sysName}@localhost:${port}") + .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"artery://${sysName}@localhost:${port}")) + 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"artery://${sysName}@localhost:${port}")))), "local-blub") + .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("artery", sysName, "localhost", port)) - children foreach (_.address.toString should ===(s"artery://${sysName}@localhost:${port}")) + 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"artery://${sysName}@localhost:${port}")))), "local-blub2") - router.path.address.toString should ===(s"artery://${sysName}@localhost:${port}") + .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"artery://${sysName}@localhost:${port}")) + 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"artery://${sysName}@localhost:${port}")))), "remote-override") - router.path.address.toString should ===(s"artery://${sysName}@localhost:${port}") + .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"artery://${sysName}@localhost:${port}")) + children foreach (_.address.toString should ===(s"akka://${sysName}@localhost:${port}")) masterSystem.stop(router) } @@ -214,7 +214,7 @@ class RemoteRouterSpec extends AkkaSpec(""" } val router = masterSystem.actorOf(new RemoteRouterConfig( RoundRobinPool(1, supervisorStrategy = escalator), - Seq(Address("artery", sysName, "localhost", port))).props(Props.empty), "blub3") + Seq(Address("akka", 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/artery/SystemMessageAckerSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageAckerSpec.scala index 58585a2e6a..5ba36ba0ef 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageAckerSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageAckerSpec.scala @@ -25,9 +25,9 @@ class SystemMessageAckerSpec extends AkkaSpec with ImplicitSender { val matSettings = ActorMaterializerSettings(system).withFuzzing(true) implicit val mat = ActorMaterializer(matSettings)(system) - val addressA = UniqueAddress(Address("artery", "sysA", "hostA", 1001), 1) - val addressB = UniqueAddress(Address("artery", "sysB", "hostB", 1002), 2) - val addressC = UniqueAddress(Address("artery", "sysC", "hostB", 1003), 3) + 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 diff --git a/akka-remote/src/test/scala/akka/remote/artery/compress/ActorRefCompressionIntegrationSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/ActorRefCompressionIntegrationSpec.scala index 5a202d9bf1..f59005268a 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/ActorRefCompressionIntegrationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/ActorRefCompressionIntegrationSpec.scala @@ -33,7 +33,7 @@ class ActorRefCompressionIntegrationSpec extends AkkaSpec(CompressionIntegration system.eventStream.subscribe(aProbe.ref, classOf[CompressionProtocol.Events.ReceivedActorRefCompressionTable]) systemB.eventStream.subscribe(b1Probe.ref, classOf[CompressionProtocol.Events.ReceivedActorRefCompressionTable]) - def voidSel = system.actorSelection(s"artery://systemB@localhost:$portB/user/void") + def voidSel = system.actorSelection(s"akka://systemB@localhost:$portB/user/void") systemB.actorOf(TestActors.blackholeProps, "void") // cause testActor-1 to become a heavy hitter @@ -53,7 +53,7 @@ class ActorRefCompressionIntegrationSpec extends AkkaSpec(CompressionIntegration } def identify(_system: String, port: Int, name: String): ActorRef = { - val selection = system.actorSelection(s"artery://${_system}@localhost:$port/user/$name").resolveOne(3.seconds) + val selection = system.actorSelection(s"akka://${_system}@localhost:$port/user/$name").resolveOne(3.seconds) Await.result(selection, 4.seconds) } diff --git a/akka-remote/src/test/scala/akka/remote/artery/compress/ClassManifestCompressionIntegrationSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/ClassManifestCompressionIntegrationSpec.scala index 919ffdef6c..fbd5568d52 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/ClassManifestCompressionIntegrationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/ClassManifestCompressionIntegrationSpec.scala @@ -35,7 +35,7 @@ class ClassManifestCompressionIntegrationSpec extends AkkaSpec(CompressionIntegr systemB.actorOf(TestActors.blackholeProps, "void-2") Thread.sleep(1000) - val voidRef = Await.result(system.actorSelection(s"artery://systemB@localhost:$portB/user/void-2").resolveOne(3.second), 3.seconds) + val voidRef = Await.result(system.actorSelection(s"akka://systemB@localhost:$portB/user/void-2").resolveOne(3.second), 3.seconds) // cause testActor-1 to become a heavy hitter (1 to messagesToExchange).foreach { i ⇒ voidRef ! TestMessage("hello") } // does not reply, but a hot receiver should be advertised @@ -56,7 +56,7 @@ class ClassManifestCompressionIntegrationSpec extends AkkaSpec(CompressionIntegr def identify(_system: String, port: Int, name: String) = { val selection = - system.actorSelection(s"artery://${_system}@localhost:$port/user/$name") + system.actorSelection(s"akka://${_system}@localhost:$port/user/$name") val ActorIdentity(1, ref) = Await.result(selection ? Identify(1), 3.seconds) ref.get } 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 index f2eff03ea0..c70438300c 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala @@ -72,7 +72,7 @@ class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDr system.eventStream.subscribe(aProbe.ref, classOf[Event]) systemB.eventStream.subscribe(b1Probe.ref, classOf[Event]) - def echoSel = system.actorSelection(s"artery://systemB@localhost:$portB/user/echo") + def echoSel = system.actorSelection(s"akka://systemB@localhost:$portB/user/echo") systemB.actorOf(TestActors.echoActorProps, "echo") // cause testActor-1 to become a heavy hitter @@ -134,7 +134,7 @@ class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDr def identify(_system: String, port: Int, name: String) = { val selection = - system.actorSelection(s"artery://${_system}@localhost:$port/user/$name") + system.actorSelection(s"akka://${_system}@localhost:$port/user/$name") val ActorIdentity(1, ref) = Await.result(selection ? Identify(1), 3.seconds) ref.get } diff --git a/akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf b/akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf index 0d00e9b66d..9b2aabe28c 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf +++ b/akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf @@ -11,8 +11,8 @@ akka { cluster { seed-nodes = [ - "artery://ClusterSystem@127.0.0.1:2551", - "artery://ClusterSystem@127.0.0.1:2552"] + "akka://ClusterSystem@127.0.0.1:2551", + "akka://ClusterSystem@127.0.0.1:2552"] #//#snippet # excluded from snippet From 529b10a9f8b4c84f2f8b77d830d94b21ede8870b Mon Sep 17 00:00:00 2001 From: Johannes Rudolph Date: Thu, 8 Sep 2016 17:58:25 +0200 Subject: [PATCH 115/186] =rem #21202 a first set of protobuf serializations for artery messages (#21380) --- .../ClusterShardingMessageSerializer.scala | 2 - .../akka/remote/ArteryControlFormats.java | 6137 +++++++++++++++++ .../main/protobuf/ArteryControlFormats.proto | 75 + akka-remote/src/main/resources/reference.conf | 6 +- .../scala/akka/remote/artery/Control.scala | 11 +- .../scala/akka/remote/artery/Handshake.scala | 1 - .../remote/artery/SystemMessageDelivery.scala | 2 +- .../artery/compress/CompressionProtocol.scala | 12 +- .../artery/compress/CompressionTable.scala | 6 +- .../ArteryMessageSerializer.scala | 213 + .../artery/InboundControlJunctionSpec.scala | 15 +- .../ArteryMessageSerializerSpec.scala | 71 + 12 files changed, 6531 insertions(+), 20 deletions(-) create mode 100644 akka-remote/src/main/java/akka/remote/ArteryControlFormats.java create mode 100644 akka-remote/src/main/protobuf/ArteryControlFormats.proto create mode 100644 akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala create mode 100644 akka-remote/src/test/scala/akka/remote/serialization/ArteryMessageSerializerSpec.scala 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-remote/src/main/java/akka/remote/ArteryControlFormats.java b/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java new file mode 100644 index 0000000000..516c71356a --- /dev/null +++ b/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java @@ -0,0 +1,6137 @@ +// 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.HandshakeReq
+   * 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.HandshakeReq
+     * 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 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 uint32 tableVersion = 2; + /** + * required uint32 tableVersion = 2; + */ + boolean hasTableVersion(); + /** + * required uint32 tableVersion = 2; + */ + int getTableVersion(); + + // repeated string keys = 3; + /** + * repeated string keys = 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
+     * 
+ */ + java.util.List + getKeysList(); + /** + * repeated string keys = 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
+     * 
+ */ + int getKeysCount(); + /** + * repeated string keys = 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
+     * 
+ */ + java.lang.String getKeys(int index); + /** + * repeated string keys = 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
+     * 
+ */ + akka.protobuf.ByteString + getKeysBytes(int index); + + // repeated uint32 values = 4; + /** + * repeated uint32 values = 4; + */ + java.util.List getValuesList(); + /** + * repeated uint32 values = 4; + */ + int getValuesCount(); + /** + * repeated uint32 values = 4; + */ + 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; + tableVersion_ = input.readUInt32(); + break; + } + case 26: { + if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) { + keys_ = new akka.protobuf.LazyStringArrayList(); + mutable_bitField0_ |= 0x00000004; + } + keys_.add(input.readBytes()); + break; + } + case 32: { + if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) { + values_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000008; + } + values_.add(input.readUInt32()); + break; + } + case 34: { + int length = input.readRawVarint32(); + int limit = input.pushLimit(length); + if (!((mutable_bitField0_ & 0x00000008) == 0x00000008) && input.getBytesUntilLimit() > 0) { + values_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000008; + } + 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_ & 0x00000004) == 0x00000004)) { + keys_ = new akka.protobuf.UnmodifiableLazyStringList(keys_); + } + if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) { + 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 uint32 tableVersion = 2; + public static final int TABLEVERSION_FIELD_NUMBER = 2; + private int tableVersion_; + /** + * required uint32 tableVersion = 2; + */ + public boolean hasTableVersion() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required uint32 tableVersion = 2; + */ + public int getTableVersion() { + return tableVersion_; + } + + // repeated string keys = 3; + public static final int KEYS_FIELD_NUMBER = 3; + private akka.protobuf.LazyStringList keys_; + /** + * repeated string keys = 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
+     * 
+ */ + public java.util.List + getKeysList() { + return keys_; + } + /** + * repeated string keys = 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
+     * 
+ */ + public int getKeysCount() { + return keys_.size(); + } + /** + * repeated string keys = 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
+     * 
+ */ + public java.lang.String getKeys(int index) { + return keys_.get(index); + } + /** + * repeated string keys = 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
+     * 
+ */ + public akka.protobuf.ByteString + getKeysBytes(int index) { + return keys_.getByteString(index); + } + + // repeated uint32 values = 4; + public static final int VALUES_FIELD_NUMBER = 4; + private java.util.List values_; + /** + * repeated uint32 values = 4; + */ + public java.util.List + getValuesList() { + return values_; + } + /** + * repeated uint32 values = 4; + */ + public int getValuesCount() { + return values_.size(); + } + /** + * repeated uint32 values = 4; + */ + public int getValues(int index) { + return values_.get(index); + } + + private void initFields() { + from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance(); + 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 (!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.writeUInt32(2, tableVersion_); + } + for (int i = 0; i < keys_.size(); i++) { + output.writeBytes(3, keys_.getByteString(i)); + } + for (int i = 0; i < values_.size(); i++) { + output.writeUInt32(4, 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 + .computeUInt32Size(2, 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); + tableVersion_ = 0; + bitField0_ = (bitField0_ & ~0x00000002); + keys_ = akka.protobuf.LazyStringArrayList.EMPTY; + bitField0_ = (bitField0_ & ~0x00000004); + values_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000008); + 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.tableVersion_ = tableVersion_; + if (((bitField0_ & 0x00000004) == 0x00000004)) { + keys_ = new akka.protobuf.UnmodifiableLazyStringList( + keys_); + bitField0_ = (bitField0_ & ~0x00000004); + } + result.keys_ = keys_; + if (((bitField0_ & 0x00000008) == 0x00000008)) { + values_ = java.util.Collections.unmodifiableList(values_); + bitField0_ = (bitField0_ & ~0x00000008); + } + 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.hasTableVersion()) { + setTableVersion(other.getTableVersion()); + } + if (!other.keys_.isEmpty()) { + if (keys_.isEmpty()) { + keys_ = other.keys_; + bitField0_ = (bitField0_ & ~0x00000004); + } else { + ensureKeysIsMutable(); + keys_.addAll(other.keys_); + } + onChanged(); + } + if (!other.values_.isEmpty()) { + if (values_.isEmpty()) { + values_ = other.values_; + bitField0_ = (bitField0_ & ~0x00000008); + } else { + ensureValuesIsMutable(); + values_.addAll(other.values_); + } + onChanged(); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasFrom()) { + + 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 uint32 tableVersion = 2; + private int tableVersion_ ; + /** + * required uint32 tableVersion = 2; + */ + public boolean hasTableVersion() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required uint32 tableVersion = 2; + */ + public int getTableVersion() { + return tableVersion_; + } + /** + * required uint32 tableVersion = 2; + */ + public Builder setTableVersion(int value) { + bitField0_ |= 0x00000002; + tableVersion_ = value; + onChanged(); + return this; + } + /** + * required uint32 tableVersion = 2; + */ + public Builder clearTableVersion() { + bitField0_ = (bitField0_ & ~0x00000002); + tableVersion_ = 0; + onChanged(); + return this; + } + + // repeated string keys = 3; + private akka.protobuf.LazyStringList keys_ = akka.protobuf.LazyStringArrayList.EMPTY; + private void ensureKeysIsMutable() { + if (!((bitField0_ & 0x00000004) == 0x00000004)) { + keys_ = new akka.protobuf.LazyStringArrayList(keys_); + bitField0_ |= 0x00000004; + } + } + /** + * repeated string keys = 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
+       * 
+ */ + public java.util.List + getKeysList() { + return java.util.Collections.unmodifiableList(keys_); + } + /** + * repeated string keys = 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
+       * 
+ */ + public int getKeysCount() { + return keys_.size(); + } + /** + * repeated string keys = 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
+       * 
+ */ + public java.lang.String getKeys(int index) { + return keys_.get(index); + } + /** + * repeated string keys = 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
+       * 
+ */ + public akka.protobuf.ByteString + getKeysBytes(int index) { + return keys_.getByteString(index); + } + /** + * repeated string keys = 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
+       * 
+ */ + 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 = 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
+       * 
+ */ + public Builder addKeys( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + ensureKeysIsMutable(); + keys_.add(value); + onChanged(); + return this; + } + /** + * repeated string keys = 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
+       * 
+ */ + public Builder addAllKeys( + java.lang.Iterable values) { + ensureKeysIsMutable(); + super.addAll(values, keys_); + onChanged(); + return this; + } + /** + * repeated string keys = 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
+       * 
+ */ + public Builder clearKeys() { + keys_ = akka.protobuf.LazyStringArrayList.EMPTY; + bitField0_ = (bitField0_ & ~0x00000004); + onChanged(); + return this; + } + /** + * repeated string keys = 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
+       * 
+ */ + public Builder addKeysBytes( + akka.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + ensureKeysIsMutable(); + keys_.add(value); + onChanged(); + return this; + } + + // repeated uint32 values = 4; + private java.util.List values_ = java.util.Collections.emptyList(); + private void ensureValuesIsMutable() { + if (!((bitField0_ & 0x00000008) == 0x00000008)) { + values_ = new java.util.ArrayList(values_); + bitField0_ |= 0x00000008; + } + } + /** + * repeated uint32 values = 4; + */ + public java.util.List + getValuesList() { + return java.util.Collections.unmodifiableList(values_); + } + /** + * repeated uint32 values = 4; + */ + public int getValuesCount() { + return values_.size(); + } + /** + * repeated uint32 values = 4; + */ + public int getValues(int index) { + return values_.get(index); + } + /** + * repeated uint32 values = 4; + */ + public Builder setValues( + int index, int value) { + ensureValuesIsMutable(); + values_.set(index, value); + onChanged(); + return this; + } + /** + * repeated uint32 values = 4; + */ + public Builder addValues(int value) { + ensureValuesIsMutable(); + values_.add(value); + onChanged(); + return this; + } + /** + * repeated uint32 values = 4; + */ + public Builder addAllValues( + java.lang.Iterable values) { + ensureValuesIsMutable(); + super.addAll(values, values_); + onChanged(); + return this; + } + /** + * repeated uint32 values = 4; + */ + public Builder clearValues() { + values_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000008); + 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) + } + + 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_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; + + 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\"q\n\035" + + "CompressionTableAdvertisement\022\034\n\004from\030\001 " + + "\002(\0132\016.UniqueAddress\022\024\n\014tableVersion\030\002 \002(" + + "\r\022\014\n\004keys\030\003 \003(\t\022\016\n\006values\030\004 \003(\r\"Q\n Compr" + + "essionTableAdvertisementAck\022\034\n\004from\030\001 \002(" + + "\0132\016.UniqueAddress\022\017\n\007version\030\002 \002(\r\"\212\001\n\025S" + + "ystemMessageEnvelope\022\017\n\007message\030\001 \002(\014\022\024\n", + "\014serializerId\030\002 \002(\005\022\027\n\017messageManifest\030\003" + + " \001(\014\022\r\n\005seqNo\030\004 \002(\004\022\"\n\nackReplyTo\030\005 \002(\0132" + + "\016.UniqueAddress\"G\n\030SystemMessageDelivery" + + "Ack\022\r\n\005seqNo\030\001 \002(\004\022\034\n\004from\030\002 \002(\0132\016.Uniqu" + + "eAddress\"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\007address\030\001 \002(\0132" + + "\010.Address\022\013\n\003uid\030\002 \002(\004B\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_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_CompressionTableAdvertisement_descriptor = + getDescriptor().getMessageTypes().get(2); + internal_static_CompressionTableAdvertisement_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_CompressionTableAdvertisement_descriptor, + new java.lang.String[] { "From", "TableVersion", "Keys", "Values", }); + internal_static_CompressionTableAdvertisementAck_descriptor = + getDescriptor().getMessageTypes().get(3); + 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(4); + 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(5); + 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(6); + 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(7); + internal_static_UniqueAddress_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_UniqueAddress_descriptor, + new java.lang.String[] { "Address", "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/protobuf/ArteryControlFormats.proto b/akka-remote/src/main/protobuf/ArteryControlFormats.proto new file mode 100644 index 0000000000..5f47710670 --- /dev/null +++ b/akka-remote/src/main/protobuf/ArteryControlFormats.proto @@ -0,0 +1,75 @@ +/** + * 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.HandshakeReq +// OutboundHandshake.HandshakeRsp +message MessageWithAddress { + required UniqueAddress address = 1; +} + +// CompressionProtocol.ActorRefCompressionAdvertisement +// CompressionProtocol.ClassManifestCompressionAdvertisement +message CompressionTableAdvertisement { + required UniqueAddress from = 1; + required uint32 tableVersion = 2; + + // 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 = 3; + repeated uint32 values = 4; +} + +// 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; +} diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 99a1116e6a..2fd385e08c 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -15,6 +15,7 @@ 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" } @@ -28,11 +29,12 @@ akka { # 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. @@ -44,7 +46,6 @@ akka { # 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 - } serialization-identifiers { @@ -52,6 +53,7 @@ akka { "akka.remote.serialization.DaemonMsgCreateSerializer" = 3 "akka.remote.serialization.MessageContainerSerializer" = 6 "akka.remote.serialization.MiscMessageSerializer" = 16 + "akka.remote.serialization.ArteryMessageSerializer" = 17 } deployment { diff --git a/akka-remote/src/main/scala/akka/remote/artery/Control.scala b/akka-remote/src/main/scala/akka/remote/artery/Control.scala index ceac57e668..76aad28dc5 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Control.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Control.scala @@ -19,6 +19,9 @@ import akka.stream.stage.OutHandler import akka.remote.UniqueAddress import akka.util.OptionVal +/** INTERNAL API: marker trait for protobuf-serializable artery messages */ +private[akka] trait ArteryMessage extends Serializable + /** * INTERNAL API: Marker trait for reply messages */ @@ -29,22 +32,22 @@ private[akka] trait Reply extends ControlMessage * 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[akka] trait ControlMessage +private[akka] trait ControlMessage extends ArteryMessage /** * INTERNAL API */ -private[akka] final case class Quarantined(from: UniqueAddress, to: UniqueAddress) extends ControlMessage // FIXME serialization +private[akka] final case class Quarantined(from: UniqueAddress, to: UniqueAddress) extends ControlMessage /** * INTERNAL API */ -private[akka] case class ActorSystemTerminating(from: UniqueAddress) extends ControlMessage // FIXME serialization +private[akka] case class ActorSystemTerminating(from: UniqueAddress) extends ControlMessage /** * INTERNAL API */ -private[akka] case class ActorSystemTerminatingAck(from: UniqueAddress) // FIXME serialization +private[akka] case class ActorSystemTerminatingAck(from: UniqueAddress) extends ArteryMessage /** * INTERNAL API diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala index 5f4a50a3b4..63d8c05693 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -32,7 +32,6 @@ private[akka] object OutboundHandshake { */ class HandshakeTimeoutException(msg: String) extends RuntimeException(msg) with NoStackTrace - // FIXME serialization for these messages final case class HandshakeReq(from: UniqueAddress) extends ControlMessage final case class HandshakeRsp(from: UniqueAddress) extends Reply diff --git a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala index a78dc3d18b..c21f3bb2d9 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -30,7 +30,7 @@ import akka.actor.ActorRef private[akka] object SystemMessageDelivery { // FIXME serialization of these messages // FIXME ackReplyTo should not be needed - final case class SystemMessageEnvelope(message: AnyRef, seqNo: Long, ackReplyTo: UniqueAddress) + 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 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 index 6aeada84d5..21e928b8f8 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionProtocol.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionProtocol.scala @@ -5,7 +5,7 @@ package akka.remote.artery.compress import scala.language.existentials -import akka.actor.{ ActorRef, Address } +import akka.actor.ActorRef import akka.remote.UniqueAddress import akka.remote.artery.ControlMessage @@ -16,12 +16,18 @@ 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 ControlMessage with CompressionMessage + extends CompressionAdvertisement[ActorRef] /** * INTERNAL API @@ -38,7 +44,7 @@ object CompressionProtocol { * 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 ControlMessage with CompressionMessage + extends CompressionAdvertisement[String] /** * INTERNAL API 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 index 244d025c96..70bbdff2b3 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala @@ -8,7 +8,7 @@ import java.util import java.util.Comparator /** INTERNAL API: Versioned compression table to be advertised between systems */ -private[artery] final case class CompressionTable[T](version: Int, map: Map[T, Int]) { +private[remote] final case class CompressionTable[T](version: Int, map: Map[T, Int]) { import CompressionTable.NotCompressedId def compress(value: T): Int = @@ -20,7 +20,7 @@ private[artery] final case class CompressionTable[T](version: Int, map: Map[T, I def invert: DecompressionTable[T] = if (map.isEmpty) DecompressionTable.empty[T].copy(version = version) else { - // TODO: these are some expensive sanity checks, about the numbers being consequitive, without gaps + // 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((map.size * (map.size + 1)) / 2) /* Dirichlet */ require(map.values.min == 0, "Compression table should start allocating from 0, yet lowest allocated id was " + map.values.min) @@ -48,7 +48,7 @@ private[artery] final case class CompressionTable[T](version: Int, map: Map[T, I } } /** INTERNAL API */ -private[artery] object CompressionTable { +private[remote] object CompressionTable { final val NotCompressedId = -1 final val CompareBy2ndValue: Comparator[(Object, Int)] = new Comparator[(Object, Int)] { 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..98fe74a0ad --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala @@ -0,0 +1,213 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ +package akka.remote.serialization + +import akka.actor.{ ActorRef, Address, ExtendedActorSystem } +import akka.protobuf.MessageLite +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.{ ArteryControlFormats, MessageSerializer, UniqueAddress, WireFormats } +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" +} + +/** 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 _: 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) + case SystemMessageDelivery.Ack(seqNo, from) ⇒ serializeSystemMessageDeliveryAck(seqNo, from) + case HandshakeReq(from) ⇒ serializeWithAddress(from) + case HandshakeRsp(from) ⇒ serializeWithAddress(from) + case SystemMessageDelivery.Nack(seqNo, from) ⇒ serializeSystemMessageDeliveryAck(seqNo, from) + case q: Quarantined ⇒ serializeQuarantined(q) + case ActorSystemTerminating(from) ⇒ serializeWithAddress(from) + case ActorSystemTerminatingAck(from) ⇒ serializeWithAddress(from) + case adv: ActorRefCompressionAdvertisement ⇒ serializeActorRefCompressionAdvertisement(adv) + case ActorRefCompressionAdvertisementAck(from, id) ⇒ serializeCompressionTableAdvertisementAck(from, id) + case adv: ClassManifestCompressionAdvertisement ⇒ serializeCompressionAdvertisement(adv)(identity) + 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 ⇒ deserializeWithFromAddress(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 _ ⇒ 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 = Serialization.serializedActorPath(ref) + def deserializeActorRef(str: String): ActorRef = system.provider.resolveActorRef(str) + + def serializeActorRefCompressionAdvertisement(adv: ActorRefCompressionAdvertisement): ArteryControlFormats.CompressionTableAdvertisement = + // FIXME: is it guaranteed that idx 0 is reserved for DeadLetters? In the best case, this knowledge should be managed in only one place. + serializeCompressionAdvertisement(adv)(serializeActorRef, _ != 0 /* 0 is reserved for DeadLetters and doesn't need to be serialized explicitly */ ) + + def deserializeActorRefCompressionAdvertisement(bytes: Array[Byte]): ActorRefCompressionAdvertisement = + deserializeCompressionAdvertisement(bytes, deserializeActorRef, ActorRefCompressionAdvertisement, Seq(system.deadLetters → 0) /* add DeadLetters explicitly */ ) + + def serializeCompressionAdvertisement[T](adv: CompressionAdvertisement[T])(keySerializer: T ⇒ String, valueFilter: Int ⇒ Boolean = _ ⇒ true): ArteryControlFormats.CompressionTableAdvertisement = { + val builder = + ArteryControlFormats.CompressionTableAdvertisement.newBuilder + .setFrom(serializeUniqueAddress(adv.from)) + .setTableVersion(adv.table.version) + + adv.table.map.foreach { + case (key, value) if valueFilter(value) ⇒ + builder + .addKeys(keySerializer(key)) + .addValues(value) + case _ ⇒ + } + + builder.build + } + + def deserializeCompressionAdvertisement[T, U](bytes: Array[Byte], keyDeserializer: String ⇒ T, create: (UniqueAddress, CompressionTable[T]) ⇒ U, extraValues: Seq[(T, Int)] = Nil): 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 */ ) ++ + extraValues + + val table = CompressionTable(protoAdv.getTableVersion, 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, Int) ⇒ AnyRef): AnyRef = { + val msg = ArteryControlFormats.CompressionTableAdvertisementAck.parseFrom(bytes) + create(deserializeUniqueAddress(msg.getFrom), msg.getVersion) + } + + 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 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) +} \ No newline at end of file diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala index ab85391a90..44e3a26692 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala @@ -20,12 +20,19 @@ import akka.testkit.TestProbe import akka.util.OptionVal object InboundControlJunctionSpec { - case object Control1 extends ControlMessage - case object Control2 extends ControlMessage - case object Control3 extends ControlMessage + trait TestControlMessage extends ControlMessage + + case object Control1 extends TestControlMessage + case object Control2 extends TestControlMessage + case object Control3 extends TestControlMessage } -class InboundControlJunctionSpec extends AkkaSpec with ImplicitSender { +class InboundControlJunctionSpec + extends AkkaSpec(""" + akka.actor.serialization-bindings { + "akka.remote.artery.InboundControlJunctionSpec$TestControlMessage" = java + } + """) with ImplicitSender { import InboundControlJunctionSpec._ val matSettings = ActorMaterializerSettings(system).withFuzzing(true) 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..34bc13b1de --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/serialization/ArteryMessageSerializerSpec.scala @@ -0,0 +1,71 @@ +/** + * Copyright (C) 2009-2016 Lightbend Inc. + */ + +package akka.remote.serialization + +import akka.actor._ +import akka.remote.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()), + "HandshakeRsp" → HandshakeRsp(uniqueAddress()), + "ActorRefCompressionAdvertisement" → ActorRefCompressionAdvertisement(uniqueAddress(), CompressionTable(123, Map(actorA → 123, actorB → 456, system.deadLetters → 0))), + "ActorRefCompressionAdvertisementAck" → ActorRefCompressionAdvertisementAck(uniqueAddress(), 23), + "ClassManifestCompressionAdvertisement" → ClassManifestCompressionAdvertisement(uniqueAddress(), CompressionTable(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()) + ).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) + } +} + From ebd1883df5b1f44f1e95564020a669f69d919acf Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 5 Sep 2016 15:08:30 +0200 Subject: [PATCH 116/186] remove or reword obsolete fixme --- .../scala/akka/remote/artery/ArteryTransport.scala | 3 ++- .../main/scala/akka/remote/artery/Association.scala | 8 +++----- .../src/main/scala/akka/remote/artery/Codecs.scala | 4 ++-- .../scala/akka/remote/artery/FlightRecorder.scala | 4 ++-- .../main/scala/akka/remote/artery/SendQueue.scala | 1 + .../akka/remote/artery/SystemMessageDelivery.scala | 3 +-- .../remote/artery/compress/InboundCompressions.scala | 12 ++++-------- .../akka/remote/artery/RemoteDeathWatchSpec.scala | 2 -- .../HandshakeShouldDropCompressionTableSpec.scala | 1 - 9 files changed, 15 insertions(+), 23 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 90efe937dd..d5187a167a 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -708,7 +708,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R implicit val ec = materializer.executionContext updateStreamCompletion(streamName, streamCompleted.recover { case _ ⇒ Done }) streamCompleted.onFailure { - case ShutdownSignal ⇒ // shutdown as expected + 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) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 55c4d38335..eaa814fe65 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -48,6 +48,7 @@ import java.util.concurrent.atomic.AtomicBoolean import akka.stream.KillSwitches import scala.util.Failure import scala.util.Success +import akka.remote.artery.ArteryTransport.AeronTerminated /** * INTERNAL API @@ -280,7 +281,6 @@ private[remote] class Association( } // allow ActorSelectionMessage to pass through quarantine, to be able to establish interaction with new system - // FIXME where is that ActorSelectionMessage check in old remoting? if (message.isInstanceOf[ActorSelectionMessage] || !associationState.isQuarantined() || message == ClearSystemMessageDelivery) { message match { case _: SystemMessage | ClearSystemMessageDelivery | _: ControlMessage ⇒ @@ -388,7 +388,6 @@ private[remote] class Association( remoteAddress, reason) } case None ⇒ - // FIXME should we do something more, old impl used gating? log.warning("Quarantine of [{}] ignored because unknown UID", remoteAddress) } @@ -573,6 +572,7 @@ private[remote] class Association( updateStreamCompletion(streamName, streamCompleted.recover { case _ ⇒ Done }) streamCompleted.onFailure { case ArteryTransport.ShutdownSignal ⇒ // shutdown as expected + 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) @@ -593,10 +593,8 @@ private[remote] class Association( 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}", + log.error(cause, s"{} to {} failed and restarted {} times within {} seconds. Terminating system. ${cause.getMessage}", streamName, remoteAddress, transport.settings.Advanced.OutboundMaxRestarts, transport.settings.Advanced.OutboundRestartTimeout.toSeconds) - - // FIXME is this the right thing to do for outbound? transport.system.terminate() } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index b9ded54aed..e62d6de02e 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -313,7 +313,7 @@ private[remote] class Decoder( val decoded = inEnvelopePool.acquire().init( recipient, - localAddress, // FIXME: Is this needed anymore? What should we do here? + localAddress, // FIXME: this is used for the "non-local recipient" check in MessageDispatcher. Is this needed anymore? sender, originUid, headerBuilder.serializer, @@ -326,7 +326,7 @@ private[remote] class Decoder( // recipient for the first message that is sent to it, best effort retry scheduleOnce(RetryResolveRemoteDeployedRecipient( retryResolveRemoteDeployedRecipientAttempts, - headerBuilder.recipientActorRefPath.get, decoded), retryResolveRemoteDeployedRecipientInterval) // FIXME IS THIS SAFE? + headerBuilder.recipientActorRefPath.get, decoded), retryResolveRemoteDeployedRecipientInterval) } else push(out, decoded) } diff --git a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala index f86248cb08..9ace0f9315 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala @@ -329,7 +329,7 @@ private[akka] class FlightRecorder(val fileChannel: FileChannel) extends AtomicB private def prepareRichRecord(recordBuffer: ByteBuffer, code: Int, metadata: Array[Byte]): Unit = { recordBuffer.clear() - // FIXME: This is a bit overkill, needs some smarter scheme later, no need to always store the wallclock + // 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) @@ -342,7 +342,7 @@ private[akka] class FlightRecorder(val fileChannel: FileChannel) extends AtomicB recordBuffer.position(0) } - // FIXME: Try to save as many bytes here as possible! We will see crazy throughput here + // TODO: Try to save as many bytes here as possible! We will see crazy throughput here override def hiFreq(code: Long, param: Long): Unit = { hiFreqBatchedEntries += 1 hiFreqBatchBuffer.putLong(code) diff --git a/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala b/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala index 6710bb1ff7..73bcd0609b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala @@ -99,6 +99,7 @@ private[remote] final class SendQueue[T] extends GraphStageWithMaterializedValue } 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() diff --git a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala index c21f3bb2d9..3eaead00b7 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -29,7 +29,6 @@ import akka.actor.ActorRef */ private[akka] object SystemMessageDelivery { // FIXME serialization of these messages - // FIXME ackReplyTo should not be needed 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 @@ -80,7 +79,6 @@ private[akka] class SystemMessageDelivery( outboundContext.controlSubject.stopped.onComplete { getAsyncCallback[Try[Done]] { - // FIXME quarantine case Success(_) ⇒ completeStage() case Failure(cause) ⇒ failStage(cause) }.invoke @@ -88,6 +86,7 @@ private[akka] class SystemMessageDelivery( } 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) 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 index ddcfd0bda5..b55c051dba 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala @@ -42,7 +42,9 @@ private[remote] final class InboundCompressionsImpl( inboundContext: InboundContext, settings: ArterySettings.Compression) extends InboundCompressions { - // FIXME we also must remove the ones that won't be used anymore - when quarantine triggers + // TODO we also must remove the ones that won't be used anymore - when quarantine triggers? + // Why is that important? Won't be naturally be removed in new advertisements since they + // are not used any more? private[this] val _actorRefsIns = new Long2ObjectHashMap[InboundActorRefCompression]() private val createInboundActorRefsForOrigin = new LongFunction[InboundActorRefCompression] { override def apply(originUid: Long): InboundActorRefCompression = { @@ -200,12 +202,6 @@ private[remote] abstract class InboundCompression[T >: Null]( lazy val log = Logging(system, getClass.getSimpleName) - // FIXME NOTE: there exist edge cases around, we advertise table 1, accumulate table 2, the remote system has not used 2 yet, - // yet we technically could already prepare table 3, then it starts using table 1 suddenly. Edge cases like that. - // SOLUTION 1: We don't start building new tables until we've seen the previous one be used (move from new to active) - // This is nice as it practically disables all the "build the table" work when the other side is not interested in using it. - // SOLUTION 2: We end up dropping messages when old table comes in (we do that anyway) - private[this] val state: AtomicReference[InboundCompression.State[T]] = new AtomicReference(InboundCompression.State.empty) // TODO calibrate properly (h/w have direct relation to preciseness and max capacity) @@ -224,7 +220,7 @@ private[remote] abstract class InboundCompression[T >: Null]( * @throws UnknownCompressedIdException if given id is not known, this may indicate a bug – such situation should not happen. */ @tailrec final def decompressInternal(incomingTableVersion: Int, idx: Int, attemptCounter: Int): OptionVal[T] = { - // effectively should never loop more than once, to avoid infinite recursion blow up eagerly + // 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 diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala index 4a7d049b0e..c7da3eab69 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala @@ -26,8 +26,6 @@ object RemoteDeathWatchSpec { } } remote.watch-failure-detector.acceptable-heartbeat-pause = 3s - # FIXME do we need the initial-system-message-delivery-timeout? - remote.initial-system-message-delivery-timeout = 3 s remote.artery.enabled = on remote.artery.hostname = localhost remote.artery.port = 0 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 index c70438300c..55433ae0f1 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala @@ -59,7 +59,6 @@ class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDr } "Outgoing compression table" must { - // FIXME this is failing, we must rethink how tables are identified and updated "be dropped on system restart" in { val messagesToExchange = 10 val systemATransport = RARP(system).provider.transport.asInstanceOf[ArteryTransport] From 3c779cebd48f905d6ca096d2a264ae4da3b84c09 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 7 Sep 2016 10:41:36 +0200 Subject: [PATCH 117/186] config of send queues --- .../akka/remote/artery/CodecBenchmark.scala | 2 ++ akka-remote/src/main/resources/reference.conf | 26 +++++++++++++++++-- .../akka/remote/artery/ArterySettings.scala | 6 +++++ .../akka/remote/artery/ArteryTransport.scala | 6 ++++- .../akka/remote/artery/Association.scala | 25 +++++++++--------- .../scala/akka/remote/artery/Control.scala | 2 +- .../akka/remote/artery/TestContext.scala | 7 +++++ 7 files changed, 58 insertions(+), 16 deletions(-) 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 index 76c52c01a5..be9f3efd90 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -75,6 +75,8 @@ class CodecBenchmark { 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 = _ diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 2fd385e08c..bd266bc3f8 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -85,7 +85,6 @@ akka { remote { - ### FIXME: Temporary switch for the PoC artery { enabled = off port = 20200 @@ -166,10 +165,33 @@ akka { # 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 tainted. + # declared to be dead and its UID marked as quarantined. system-message-buffer-size = 20000 # unacknowledged system messages are re-delivered with this interval diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala index 365f71daf8..fa68f8c5e5 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala @@ -57,6 +57,12 @@ private[akka] final class ArterySettings private (config: Config) { 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 ⇒ diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index d5187a167a..d9f6bc05f1 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -104,6 +104,8 @@ private[akka] trait InboundContext { def completeHandshake(peer: UniqueAddress): Future[Done] + def settings: ArterySettings + } /** @@ -219,6 +221,8 @@ private[akka] trait OutboundContext { */ def controlSubject: ControlMessageSubject + def settings: ArterySettings + } /** @@ -372,7 +376,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R priorityMessageDestinations, outboundEnvelopePool)) - def settings = provider.remoteSettings.Artery + override def settings = provider.remoteSettings.Artery override def start(): Unit = { startMediaDriver() diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index eaa814fe65..288799e373 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -100,7 +100,10 @@ private[remote] class Association( private val log = Logging(transport.system, getClass.getName) - private val restartCounter = new RestartCounter(transport.settings.Advanced.OutboundMaxRestarts, transport.settings.Advanced.OutboundRestartTimeout) + 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 @@ -109,12 +112,10 @@ private[remote] class Association( def createQueue(capacity: Int): Queue[OutboundEnvelope] = new ManyToOneConcurrentArrayQueue[OutboundEnvelope](capacity) - private val outboundLanes = transport.settings.Advanced.OutboundLanes - private val controlQueueSize = transport.settings.Advanced.SysMsgBufferSize - // FIXME config queue size, and it should perhaps also be possible to use some kind of LinkedQueue - // such as agrona.ManyToOneConcurrentLinkedQueue or AbstractNodeQueue for less memory consumption - private val queueSize = 3072 - private val largeQueueSize = 256 + 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 @@ -429,7 +430,7 @@ private[remote] class Association( .toMat(transport.outboundControl(this))(Keep.both) .run()(materializer) - if (transport.settings.Advanced.TestMode) + if (advancedSettings.TestMode) _testStages.add(testMgmt) queueValue.inject(wrapper.queue) @@ -465,7 +466,7 @@ private[remote] class Association( .toMat(transport.outbound(this))(Keep.both) .run()(materializer) - if (transport.settings.Advanced.TestMode) + if (advancedSettings.TestMode) _testStages.add(testMgmt) queueValue.inject(wrapper.queue) @@ -509,7 +510,7 @@ private[remote] class Association( val (queueValues, testMgmtValues) = a.unzip val (changeCompressionValues, laneCompletedValues) = b.unzip - if (transport.settings.Advanced.TestMode) + if (advancedSettings.TestMode) testMgmtValues.foreach(_testStages.add) import transport.system.dispatcher @@ -545,7 +546,7 @@ private[remote] class Association( .toMat(transport.outboundLarge(this))(Keep.both) .run()(materializer) - if (transport.settings.Advanced.TestMode) + if (advancedSettings.TestMode) _testStages.add(testMgmt) queueValue.inject(wrapper.queue) @@ -594,7 +595,7 @@ private[remote] class Association( lazyRestart() } else { log.error(cause, s"{} to {} failed and restarted {} times within {} seconds. Terminating system. ${cause.getMessage}", - streamName, remoteAddress, transport.settings.Advanced.OutboundMaxRestarts, transport.settings.Advanced.OutboundRestartTimeout.toSeconds) + streamName, remoteAddress, advancedSettings.OutboundMaxRestarts, advancedSettings.OutboundRestartTimeout.toSeconds) transport.system.terminate() } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Control.scala b/akka-remote/src/main/scala/akka/remote/artery/Control.scala index 76aad28dc5..66d957244d 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Control.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Control.scala @@ -176,7 +176,7 @@ private[akka] class OutboundControlJunction( import OutboundControlJunction._ private val sendControlMessageCallback = getAsyncCallback[ControlMessage](internalSendControlMessage) - private val maxControlMessageBufferSize: Int = 1024 // FIXME config + private val maxControlMessageBufferSize: Int = outboundContext.settings.Advanced.OutboundControlQueueSize private val buffer = new ArrayDeque[OutboundEnvelope] override def preStart(): Unit = { diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala index 9a350984ef..db5965589d 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -19,6 +19,7 @@ import akka.remote.artery.InboundControlJunction.ControlMessageSubject import akka.util.OptionVal import akka.actor.InternalActorRef import akka.dispatch.ExecutionContexts +import com.typesafe.config.ConfigFactory private[remote] class TestInboundContext( override val localAddress: UniqueAddress, @@ -59,6 +60,9 @@ private[remote] class TestInboundContext( 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( @@ -94,6 +98,9 @@ private[remote] class TestOutboundContext( OptionVal.None)) } + override lazy val settings: ArterySettings = + ArterySettings(ConfigFactory.load().getConfig("akka.remote.artery")) + } private[remote] class TestControlMessageSubject extends ControlMessageSubject { From 8756ffd75ce3c2b1707dd6d0c06be2993c79a4b2 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 7 Sep 2016 15:38:13 +0200 Subject: [PATCH 118/186] handle Aeron Publication.CLOSED --- .../scala/akka/remote/artery/AeronSink.scala | 54 +++++++++++++------ .../akka/remote/artery/Association.scala | 1 + 2 files changed, 39 insertions(+), 16 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala index f93a6ab013..b90d412403 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala @@ -30,11 +30,13 @@ object AeronSink { final class GaveUpSendingException(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]) + giveUpAfter: Duration, onGiveUp: AsyncCallback[Unit], onPublicationClosed: AsyncCallback[Unit]) extends (() ⇒ Boolean) { val giveUpAfterNanos = giveUpAfter match { case f: FiniteDuration ⇒ f.toNanos @@ -54,6 +56,9 @@ object AeronSink { 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 @@ -99,7 +104,7 @@ class AeronSink( private var backoffCount = spinning private var lastMsgSize = 0 private val offerTask = new OfferTask(pub, null, lastMsgSize, getAsyncCallback(_ ⇒ taskOnOfferSuccess()), - giveUpSendAfter, getAsyncCallback(_ ⇒ onGiveUp())) + giveUpSendAfter, getAsyncCallback(_ ⇒ onGiveUp()), getAsyncCallback(_ ⇒ onPublicationClosed())) private val addOfferTask: Add = Add(offerTask) private var offerTaskInProgress = false @@ -135,21 +140,18 @@ class AeronSink( @tailrec private def publish(): Unit = { val result = pub.offer(envelopeInFlight.aeronBuffer, 0, lastMsgSize) - // FIXME handle Publication.CLOSED if (result < 0) { - backoffCount -= 1 - if (backoffCount > 0) { - ThreadHints.onSpinWait() - publish() // recursive - } else { - // 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) + 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 @@ -157,6 +159,17 @@ class AeronSink( } } + 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) @@ -184,6 +197,15 @@ class AeronSink( 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) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 288799e373..17b9d2af7b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -267,6 +267,7 @@ private[remote] class Association( outboundControlIngress.sendControlMessage(message) def send(message: Any, sender: OptionVal[ActorRef], recipient: OptionVal[RemoteActorRef]): Unit = { + def createOutboundEnvelope(): OutboundEnvelope = outboundEnvelopePool.acquire().init(recipient, message.asInstanceOf[AnyRef], sender) From 9a7d79c88276cf98aa494e1ae42e9336ad34c68e Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 7 Sep 2016 15:43:08 +0200 Subject: [PATCH 119/186] size of outboundEnvelopePool --- .../akka/remote/artery/ArteryTransport.scala | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index d9f6bc05f1..b257f38da0 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -128,9 +128,9 @@ private[akka] object AssociationState { * INTERNAL API */ private[akka] final class AssociationState( - val incarnation: Int, + val incarnation: Int, val uniqueRemoteAddressPromise: Promise[UniqueAddress], - val quarantined: ImmutableLongMap[AssociationState.QuarantinedTimestamp]) { + val quarantined: ImmutableLongMap[AssociationState.QuarantinedTimestamp]) { import AssociationState.QuarantinedTimestamp @@ -230,7 +230,7 @@ private[akka] trait OutboundContext { */ private[remote] object FlushOnShutdown { def props(done: Promise[Done], timeout: FiniteDuration, - inboundContext: InboundContext, associations: Set[Association]): Props = { + inboundContext: InboundContext, associations: Set[Association]): Props = { require(associations.nonEmpty) Props(new FlushOnShutdown(done, timeout, inboundContext, associations)) } @@ -242,7 +242,7 @@ private[remote] object FlushOnShutdown { * INTERNAL API */ private[remote] class FlushOnShutdown(done: Promise[Done], timeout: FiniteDuration, - inboundContext: InboundContext, associations: Set[Association]) extends Actor { + inboundContext: InboundContext, associations: Set[Association]) extends Actor { var remaining = associations.flatMap(_.associationState.uniqueRemoteAddressValue) @@ -343,9 +343,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val largeEnvelopeBufferPool = new EnvelopeBufferPool(settings.Advanced.MaximumLargeFrameSize, settings.Advanced.MaximumPooledBuffers) private val inboundEnvelopePool = ReusableInboundEnvelope.createObjectPool(capacity = 16) - // FIXME capacity of outboundEnvelopePool should probably be derived from the sendQueue capacity - // times a factor (for reasonable number of outbound streams) - private val outboundEnvelopePool = ReusableOutboundEnvelope.createObjectPool(capacity = 3072 * 2) + // The outboundEnvelopePool is shared among all outbound associations + private val outboundEnvelopePool = ReusableOutboundEnvelope.createObjectPool(capacity = + settings.Advanced.OutboundMessageQueueSize * settings.Advanced.OutboundLanes * 3) val (afrFileChannel, afrFlie, flightRecorder) = initializeFlightRecorder() match { case None ⇒ (None, None, None) @@ -853,7 +853,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R createOutboundSink(ordinaryStreamId, outboundContext, envelopeBufferPool) private def createOutboundSink(streamId: Int, outboundContext: OutboundContext, - bufferPool: EnvelopeBufferPool): Sink[OutboundEnvelope, (ChangeOutboundCompression, Future[Done])] = { + bufferPool: EnvelopeBufferPool): Sink[OutboundEnvelope, (ChangeOutboundCompression, Future[Done])] = { outboundLane(outboundContext, bufferPool) .toMat(aeronSink(outboundContext, streamId))(Keep.both) @@ -872,7 +872,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private def outboundLane( outboundContext: OutboundContext, - bufferPool: EnvelopeBufferPool): Flow[OutboundEnvelope, EnvelopeBuffer, ChangeOutboundCompression] = { + bufferPool: EnvelopeBufferPool): Flow[OutboundEnvelope, EnvelopeBuffer, ChangeOutboundCompression] = { Flow.fromGraph(killSwitch.flow[OutboundEnvelope]) .via(new OutboundHandshake(system, outboundContext, outboundEnvelopePool, settings.Advanced.HandshakeTimeout, From 3b7a7dfa59eb10351f5fd1cdfeb07486ee1f455c Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 7 Sep 2016 16:07:29 +0200 Subject: [PATCH 120/186] add reason param to quarantine method --- .../main/scala/akka/cluster/ClusterRemoteWatcher.scala | 2 +- .../akka/remote/PiercingShouldKeepQuarantineSpec.scala | 2 +- .../scala/akka/remote/RemoteQuarantinePiercingSpec.scala | 2 +- .../akka/remote/RemoteRestartedQuarantinedSpec.scala | 2 +- .../remote/artery/RemoteRestartedQuarantinedSpec.scala | 2 +- .../main/scala/akka/remote/RemoteActorRefProvider.scala | 3 ++- .../src/main/scala/akka/remote/RemoteTransport.scala | 2 +- .../src/main/scala/akka/remote/RemoteWatcher.scala | 8 ++++---- akka-remote/src/main/scala/akka/remote/Remoting.scala | 2 +- .../main/scala/akka/remote/artery/ArteryTransport.scala | 8 ++++---- .../src/main/scala/akka/remote/artery/Control.scala | 4 ++-- .../src/test/scala/akka/remote/ActorsLeakSpec.scala | 2 +- .../src/test/scala/akka/remote/RemoteWatcherSpec.scala | 2 +- akka-remote/src/test/scala/akka/remote/RemotingSpec.scala | 4 ++-- .../test/scala/akka/remote/artery/RemoteWatcherSpec.scala | 2 +- project/MiMa.scala | 4 ++++ 16 files changed, 28 insertions(+), 23 deletions(-) diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala index 8fb729930a..454b9ba81a 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala @@ -90,7 +90,7 @@ private[cluster] class ClusterRemoteWatcher( 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), "Cluster member removed") } publishAddressTerminated(m.address) } 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 2166bdac0d..7a337fa57c 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 @@ -58,7 +58,7 @@ abstract class PiercingShouldKeepQuarantineSpec(multiNodeConfig: PiercingShouldK 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/RemoteQuarantinePiercingSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemoteQuarantinePiercingSpec.scala index 5fa058abda..ce0a22005b 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 @@ -78,7 +78,7 @@ abstract class RemoteQuarantinePiercingSpec(multiNodeConfig: RemoteQuarantinePie 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" 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..bf66b9795a 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 @@ -80,7 +80,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/artery/RemoteRestartedQuarantinedSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/RemoteRestartedQuarantinedSpec.scala index 8712d7cb03..1cdcc74fc1 100644 --- 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 @@ -73,7 +73,7 @@ abstract class RemoteRestartedQuarantinedSpec val (uid, ref) = identifyWithUid(second, "subject", 5.seconds) enterBarrier("before-quarantined") - 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/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 46b195cd4e..1aa85416f6 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -449,7 +449,8 @@ private[akka] class RemoteActorRefProvider( * @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[Int], reason: String): Unit = + transport.quarantine(address, uid, reason) } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala index c865aa508d..c8abdabcba 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala @@ -90,6 +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 + def quarantine(address: Address, uid: Option[Int], 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..7681eaf8a3 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteWatcher.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteWatcher.scala @@ -158,7 +158,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 +167,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[Int], reason: String): Unit = + remoteProvider.quarantine(address, uid, reason) def addWatch(watchee: InternalActorRef, watcher: InternalActorRef): Unit = { assert(watcher != self) @@ -282,4 +282,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 bd59bc4f7c..21597bf459 100644 --- a/akka-remote/src/main/scala/akka/remote/Remoting.scala +++ b/akka-remote/src/main/scala/akka/remote/Remoting.scala @@ -225,7 +225,7 @@ 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 { + override def quarantine(remoteAddress: Address, uid: Option[Int], reason: String): Unit = endpointManager match { case Some(manager) ⇒ manager ! Quarantine(remoteAddress, uid) case _ ⇒ throw new RemoteTransportExceptionNoStackTrace( s"Attempted to quarantine address [$remoteAddress] with uid [$uid] but Remoting is not running", null) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index b257f38da0..25aefe20c3 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -543,7 +543,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private def startAeronErrorLog(): Unit = { val errorLog = new AeronErrorLog(new File(aeronDir, CncFileDescriptor.CNC_FILE)) val lastTimestamp = new AtomicLong(0L) - import system.dispatcher // FIXME perhaps use another dispatcher for this + import system.dispatcher aeronErrorLogTask = system.scheduler.schedule(3.seconds, 5.seconds) { if (!isShutdown) { val newLastTimestamp = errorLog.logErrors(log, lastTimestamp.get) @@ -840,9 +840,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private def publishLifecycleEvent(event: RemotingLifecycleEvent): Unit = eventPublisher.notifyListeners(event) - override def quarantine(remoteAddress: Address, uid: Option[Int]): Unit = { - // FIXME change the method signature (old remoting) to include reason and use Long uid? - association(remoteAddress).quarantine(reason = "", uid.map(_.toLong)) + override def quarantine(remoteAddress: Address, uid: Option[Int], reason: String): Unit = { + // FIXME use Long uid + association(remoteAddress).quarantine(reason, uid.map(_.toLong)) } def outboundLarge(outboundContext: OutboundContext): Sink[OutboundEnvelope, Future[Done]] = diff --git a/akka-remote/src/main/scala/akka/remote/artery/Control.scala b/akka-remote/src/main/scala/akka/remote/artery/Control.scala index 66d957244d..d28df42a05 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Control.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Control.scala @@ -60,13 +60,13 @@ private[akka] object InboundControlJunction { * subject to get notification of incoming control * messages. */ - private[akka] trait ControlMessageSubject { + private[remote] trait ControlMessageSubject { def attach(observer: ControlMessageObserver): Future[Done] def detach(observer: ControlMessageObserver): Unit def stopped: Future[Done] } - private[akka] trait ControlMessageObserver { + private[remote] trait ControlMessageObserver { /** * Notification of incoming control message. The message diff --git a/akka-remote/src/test/scala/akka/remote/ActorsLeakSpec.scala b/akka-remote/src/test/scala/akka/remote/ActorsLeakSpec.scala index 409891d001..7012986e2d 100644 --- a/akka-remote/src/test/scala/akka/remote/ActorsLeakSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/ActorsLeakSpec.scala @@ -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/RemoteWatcherSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteWatcherSpec.scala index 59f23d4398..56ff462e8e 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteWatcherSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteWatcherSpec.scala @@ -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[Int], reason: String): Unit = { // don't quarantine in remoting, but publish a testable message context.system.eventStream.publish(TestRemoteWatcher.Quarantined(address, uid)) } diff --git a/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala b/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala index e08a1a0267..701cc5c670 100644 --- a/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala @@ -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), "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/artery/RemoteWatcherSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala index ede7c7f62e..3267448214 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala @@ -54,7 +54,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[Int], reason: String): Unit = { // don't quarantine in remoting, but publish a testable message context.system.eventStream.publish(TestRemoteWatcher.Quarantined(address, uid)) } diff --git a/project/MiMa.scala b/project/MiMa.scala index 13351fc3f3..3c293e9b4f 100644 --- a/project/MiMa.scala +++ b/project/MiMa.scala @@ -966,6 +966,10 @@ object MiMa extends AutoPlugin { ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOps.orElseGraph"), ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOps.orElse"), ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOpsMat.orElseMat") + ), + "2.4.10" -> Seq( + ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.RemoteActorRefProvider.quarantine"), + ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.RemoteWatcher.quarantine") ) ) } From 74a8bb3a006e0a88a27ee39dbe179ad0f82f4dbf Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 7 Sep 2016 17:50:17 +0200 Subject: [PATCH 121/186] flight recorder event for send queue overflow --- .../akka/remote/artery/Association.scala | 24 ++++++------ .../remote/artery/FlightRecorderEvents.scala | 37 ++++++++++--------- 2 files changed, 32 insertions(+), 29 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 17b9d2af7b..1592a85abc 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -97,8 +97,10 @@ private[remote] class Association( 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 @@ -274,11 +276,11 @@ private[remote] class Association( // volatile read to see latest queue array val unused = queuesVisibility - def dropped(qSize: Int, env: OutboundEnvelope): Unit = { + def dropped(queueIndex: Int, qSize: Int, env: OutboundEnvelope): Unit = { log.debug( "Dropping message [{}] from [{}] to [{}] due to overflow of send queue, size [{}]", message.getClass, sender.getOrElse(deadletters), recipient.getOrElse(recipient), qSize) - // FIXME AFR + flightRecorder.hiFreq(Transport_SendQueueOverflow, queueIndex) deadletters ! env } @@ -289,7 +291,7 @@ private[remote] class Association( val outboundEnvelope = createOutboundEnvelope() if (!controlQueue.offer(createOutboundEnvelope())) { quarantine(reason = s"Due to overflow of control queue, size [$controlQueueSize]") - dropped(controlQueueSize, outboundEnvelope) + dropped(ControlQueueIndex, controlQueueSize, outboundEnvelope) } case _: DaemonMsgCreate ⇒ // DaemonMsgCreate is not a SystemMessage, but must be sent over the control stream because @@ -298,18 +300,19 @@ private[remote] class Association( // destination) before the first ordinary message arrives. val outboundEnvelope1 = createOutboundEnvelope() if (!controlQueue.offer(outboundEnvelope1)) - dropped(controlQueueSize, outboundEnvelope1) + dropped(ControlQueueIndex, controlQueueSize, outboundEnvelope1) (0 until outboundLanes).foreach { i ⇒ val outboundEnvelope2 = createOutboundEnvelope() if (!queues(OrdinaryQueueIndex + i).offer(outboundEnvelope2)) - dropped(queueSize, outboundEnvelope2) + dropped(OrdinaryQueueIndex + i, queueSize, outboundEnvelope2) } case _ ⇒ val outboundEnvelope = createOutboundEnvelope() - val queue = selectQueue(recipient) + val queueIndex = selectQueue(recipient) + val queue = queues(queueIndex) val offerOk = queue.offer(outboundEnvelope) if (!offerOk) - dropped(queueSize, outboundEnvelope) + dropped(queueIndex, queueSize, outboundEnvelope) } } else if (log.isDebugEnabled) @@ -318,10 +321,10 @@ private[remote] class Association( message.getClass, sender.getOrElse(deadletters), recipient.getOrElse(recipient), remoteAddress) } - private def selectQueue(recipient: OptionVal[RemoteActorRef]): ProducerApi[OutboundEnvelope] = { + private def selectQueue(recipient: OptionVal[RemoteActorRef]): Int = { recipient match { case OptionVal.Some(r) ⇒ - val queueIndex = r.cachedSendQueueIndex match { + r.cachedSendQueueIndex match { case -1 ⇒ // only happens when messages are sent to new remote destination // and is then cached on the RemoteActorRef @@ -343,10 +346,9 @@ private[remote] class Association( idx case idx ⇒ idx } - queues(queueIndex) case OptionVal.None ⇒ - queues(OrdinaryQueueIndex) + OrdinaryQueueIndex } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala index 4d1497025c..e800e5493f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala @@ -18,29 +18,30 @@ object FlightRecorderEvents { val Transport_AeronErrorLogTaskStopped = 10 val Transport_MediaFileDeleted = 11 val Transport_FlightRecorderClose = 12 + val Transport_SendQueueOverflow = 13 // Aeron Sink events - val AeronSink_Started = 13 - val AeronSink_TaskRunnerRemoved = 14 - val AeronSink_PublicationClosed = 15 - val AeronSink_Stopped = 16 - val AeronSink_EnvelopeGrabbed = 17 - val AeronSink_EnvelopeOffered = 18 - val AeronSink_GaveUpEnvelope = 19 - val AeronSink_DelegateToTaskRunner = 20 - val AeronSink_ReturnFromTaskRunner = 21 + 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 = 22 - val AeronSource_Stopped = 23 - val AeronSource_Received = 24 - val AeronSource_DelegateToTaskRunner = 25 - val AeronSource_ReturnFromTaskRunner = 26 + val AeronSource_Started = 70 + val AeronSource_Stopped = 71 + val AeronSource_Received = 72 + val AeronSource_DelegateToTaskRunner = 72 + val AeronSource_ReturnFromTaskRunner = 73 // Compression events - val Compression_CompressedActorRef = 25 - val Compression_AllocatedActorRefCompressionId = 26 - val Compression_CompressedManifest = 27 - val Compression_AllocatedManifestCompressionId = 28 + val Compression_CompressedActorRef = 90 + val Compression_AllocatedActorRefCompressionId = 91 + val Compression_CompressedManifest = 91 + val Compression_AllocatedManifestCompressionId = 92 } From c74ece957395d7931a2860d1040adbd81ea7cb8b Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 7 Sep 2016 21:19:27 +0200 Subject: [PATCH 122/186] require power of two --- .../scala/akka/remote/artery/ArteryTransport.scala | 12 ++++++------ .../scala/akka/remote/artery/FlightRecorder.scala | 6 ++++-- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 25aefe20c3..83e7c81ea9 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -128,9 +128,9 @@ private[akka] object AssociationState { * INTERNAL API */ private[akka] final class AssociationState( - val incarnation: Int, + val incarnation: Int, val uniqueRemoteAddressPromise: Promise[UniqueAddress], - val quarantined: ImmutableLongMap[AssociationState.QuarantinedTimestamp]) { + val quarantined: ImmutableLongMap[AssociationState.QuarantinedTimestamp]) { import AssociationState.QuarantinedTimestamp @@ -230,7 +230,7 @@ private[akka] trait OutboundContext { */ private[remote] object FlushOnShutdown { def props(done: Promise[Done], timeout: FiniteDuration, - inboundContext: InboundContext, associations: Set[Association]): Props = { + inboundContext: InboundContext, associations: Set[Association]): Props = { require(associations.nonEmpty) Props(new FlushOnShutdown(done, timeout, inboundContext, associations)) } @@ -242,7 +242,7 @@ private[remote] object FlushOnShutdown { * INTERNAL API */ private[remote] class FlushOnShutdown(done: Promise[Done], timeout: FiniteDuration, - inboundContext: InboundContext, associations: Set[Association]) extends Actor { + inboundContext: InboundContext, associations: Set[Association]) extends Actor { var remaining = associations.flatMap(_.associationState.uniqueRemoteAddressValue) @@ -853,7 +853,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R createOutboundSink(ordinaryStreamId, outboundContext, envelopeBufferPool) private def createOutboundSink(streamId: Int, outboundContext: OutboundContext, - bufferPool: EnvelopeBufferPool): Sink[OutboundEnvelope, (ChangeOutboundCompression, Future[Done])] = { + bufferPool: EnvelopeBufferPool): Sink[OutboundEnvelope, (ChangeOutboundCompression, Future[Done])] = { outboundLane(outboundContext, bufferPool) .toMat(aeronSink(outboundContext, streamId))(Keep.both) @@ -872,7 +872,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private def outboundLane( outboundContext: OutboundContext, - bufferPool: EnvelopeBufferPool): Flow[OutboundEnvelope, EnvelopeBuffer, ChangeOutboundCompression] = { + bufferPool: EnvelopeBufferPool): Flow[OutboundEnvelope, EnvelopeBuffer, ChangeOutboundCompression] = { Flow.fromGraph(killSwitch.flow[OutboundEnvelope]) .via(new OutboundHandshake(system, outboundContext, outboundEnvelopePool, settings.Advanced.HandshakeTimeout, diff --git a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala index 9ace0f9315..4d3353beb0 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala @@ -132,7 +132,8 @@ private[remote] class RollingEventLogSection( recordSize: Int) { import RollingEventLogSection._ - // FIXME: check if power of two + 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 ⇒ @@ -237,7 +238,8 @@ private[akka] class FlightRecorder(val fileChannel: FileChannel) extends AtomicB private[this] val globalSection = new MappedResizeableBuffer(fileChannel, 0, GlobalSectionSize) - // FIXME: check if power of two + 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( From fea18a9a1bd43657e5f5101a2e13a264a6a441eb Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 8 Sep 2016 07:48:56 +0200 Subject: [PATCH 123/186] mute expected exception in RemoteDeathWatchSpec --- .../test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala index c7da3eab69..b070097412 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala @@ -36,6 +36,9 @@ object RemoteDeathWatchSpec { 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.port=$otherPort") .withFallback(system.settings.config)) From 494ccc00dc9c51ac5c4ca51cdea7e9091a008faf Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 8 Sep 2016 19:34:15 +0200 Subject: [PATCH 124/186] add recover in front of MergeHub, to avoid logging, #21397 --- akka-remote/src/main/scala/akka/remote/artery/Association.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 1592a85abc..57c3531175 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -496,6 +496,8 @@ private[remote] class Association( .viaMat(transport.outboundTestFlow(this))(Keep.both) .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, m), c), w) ⇒ ((q, m), (c, w)) } From 97e06281734c71412e8a76e80095200fe2dc6304 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 9 Sep 2016 09:01:15 +0200 Subject: [PATCH 125/186] enable misc serializers by default for Artery, #21339 * placed them in a new section additional-serialization-bindings, which is included by default when Artery is enabled * can also be enabled with enable-additional-serialization-bindings flag to simplify usage with old remoting * added a JavaSerializable marker trait that is bound to JavaSerializer in testkit, this can be used in tests so that we eventually can run tests without the java.io.Serializable binding --- akka-actor/src/main/resources/reference.conf | 13 ++++++++++++ .../akka/serialization/Serialization.scala | 16 +++++++++++---- .../remote/artery/MaxThroughputSpec.scala | 4 ++-- akka-remote/src/main/resources/reference.conf | 20 ++++++++++--------- .../artery/LargeMessagesStreamSpec.scala | 6 ++++-- .../remote/artery/RemoteActorForSpec.scala | 2 +- .../artery/RemoteActorSelectionSpec.scala | 5 +++-- .../remote/artery/RemoteWatcherSpec.scala | 4 ++-- .../akka/remote/artery/UntrustedSpec.scala | 5 +++-- .../MiscMessageSerializerSpec.scala | 11 ++++------ .../src/main/resources/reference.conf | 4 ++++ .../scala/akka/testkit/JavaSerializable.scala | 11 ++++++++++ 12 files changed, 70 insertions(+), 31 deletions(-) create mode 100644 akka-testkit/src/main/scala/akka/testkit/JavaSerializable.scala diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index 7f6a508f2b..10ff0eee8d 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -585,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 diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index 4017ac6689..7dfdf6eff1 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -34,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) } } } 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 index 0932066320..9488ddd898 100644 --- 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 @@ -70,10 +70,10 @@ object MaxThroughputSpec extends MultiNodeConfig { """))) case object Run - sealed trait Echo extends DeadLetterSuppression + 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) + 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 = diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index bd266bc3f8..2ed7aea466 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -22,15 +22,7 @@ akka { 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 - # "scala.Some" = akka-misc - # "scala.None$" = akka-misc + "akka.remote.DaemonMsgCreate" = daemon-create "akka.remote.artery.ArteryMessage" = artery @@ -47,6 +39,16 @@ akka { # 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 + } serialization-identifiers { "akka.remote.serialization.ProtobufSerializer" = 2 diff --git a/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala index 6312a8bc28..378260c321 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/LargeMessagesStreamSpec.scala @@ -10,10 +10,12 @@ 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) - case class Pong(bytesReceived: Long) + 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) diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorForSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorForSpec.scala index 4ddd4f4ea5..9474b11be5 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorForSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorForSpec.scala @@ -13,7 +13,7 @@ import akka.util.Timeout import scala.concurrent.duration._ object RemoteActorForSpec { - final case class ActorForReq(s: String) + final case class ActorForReq(s: String) extends JavaSerializable } class RemoteActorForSpec extends ArteryMultiNodeSpec("akka.loglevel=INFO") with ImplicitSender with DefaultTimeout { diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala index d8e0c1e115..b4b023c81e 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorSelectionSpec.scala @@ -9,10 +9,11 @@ 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) - final case class ActorCreateReq(props: Props, name: String) + 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") diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala index 3267448214..82650e08d6 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala @@ -37,8 +37,8 @@ object RemoteWatcherSpec { } object TestRemoteWatcher { - final case class AddressTerm(address: Address) - final case class Quarantined(address: Address, uid: Option[Int]) + final case class AddressTerm(address: Address) extends JavaSerializable + final case class Quarantined(address: Address, uid: Option[Int]) extends JavaSerializable } class TestRemoteWatcher(heartbeatExpectedResponseAfter: FiniteDuration) extends RemoteWatcher( diff --git a/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala index 837ffd7dbe..5bbf0f79f8 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala @@ -25,10 +25,11 @@ 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) - final case class StopChild(name: String) + 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") 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..a5ed4c45b7 100644 --- a/akka-remote/src/test/scala/akka/remote/serialization/MiscMessageSerializerSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/serialization/MiscMessageSerializerSpec.scala @@ -13,13 +13,10 @@ import com.typesafe.config.ConfigFactory 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 + """ val testConfig = ConfigFactory.parseString(serializationTestOverrides).withFallback(AkkaSpec.testConf) } 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 From 7b665f705e3b71dd5d5b264de8fa03dfcedd2997 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 9 Sep 2016 10:15:12 +0200 Subject: [PATCH 126/186] try delayed retry resolve of remote deployed refs only once, #21403 * 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. --- .../scala/akka/remote/artery/Codecs.scala | 35 +++++++++++++++---- 1 file changed, 28 insertions(+), 7 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index e62d6de02e..e2ee25c58e 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -232,6 +232,7 @@ private[remote] class Decoder( private val headerBuilder = HeaderBuilder.in(compression) private val actorRefResolver: ActorRefResolveCache = new ActorRefResolveCache(system.provider.asInstanceOf[RemoteActorRefProvider], uniqueLocalAddress) + private val bannedRemoteDeployedActorRefs = new java.util.HashSet[String] private val retryResolveRemoteDeployedRecipientInterval = 50.millis private val retryResolveRemoteDeployedRecipientAttempts = 20 @@ -322,11 +323,23 @@ private[remote] class Decoder( 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 - scheduleOnce(RetryResolveRemoteDeployedRecipient( - retryResolveRemoteDeployedRecipientAttempts, - headerBuilder.recipientActorRefPath.get, decoded), retryResolveRemoteDeployedRecipientInterval) + + // 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)) { + log.debug( + "Dropping message for banned (terminated) remote deployed recipient [{}].", + recipientActorRefPath) + pull(in) + } else + scheduleOnce(RetryResolveRemoteDeployedRecipient( + retryResolveRemoteDeployedRecipientAttempts, + recipientActorRefPath, decoded), retryResolveRemoteDeployedRecipientInterval) } else push(out, decoded) } @@ -336,7 +349,6 @@ private[remote] class Decoder( actorRefResolver.getOrCompute(path) match { case empty: EmptyLocalActorRef ⇒ val pathElements = empty.path.elements - // FIXME remote deployment corner case, please fix @patriknw (see also below, in onTimer) if (pathElements.nonEmpty && pathElements.head == "remote") OptionVal.None else OptionVal(empty) case ref ⇒ OptionVal(ref) @@ -372,8 +384,17 @@ private[remote] class Decoder( 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) - // FIXME only retry for the first message, need to keep them in a cache push(out, inboundEnvelope.withRecipient(recipient)) } case OptionVal.Some(recipient) ⇒ From 1ce7d7d7e999d5ed2cd3caa319e975046821d547 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Martynas=20Mickevi=C4=8Dius?= Date: Fri, 9 Sep 2016 13:46:50 +0300 Subject: [PATCH 127/186] #20946 Add bind address (#21404) --- .../akka/remote/testkit/MultiNodeSpec.scala | 4 +- .../RemoteNodeRestartDeathWatchSpec.scala | 2 +- .../RemoteNodeShutdownAndComesBackSpec.scala | 2 +- .../remote/RemoteQuarantinePiercingSpec.scala | 2 +- .../artery/HandshakeRestartReceiverSpec.scala | 2 +- .../RemoteRestartedQuarantinedSpec.scala | 2 +- akka-remote/src/main/resources/reference.conf | 61 +++++++++++--- .../akka/remote/BoundAddressesExtension.scala | 7 +- .../akka/remote/artery/ArterySettings.scala | 34 ++++++-- .../akka/remote/artery/ArteryTransport.scala | 21 ++++- .../remote/artery/ArteryMultiNodeSpec.scala | 6 +- .../artery/BindCanonicalAddressSpec.scala | 83 +++++++++++++++++++ .../remote/artery/HandshakeFailureSpec.scala | 6 +- .../remote/artery/HandshakeRetrySpec.scala | 6 +- .../akka/remote/artery/LateConnectSpec.scala | 6 +- .../remote/artery/RemoteConnectionSpec.scala | 5 +- .../remote/artery/RemoteDeathWatchSpec.scala | 6 +- .../remote/artery/RemoteDeployerSpec.scala | 4 +- .../remote/artery/RemoteDeploymentSpec.scala | 4 +- .../akka/remote/artery/RemoteRouterSpec.scala | 4 +- .../artery/RemoteSendConsistencySpec.scala | 4 +- .../remote/artery/RemoteWatcherSpec.scala | 4 +- .../artery/SerializationErrorSpec.scala | 4 +- .../artery/SystemMessageDeliverySpec.scala | 4 +- .../akka/remote/artery/UntrustedSpec.scala | 8 +- .../compress/CompressionIntegrationSpec.scala | 8 +- ...dshakeShouldDropCompressionTableSpec.scala | 7 +- 27 files changed, 232 insertions(+), 74 deletions(-) create mode 100644 akka-remote/src/test/scala/akka/remote/artery/BindCanonicalAddressSpec.scala 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 dabd86f341..2ba21c2207 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 @@ -200,10 +200,10 @@ object MultiNodeSpec { private[testkit] val nodeConfig = mapToConfig(Map( "akka.actor.provider" → "remote", - "akka.remote.artery.hostname" → selfName, + "akka.remote.artery.canonical.hostname" → selfName, "akka.remote.netty.tcp.hostname" → selfName, "akka.remote.netty.tcp.port" → selfPort, - "akka.remote.artery.port" → selfPort)) + "akka.remote.artery.canonical.port" → selfPort)) private[testkit] val baseConfig: Config = ConfigFactory.parseString(""" akka { 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 7ac9d1c531..2a7a838e69 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 @@ -114,7 +114,7 @@ abstract class RemoteNodeRestartDeathWatchSpec(multiNodeConfig: RemoteNodeRestar val freshSystem = ActorSystem(system.name, ConfigFactory.parseString(s""" akka.remote.netty.tcp.port = ${addr.port.get} - akka.remote.artery.port = ${addr.port.get} + akka.remote.artery.canonical.port = ${addr.port.get} """).withFallback(system.settings.config)) freshSystem.actorOf(Props[Subject], "subject") 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 9102832593..dc72e998a0 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 @@ -136,7 +136,7 @@ abstract class RemoteNodeShutdownAndComesBackSpec val freshSystem = ActorSystem(system.name, ConfigFactory.parseString(s""" akka.remote.netty.tcp.port = ${addr.port.get} - akka.remote.artery.port = ${addr.port.get} + akka.remote.artery.canonical.port = ${addr.port.get} """).withFallback(system.settings.config)) freshSystem.actorOf(Props[Subject], "subject") 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 ce0a22005b..170fdfa4f6 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 @@ -115,7 +115,7 @@ abstract class RemoteQuarantinePiercingSpec(multiNodeConfig: RemoteQuarantinePie val freshSystem = ActorSystem(system.name, ConfigFactory.parseString(s""" akka.remote.netty.tcp.port = ${addr.port.get} - akka.remote.artery.port = ${addr.port.get} + akka.remote.artery.canonical.port = ${addr.port.get} """).withFallback(system.settings.config)) freshSystem.actorOf(Props[Subject], "subject") 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 index 648c4e41ff..6e5ec4d812 100644 --- 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 @@ -105,7 +105,7 @@ abstract class HandshakeRestartReceiverSpec Await.result(system.whenTerminated, 10.seconds) val freshSystem = ActorSystem(system.name, ConfigFactory.parseString(s""" - akka.remote.artery.port = ${addr.port.get} + akka.remote.artery.canonical.port = ${addr.port.get} """).withFallback(system.settings.config)) freshSystem.actorOf(Props[Subject], "subject2") 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 index 1cdcc74fc1..b7bc87155a 100644 --- 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 @@ -116,7 +116,7 @@ abstract class RemoteRestartedQuarantinedSpec Await.result(system.whenTerminated, 10.seconds) val freshSystem = ActorSystem(system.name, ConfigFactory.parseString(s""" - akka.remote.artery.port = ${addr.port.get} + akka.remote.artery.canonical.port = ${addr.port.get} """).withFallback(system.settings.config)) val probe = TestProbe()(freshSystem) diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index bd266bc3f8..eed3fe538c 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -86,15 +86,50 @@ akka { remote { artery { - enabled = off - port = 20200 - # The hostname or ip clients should connect to. - # InetAddress.getLocalHost.getHostAddress is used if empty or - # "" is specified. - # InetAddress.getLocalHost.getHostName is used if - # "" is specified. - hostname = "" + ### FIXME: Temporary switch for the PoC + 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 @@ -153,16 +188,16 @@ akka { # Level 1 strongly prefer low CPU consumption over low latency. # Level 10 strongly prefer low latency over low CPU consumption. idle-cpu-level = 5 - + # 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 - - # 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 + + # 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 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/artery/ArterySettings.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala index fa68f8c5e5..6f008d4235 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala @@ -22,12 +22,30 @@ private[akka] final class ArterySettings private (config: Config) { import ArterySettings._ val Enabled: Boolean = getBoolean("enabled") - val Port: Int = getInt("port") - val Hostname: String = getString("hostname") match { - case "" | "" ⇒ InetAddress.getLocalHost.getHostAddress - case "" ⇒ InetAddress.getLocalHost.getHostName - case other ⇒ other + + 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 @@ -126,4 +144,10 @@ private[akka] object ArterySettings { // Compile time constants final val Debug = false // unlocks additional very verbose debug logging of compression events (on DEBUG log level) } + + 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 index 8cbbcbba9b..c7f04f6e81 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -287,6 +287,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // 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 controlSubject: ControlMessageSubject = _ @@ -297,6 +298,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R @volatile private[this] var inboundCompressions: Option[InboundCompressions] = None + def bindAddress: UniqueAddress = _bindAddress override def localAddress: UniqueAddress = _localAddress override def defaultAddress: Address = localAddress.address override def addresses: Set[Address] = _addresses @@ -328,7 +330,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .insert(Array("system", "cluster", "core", "daemon", "heartbeatSender"), NotUsed) .insert(Array("system", "cluster", "heartbeatReceiver"), NotUsed) - private def inboundChannel = s"aeron:udp?endpoint=${localAddress.address.host.get}:${localAddress.address.port.get}" + 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 @@ -388,16 +390,27 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R topLevelFREvents.loFreq(Transport_TaskRunnerStarted, NoMetaData) val port = - if (settings.Port == 0) ArteryTransport.autoSelectPort(settings.Hostname) - else settings.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 // TODO: Configure materializer properly // TODO: Have a supervisor actor _localAddress = UniqueAddress( - Address(ArteryTransport.ProtocolName, system.name, settings.Hostname, port), + 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")) diff --git a/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala index 4640772ec8..908bf25f3d 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala @@ -16,8 +16,10 @@ object ArteryMultiNodeSpec { actor.warn-about-java-serializer-usage = off remote.artery { enabled = on - hostname = localhost - port = 0 + canonical { + hostname = localhost + port = 0 + } } } """) 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/HandshakeFailureSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala index 9910d72653..802f5a58f8 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala @@ -21,13 +21,13 @@ object HandshakeFailureSpec { akka { actor.provider = remote remote.artery.enabled = on - remote.artery.hostname = localhost - remote.artery.port = 0 + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 remote.artery.advanced.handshake-timeout = 2s } """) - val configB = ConfigFactory.parseString(s"akka.remote.artery.port = $portB") + val configB = ConfigFactory.parseString(s"akka.remote.artery.canonical.port = $portB") .withFallback(commonConfig) } diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala index fa003aa628..3e8feadf59 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala @@ -20,14 +20,14 @@ object HandshakeRetrySpec { akka { actor.provider = remote remote.artery.enabled = on - remote.artery.hostname = localhost - remote.artery.port = 0 + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 remote.handshake-timeout = 10s } """) - val configB = ConfigFactory.parseString(s"akka.remote.artery.port = $portB") + val configB = ConfigFactory.parseString(s"akka.remote.artery.canonical.port = $portB") .withFallback(commonConfig) } diff --git a/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala index edf74c2736..36f1690dd2 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala @@ -21,8 +21,8 @@ object LateConnectSpec { akka { actor.provider = remote remote.artery.enabled = on - remote.artery.hostname = localhost - remote.artery.port = 0 + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 remote.artery.advanced.handshake-timeout = 3s } """) @@ -32,7 +32,7 @@ object LateConnectSpec { class LateConnectSpec extends AkkaSpec(LateConnectSpec.config) with ImplicitSender { val portB = SocketUtil.temporaryServerAddress("localhost", udp = true).getPort - val configB = ConfigFactory.parseString(s"akka.remote.artery.port = $portB") + val configB = ConfigFactory.parseString(s"akka.remote.artery.canonical.port = $portB") .withFallback(system.settings.config) lazy val systemB = ActorSystem("systemB", configB) diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteConnectionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteConnectionSpec.scala index 7ff1fad35c..8389797538 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteConnectionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteConnectionSpec.scala @@ -34,7 +34,7 @@ class RemoteConnectionSpec extends ArteryMultiNodeSpec("akka.remote.retry-gate-c localProbe.expectNoMsg(1.seconds) // then start the remote system and try again - val remoteSystem = newRemoteSystem(extraConfig = Some(s"akka.remote.artery.port=$remotePort")) + val remoteSystem = newRemoteSystem(extraConfig = Some(s"akka.remote.artery.canonical.port=$remotePort")) muteSystem(remoteSystem) localProbe.expectNoMsg(2.seconds) @@ -65,7 +65,7 @@ class RemoteConnectionSpec extends ArteryMultiNodeSpec("akka.remote.retry-gate-c localProbe.expectNoMsg(1.seconds) // then when it is up, talk from other system - val remoteSystem = newRemoteSystem(extraConfig = Some(s"akka.remote.artery.port=$remotePort")) + val remoteSystem = newRemoteSystem(extraConfig = Some(s"akka.remote.artery.canonical.port=$remotePort")) muteSystem(remoteSystem) localProbe.expectNoMsg(2.seconds) @@ -82,4 +82,3 @@ class RemoteConnectionSpec extends ArteryMultiNodeSpec("akka.remote.retry-gate-c } } - diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala index b070097412..635b73dc8e 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeathWatchSpec.scala @@ -27,8 +27,8 @@ object RemoteDeathWatchSpec { } remote.watch-failure-detector.acceptable-heartbeat-pause = 3s remote.artery.enabled = on - remote.artery.hostname = localhost - remote.artery.port = 0 + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 } """) } @@ -39,7 +39,7 @@ class RemoteDeathWatchSpec extends AkkaSpec(RemoteDeathWatchSpec.config) with Im system.eventStream.publish(TestEvent.Mute( EventFilter[io.aeron.exceptions.RegistrationException]())) - val other = ActorSystem("other", ConfigFactory.parseString(s"akka.remote.artery.port=$otherPort") + val other = ActorSystem("other", ConfigFactory.parseString(s"akka.remote.artery.canonical.port=$otherPort") .withFallback(system.settings.config)) override def afterTermination() { diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeployerSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeployerSpec.scala index 5c95b122e8..b4152ae111 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeployerSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeployerSpec.scala @@ -22,8 +22,8 @@ object RemoteDeployerSpec { } } akka.remote.artery.enabled = on - akka.remote.artery.hostname = localhost - akka.remote.artery.port = 0 + akka.remote.artery.canonical.hostname = localhost + akka.remote.artery.canonical.port = 0 """, ConfigParseOptions.defaults) class RecipeActor extends Actor { diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala index 7189141f3d..90f89c3147 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteDeploymentSpec.scala @@ -36,8 +36,8 @@ class RemoteDeploymentSpec extends AkkaSpec(""" #akka.loglevel=DEBUG akka.actor.provider = remote akka.remote.artery.enabled = on - akka.remote.artery.hostname = localhost - akka.remote.artery.port = 0 + akka.remote.artery.canonical.hostname = localhost + akka.remote.artery.canonical.port = 0 """) { import RemoteDeploymentSpec._ diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala index 00fbf374d6..80f133b7cc 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteRouterSpec.scala @@ -24,8 +24,8 @@ object RemoteRouterSpec { class RemoteRouterSpec extends AkkaSpec(""" akka.actor.provider = remote akka.remote.artery.enabled = on - akka.remote.artery.hostname = localhost - akka.remote.artery.port = 0 + akka.remote.artery.canonical.hostname = localhost + akka.remote.artery.canonical.port = 0 akka.actor.deployment { /remote-override { router = round-robin-pool diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala index da8b7bea11..fd874fc152 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala @@ -21,8 +21,8 @@ object RemoteSendConsistencySpec { akka { actor.provider = remote remote.artery.enabled = on - remote.artery.hostname = localhost - remote.artery.port = 0 + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 } """) diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala index 3267448214..db5d67ff6b 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala @@ -69,8 +69,8 @@ class RemoteWatcherSpec extends AkkaSpec( log-dead-letters-during-shutdown = false actor.provider = remote remote.artery.enabled = on - remote.artery.hostname = localhost - remote.artery.port = 0 + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 }""") with ImplicitSender { import RemoteWatcherSpec._ diff --git a/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala index a555c51fc6..63b19320d4 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SerializationErrorSpec.scala @@ -17,8 +17,8 @@ object SerializationErrorSpec { akka { actor.provider = remote remote.artery.enabled = on - remote.artery.hostname = localhost - remote.artery.port = 0 + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 actor { serialize-creators = false serialize-messages = false diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index 624acf337a..3ad664f708 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -38,8 +38,8 @@ object SystemMessageDeliverySpec { akka { actor.provider = remote remote.artery.enabled = on - remote.artery.hostname = localhost - remote.artery.port = 0 + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 } akka.actor.serialize-creators = off akka.actor.serialize-messages = off diff --git a/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala index 837ffd7dbe..7963b8f025 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala @@ -65,8 +65,8 @@ class UntrustedSpec extends AkkaSpec(""" akka.remote.untrusted-mode = on akka.remote.trusted-selection-paths = ["/user/receptionist", ] akka.remote.artery.enabled = on - akka.remote.artery.hostname = localhost - akka.remote.artery.port = 0 + akka.remote.artery.canonical.hostname = localhost + akka.remote.artery.canonical.port = 0 akka.loglevel = DEBUG # the test is verifying some Debug logging """) with ImplicitSender { @@ -75,8 +75,8 @@ class UntrustedSpec extends AkkaSpec(""" val client = ActorSystem("UntrustedSpec-client", ConfigFactory.parseString(""" akka.actor.provider = remote akka.remote.artery.enabled = on - akka.remote.artery.hostname = localhost - akka.remote.artery.port = 0 + akka.remote.artery.canonical.hostname = localhost + akka.remote.artery.canonical.port = 0 """)) val addr = RARP(system).provider.getDefaultAddress 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 index e3d479c7fa..47f7523603 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala @@ -17,7 +17,7 @@ object CompressionIntegrationSpec { actor { provider = "akka.remote.RemoteActorRefProvider" - + serializers { test-message = "akka.remote.artery.compress.TestMessageSerializer" } @@ -26,8 +26,8 @@ object CompressionIntegrationSpec { } } remote.artery.enabled = on - remote.artery.hostname = localhost - remote.artery.port = 0 + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 remote.artery.advanced.handshake-timeout = 10s remote.artery.advanced.compression { @@ -38,6 +38,6 @@ object CompressionIntegrationSpec { } """) - val configB = ConfigFactory.parseString(s"akka.remote.artery.port = $portB") + val configB = ConfigFactory.parseString(s"akka.remote.artery.canonical.port = $portB") .withFallback(commonConfig) } 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 index 55433ae0f1..4cf2c69646 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala @@ -28,8 +28,8 @@ object HandshakeShouldDropCompressionTableSpec { actor.provider = "akka.remote.RemoteActorRefProvider" remote.artery.enabled = on - remote.artery.hostname = localhost - remote.artery.port = 0 + remote.artery.canonical.hostname = localhost + remote.artery.canonical.port = 0 remote.artery.advanced.handshake-timeout = 10s remote.artery.advanced.compression { @@ -41,7 +41,7 @@ object HandshakeShouldDropCompressionTableSpec { } """) - val configB = ConfigFactory.parseString(s"akka.remote.artery.port = $portB") + val configB = ConfigFactory.parseString(s"akka.remote.artery.canonical.port = $portB") .withFallback(commonConfig) } @@ -149,4 +149,3 @@ class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDr if (systemB != null) shutdown(systemB) } } - From 0d77034adc61c98144cb11ccc62985dc6304bd45 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Endre=20S=C3=A1ndor=20Varga?= Date: Fri, 9 Sep 2016 14:29:04 +0200 Subject: [PATCH 128/186] 20623 Make sure external (mapped) resources are properly cleaned on shutdown --- .../artery/AeronStreamLatencySpec.scala | 4 ++- .../artery/AeronStreamMaxThroughputSpec.scala | 4 ++- .../akka/remote/artery/LatencySpec.scala | 15 +++++++-- .../akka/remote/artery/AeronErrorLog.java | 33 ++++++++++++------- .../akka/remote/artery/ArteryTransport.scala | 6 ++-- .../java/akka/remote/artery/AeronStat.java | 21 ++++++++++-- 6 files changed, 62 insertions(+), 21 deletions(-) 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 index 4b0fd67dd0..ef840e78cd 100644 --- 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 @@ -88,8 +88,9 @@ abstract class AeronStreamLatencySpec 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(new File(driver.aeronDirectoryName, CncFileDescriptor.CNC_FILE))) + new AeronStat(AeronStat.mapCounters(cncByteBuffer)) val aeron = { val ctx = new Aeron.Context @@ -129,6 +130,7 @@ abstract class AeronStreamLatencySpec 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")) 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 index a1f9a7ee4d..1f9875afee 100644 --- 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 @@ -86,8 +86,9 @@ abstract class AeronStreamMaxThroughputSpec 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(new File(driver.aeronDirectoryName, CncFileDescriptor.CNC_FILE))) + new AeronStat(AeronStat.mapCounters(cncByteBuffer)) val aeron = { val ctx = new Aeron.Context @@ -129,6 +130,7 @@ abstract class AeronStreamMaxThroughputSpec taskRunner.stop() aeron.close() driver.close() + IoUtil.unmap(cncByteBuffer) IoUtil.delete(new File(driver.aeronDirectoryName), true) runOn(second) { println(plot.csv(system.name)) 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 index c2251ed9f4..f8e1afe19b 100644 --- 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 @@ -4,8 +4,9 @@ package akka.remote.artery import java.util.concurrent.Executors -import java.util.concurrent.atomic.AtomicLongArray +import java.util.concurrent.atomic.{ AtomicBoolean, AtomicLongArray } import java.util.concurrent.locks.LockSupport + import scala.concurrent.duration._ import akka.actor._ import akka.remote.testconductor.RoleName @@ -83,6 +84,7 @@ object LatencySpec extends MultiNodeConfig { var count = 0 var startTime = System.nanoTime() val taskRunnerMetrics = new TaskRunnerMetrics(context.system) + var reportedArrayOOB = false def receive = { case bytes: Array[Byte] ⇒ @@ -100,7 +102,16 @@ object LatencySpec extends MultiNodeConfig { reporter.onMessage(1, payloadSize) count += 1 val d = System.nanoTime() - sendTimes.get(count - 1) - histogram.recordValue(d) + 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) diff --git a/akka-remote/src/main/java/akka/remote/artery/AeronErrorLog.java b/akka-remote/src/main/java/akka/remote/artery/AeronErrorLog.java index edee08f987..87f0389594 100644 --- a/akka-remote/src/main/java/akka/remote/artery/AeronErrorLog.java +++ b/akka-remote/src/main/java/akka/remote/artery/AeronErrorLog.java @@ -37,26 +37,31 @@ import java.util.concurrent.atomic.AtomicLong; 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) { - final MappedByteBuffer cncByteBuffer = IoUtil.mapExistingFile(cncFile, "cnc"); - final DirectBuffer cncMetaDataBuffer = CncFileDescriptor.createMetaDataBuffer(cncByteBuffer); - final int cncVersion = cncMetaDataBuffer.getInt(CncFileDescriptor.cncVersionOffset(0)); - - if (CncFileDescriptor.CNC_VERSION != cncVersion) - { - throw new IllegalStateException("CNC version not supported: file version=" + cncVersion); - } - - final AtomicBuffer buffer = CncFileDescriptor.createErrorLogBuffer(cncByteBuffer, cncMetaDataBuffer); - final SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSSZ"); - // using AtomicLong because access from lambda, not because of currency final AtomicLong lastTimestamp = new AtomicLong(sinceTimestamp); @@ -73,4 +78,8 @@ public class AeronErrorLog }, sinceTimestamp); return lastTimestamp.get(); } + + public void close() { + IoUtil.unmap(cncByteBuffer); + } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 8cbbcbba9b..a0c19bf561 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -294,6 +294,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R 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 @@ -541,12 +542,12 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // TODO Add FR Events private def startAeronErrorLog(): Unit = { - val errorLog = new AeronErrorLog(new File(aeronDir, CncFileDescriptor.CNC_FILE)) + 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 = errorLog.logErrors(log, lastTimestamp.get) + val newLastTimestamp = areonErrorLog.logErrors(log, lastTimestamp.get) lastTimestamp.set(newLastTimestamp + 1) } } @@ -757,6 +758,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R topLevelFREvents.loFreq(Transport_AeronErrorLogTaskStopped, NoMetaData) } if (aeron != null) aeron.close() + if (areonErrorLog != null) areonErrorLog.close() if (mediaDriver.get.isDefined) { stopMediaDriver() topLevelFREvents.loFreq(Transport_MediaFileDeleted, NoMetaData) diff --git a/akka-remote/src/test/java/akka/remote/artery/AeronStat.java b/akka-remote/src/test/java/akka/remote/artery/AeronStat.java index feab499287..3b63943f12 100644 --- a/akka-remote/src/test/java/akka/remote/artery/AeronStat.java +++ b/akka-remote/src/test/java/akka/remote/artery/AeronStat.java @@ -117,7 +117,22 @@ public class AeronStat { 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); @@ -132,8 +147,8 @@ public class AeronStat } return new CountersReader( - createCountersMetaDataBuffer(cncByteBuffer, cncMetaData), - createCountersValuesBuffer(cncByteBuffer, cncMetaData)); + createCountersMetaDataBuffer(cncByteBuffer, cncMetaData), + createCountersValuesBuffer(cncByteBuffer, cncMetaData)); } public static void main(final String[] args) throws Exception From cd4a31e74db45f14b2837ac2defd0d70346b7412 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 8 Sep 2016 15:01:32 +0200 Subject: [PATCH 129/186] No ack delivery for prio messages, #21371 * and send prio messages enclosed in actor selection over the control stream --- .../akka/remote/artery/Association.scala | 8 +++- .../remote/artery/SystemMessageDelivery.scala | 39 ++++++++++-------- .../artery/SystemMessageDeliverySpec.scala | 40 ++++++++++--------- 3 files changed, 51 insertions(+), 36 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 57c3531175..cd42e7ec73 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -287,12 +287,18 @@ private[remote] class Association( // allow ActorSelectionMessage to pass through quarantine, to be able to establish interaction with new system if (message.isInstanceOf[ActorSelectionMessage] || !associationState.isQuarantined() || message == ClearSystemMessageDelivery) { message match { - case _: SystemMessage | ClearSystemMessageDelivery | _: ControlMessage ⇒ + 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. diff --git a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala index 3eaead00b7..957c432036 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -23,6 +23,9 @@ import akka.stream.stage.OutHandler import akka.stream.stage.TimerGraphStageLogic import akka.remote.artery.OutboundHandshake.HandshakeReq import akka.actor.ActorRef +import akka.remote.PriorityMessage +import akka.actor.ActorSelectionMessage +import akka.dispatch.sysmsg.SystemMessage /** * INTERNAL API @@ -36,6 +39,10 @@ private[akka] object SystemMessageDelivery { final case object ClearSystemMessageDelivery 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 } /** @@ -166,22 +173,7 @@ private[akka] class SystemMessageDelivery( override def onPush(): Unit = { val outboundEnvelope = grab(in) outboundEnvelope.message match { - case _: HandshakeReq ⇒ - // pass on HandshakeReq - if (isAvailable(out)) - pushCopy(outboundEnvelope) - case ClearSystemMessageDelivery ⇒ - clear() - pull(in) - case _: ControlMessage ⇒ - // e.g. ActorSystemTerminating, no need for acked delivery - if (resending.isEmpty && isAvailable(out)) - pushCopy(outboundEnvelope) - else { - resending.offer(outboundEnvelope) - tryResend() - } - case msg ⇒ + case msg @ (_: SystemMessage | _: AckedDeliveryMessage) ⇒ if (unacknowledged.size < maxBufferSize) { seqNo += 1 val sendEnvelope = outboundEnvelope.withMessage(SystemMessageEnvelope(msg, seqNo, localAddress)) @@ -199,6 +191,21 @@ private[akka] class SystemMessageDelivery( 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() + } } } diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index 3ad664f708..9264c7d980 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -45,6 +45,8 @@ object SystemMessageDeliverySpec { akka.actor.serialize-messages = off """) + case class TestSysMsg(s: String) extends SystemMessageDelivery.AckedDeliveryMessage + } class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.config) with ImplicitSender { @@ -68,7 +70,7 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi 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, "msg-" + n, OptionVal.None)) + .map(n ⇒ outboundEnvelopePool.acquire().init(OptionVal.None, TestSysMsg("msg-" + n), OptionVal.None)) .via(new SystemMessageDelivery(outboundContext, deadLetters, resendInterval, maxBufferSize = 1000)) } @@ -159,12 +161,12 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi val sink = send(sendCount = 5, resendInterval = 60.seconds, outboundContextA) .via(drop(dropSeqNumbers = Vector(3L, 4L))) .via(inbound(inboundContextB)) - .map(_.message.asInstanceOf[String]) + .map(_.message.asInstanceOf[TestSysMsg]) .runWith(TestSink.probe) sink.request(100) - sink.expectNext("msg-1") - sink.expectNext("msg-2") + 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 @@ -172,11 +174,11 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi sink.expectNoMsg(100.millis) // 3 was dropped inboundContextB.deliverLastReply() // resending 3, 4, 5 - sink.expectNext("msg-3") + sink.expectNext(TestSysMsg("msg-3")) replyProbe.expectMsg(Ack(3L, addressB)) - sink.expectNext("msg-4") + sink.expectNext(TestSysMsg("msg-4")) replyProbe.expectMsg(Ack(4L, addressB)) - sink.expectNext("msg-5") + sink.expectNext(TestSysMsg("msg-5")) replyProbe.expectMsg(Ack(5L, addressB)) replyProbe.expectNoMsg(100.millis) inboundContextB.deliverLastReply() @@ -193,7 +195,7 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi val sink = send(sendCount = 3, resendInterval = 60.seconds, outboundContextA) .via(drop(dropSeqNumbers = Vector(1L))) .via(inbound(inboundContextB)) - .map(_.message.asInstanceOf[String]) + .map(_.message.asInstanceOf[TestSysMsg]) .runWith(TestSink.probe) sink.request(100) @@ -202,11 +204,11 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi sink.expectNoMsg(100.millis) // 1 was dropped inboundContextB.deliverLastReply() // it's ok to not delivery all nacks // resending 1, 2, 3 - sink.expectNext("msg-1") + sink.expectNext(TestSysMsg("msg-1")) replyProbe.expectMsg(Ack(1L, addressB)) - sink.expectNext("msg-2") + sink.expectNext(TestSysMsg("msg-2")) replyProbe.expectMsg(Ack(2L, addressB)) - sink.expectNext("msg-3") + sink.expectNext(TestSysMsg("msg-3")) replyProbe.expectMsg(Ack(3L, addressB)) inboundContextB.deliverLastReply() sink.expectComplete() @@ -222,19 +224,19 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi val sink = send(sendCount = 3, resendInterval = 2.seconds, outboundContextA) .via(drop(dropSeqNumbers = Vector(3L))) .via(inbound(inboundContextB)) - .map(_.message.asInstanceOf[String]) + .map(_.message.asInstanceOf[TestSysMsg]) .runWith(TestSink.probe) sink.request(100) - sink.expectNext("msg-1") + sink.expectNext(TestSysMsg("msg-1")) replyProbe.expectMsg(Ack(1L, addressB)) inboundContextB.deliverLastReply() - sink.expectNext("msg-2") + 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("msg-3") + sink.expectNext(TestSysMsg("msg-3")) replyProbe.expectMsg(4.seconds, Ack(3L, addressB)) // continue resending replyProbe.expectMsg(4.seconds, Ack(3L, addressB)) @@ -255,10 +257,10 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi send(N, 1.second, outboundContextA) .via(randomDrop(dropRate)) .via(inbound(inboundContextB)) - .map(_.message.asInstanceOf[String]) + .map(_.message.asInstanceOf[TestSysMsg]) .runWith(Sink.seq) - Await.result(output, 20.seconds) should ===((1 to N).map("msg-" + _).toVector) + Await.result(output, 20.seconds) should ===((1 to N).map(n ⇒ TestSysMsg("msg-" + n)).toVector) } "deliver all during throttling and random dropping" in { @@ -274,10 +276,10 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi .throttle(200, 1.second, 10, ThrottleMode.shaping) .via(randomDrop(dropRate)) .via(inbound(inboundContextB)) - .map(_.message.asInstanceOf[String]) + .map(_.message.asInstanceOf[TestSysMsg]) .runWith(Sink.seq) - Await.result(output, 20.seconds) should ===((1 to N).map("msg-" + _).toVector) + Await.result(output, 20.seconds) should ===((1 to N).map(n ⇒ TestSysMsg("msg-" + n)).toVector) } } From 1584c521902254452e8551a1323621c21e0df3bd Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 9 Sep 2016 07:45:21 +0200 Subject: [PATCH 130/186] handle longer network partitions, #21399 * system messages in flight should not trigger premature quarantine in case of longer network partitions, therefore we keep the control stream alive * add give-up-system-message-after property that is used by both SystemMessageDelivery and AeronSink in the control stream * also unwrap SystemMessageEnvelope in RemoteDeadLetterActorRef * skip sending control messages after shutdown, can be triggered by scheduled compression advertisment --- .../artery/AeronStreamConcistencySpec.scala | 8 +- .../artery/AeronStreamLatencySpec.scala | 8 +- .../artery/AeronStreamMaxThroughputSpec.scala | 4 +- .../artery/SurviveNetworkPartitionSpec.scala | 113 ++++++++++++++++++ akka-remote/src/main/resources/reference.conf | 7 +- .../akka/remote/RemoteActorRefProvider.scala | 10 +- .../scala/akka/remote/artery/AeronSink.scala | 21 ++-- .../akka/remote/artery/ArterySettings.scala | 6 +- .../akka/remote/artery/ArteryTransport.scala | 5 +- .../akka/remote/artery/Association.scala | 4 +- .../remote/artery/SystemMessageDelivery.scala | 22 +++- .../akka/remote/artery/AeronSinkSpec.scala | 4 +- 12 files changed, 181 insertions(+), 31 deletions(-) create mode 100644 akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/SurviveNetworkPartitionSpec.scala 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 index 9103899eb4..240607651d 100644 --- 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 @@ -82,7 +82,7 @@ abstract class AeronStreamConsistencySpec } val streamId = 1 - val giveUpSendAfter = 30.seconds + val giveUpMessageAfter = 30.seconds override def afterAll(): Unit = { taskRunner.stop() @@ -98,7 +98,7 @@ abstract class AeronStreamConsistencySpec 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, giveUpSendAfter, IgnoreEventSink)) + .runWith(new AeronSink(channel(first), streamId, aeron, taskRunner, pool, giveUpMessageAfter, IgnoreEventSink)) } enterBarrier("echo-started") } @@ -139,7 +139,7 @@ abstract class AeronStreamConsistencySpec envelope } .throttle(1, 200.milliseconds, 1, ThrottleMode.Shaping) - .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpMessageAfter, IgnoreEventSink)) started.expectMsg(Done) } @@ -151,7 +151,7 @@ abstract class AeronStreamConsistencySpec envelope.byteBuffer.flip() envelope } - .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpMessageAfter, IgnoreEventSink)) Await.ready(done, 20.seconds) killSwitch.shutdown() 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 index 4b0fd67dd0..8b279d21d6 100644 --- 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 @@ -115,7 +115,7 @@ abstract class AeronStreamLatencySpec } val streamId = 1 - val giveUpSendAfter = 30.seconds + val giveUpMessageAfter = 30.seconds lazy val reporterExecutor = Executors.newFixedThreadPool(1) def reporter(name: String): TestRateReporter = { @@ -245,7 +245,7 @@ abstract class AeronStreamLatencySpec envelope } .throttle(1, 200.milliseconds, 1, ThrottleMode.Shaping) - .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpMessageAfter, IgnoreEventSink)) started.expectMsg(Done) } @@ -264,7 +264,7 @@ abstract class AeronStreamLatencySpec val queueValue = Source.fromGraph(new SendQueue[Unit]) .via(sendFlow) - .to(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) + .to(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpMessageAfter, IgnoreEventSink)) .run() val queue = new ManyToOneConcurrentArrayQueue[Unit](1024) @@ -314,7 +314,7 @@ abstract class AeronStreamLatencySpec 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, giveUpSendAfter, IgnoreEventSink)) + .runWith(new AeronSink(channel(first), streamId, aeron, taskRunner, pool, giveUpMessageAfter, IgnoreEventSink)) } enterBarrier("echo-started") } 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 index a1f9a7ee4d..cd75f21fef 100644 --- 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 @@ -115,7 +115,7 @@ abstract class AeronStreamMaxThroughputSpec } val streamId = 1 - val giveUpSendAfter = 30.seconds + val giveUpMessageAfter = 30.seconds lazy val reporterExecutor = Executors.newFixedThreadPool(1) def reporter(name: String): TestRateReporter = { @@ -213,7 +213,7 @@ abstract class AeronStreamMaxThroughputSpec envelope.byteBuffer.flip() envelope } - .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpSendAfter, IgnoreEventSink)) + .runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpMessageAfter, IgnoreEventSink)) printStats("sender") enterBarrier(testName + "-done") 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..9eab4bab1b --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/artery/SurviveNetworkPartitionSpec.scala @@ -0,0 +1,113 @@ +/** + * 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.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 +import akka.remote.QuarantinedEvent + +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 + """))) + + testTransport(on = true) +} + +class SurviveNetworkPartitionSpecMultiJvmNode1 extends SurviveNetworkPartitionSpec +class SurviveNetworkPartitionSpecMultiJvmNode2 extends SurviveNetworkPartitionSpec + +abstract class SurviveNetworkPartitionSpec + extends MultiNodeSpec(SurviveNetworkPartitionSpec) + with STMultiNodeSpec with ImplicitSender { + + 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/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index aca07a396a..2a210b6ee5 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -246,7 +246,12 @@ akka { inject-handshake-interval = 1 second # messages that are not accepted by Aeron are dropped after retrying for this period - give-up-send-after = 60 seconds + 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 diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 1aa85416f6..28ced7a91b 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -20,6 +20,7 @@ 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 /** * INTERNAL API @@ -98,14 +99,19 @@ private[akka] object RemoteActorRefProvider { // the dead letter status if (seqOpt.isEmpty) super.!(DeadLetter(m, senderOption.getOrElse(_provider.deadLetters), recipient)) case env: OutboundEnvelope ⇒ - super.!(DeadLetter(env.message, env.sender.getOrElse(_provider.deadLetters), + super.!(DeadLetter(unwrapSystemMessageEnvelope(env.message), env.sender.getOrElse(_provider.deadLetters), env.recipient.getOrElse(_provider.deadLetters))) case DeadLetter(env: OutboundEnvelope, _, _) ⇒ - super.!(DeadLetter(env.message, env.sender.getOrElse(_provider.deadLetters), + 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 } diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala index b90d412403..9d3e5a2fd3 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala @@ -3,6 +3,7 @@ */ package akka.remote.artery +import akka.util.PrettyDuration.PrettyPrintableDuration import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicInteger import scala.annotation.tailrec @@ -28,7 +29,7 @@ import org.agrona.hints.ThreadHints object AeronSink { - final class GaveUpSendingException(msg: String) extends RuntimeException(msg) with NoStackTrace + final class GaveUpMessageException(msg: String) extends RuntimeException(msg) with NoStackTrace final class PublicationClosedException(msg: String) extends RuntimeException(msg) with NoStackTrace @@ -75,13 +76,13 @@ object AeronSink { * @param channel eg. "aeron:udp?endpoint=localhost:40123" */ class AeronSink( - channel: String, - streamId: Int, - aeron: Aeron, - taskRunner: TaskRunner, - pool: EnvelopeBufferPool, - giveUpSendAfter: Duration, - flightRecorder: EventSink) + channel: String, + streamId: Int, + aeron: Aeron, + taskRunner: TaskRunner, + pool: EnvelopeBufferPool, + giveUpAfter: Duration, + flightRecorder: EventSink) extends GraphStageWithMaterializedValue[SinkShape[EnvelopeBuffer], Future[Done]] { import AeronSink._ import TaskRunner._ @@ -104,7 +105,7 @@ class AeronSink( private var backoffCount = spinning private var lastMsgSize = 0 private val offerTask = new OfferTask(pub, null, lastMsgSize, getAsyncCallback(_ ⇒ taskOnOfferSuccess()), - giveUpSendAfter, getAsyncCallback(_ ⇒ onGiveUp()), getAsyncCallback(_ ⇒ onPublicationClosed())) + giveUpAfter, getAsyncCallback(_ ⇒ onGiveUp()), getAsyncCallback(_ ⇒ onPublicationClosed())) private val addOfferTask: Add = Add(offerTask) private var offerTaskInProgress = false @@ -191,7 +192,7 @@ class AeronSink( private def onGiveUp(): Unit = { offerTaskInProgress = false - val cause = new GaveUpSendingException(s"Gave up sending message to $channel after $giveUpSendAfter.") + 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) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala index 6f008d4235..0d3c6cc1a8 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala @@ -89,8 +89,10 @@ private[akka] final class ArterySettings private (config: Config) { 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 GiveUpSendAfter = config.getMillisDuration("give-up-send-after").requiring(interval ⇒ - interval > Duration.Zero, "give-up-send-after 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 ⇒ diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index c7f04f6e81..35cc7ee5e9 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -876,8 +876,11 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R 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, settings.Advanced.GiveUpSendAfter, createFlightRecorderEventSink())) + envelopeBufferPool, giveUpAfter, createFlightRecorderEventSink())) } def outboundLane(outboundContext: OutboundContext): Flow[OutboundEnvelope, EnvelopeBuffer, ChangeOutboundCompression] = diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index cd42e7ec73..6894e02e49 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -24,7 +24,7 @@ import akka.event.Logging import akka.remote._ import akka.remote.DaemonMsgCreate import akka.remote.QuarantinedEvent -import akka.remote.artery.AeronSink.GaveUpSendingException +import akka.remote.artery.AeronSink.GaveUpMessageException import akka.remote.artery.Encoder.ChangeOutboundCompression import akka.remote.artery.Encoder.ChangeOutboundCompressionFailed import akka.remote.artery.InboundControlJunction.ControlMessageSubject @@ -589,7 +589,7 @@ private[remote] class Association( // don't restart after shutdown, but log some details so we notice log.error(cause, s"{} to {} failed after shutdown. {}", streamName, remoteAddress, cause.getMessage) case _: AbruptTerminationException ⇒ // ActorSystem shutdown - case cause: GaveUpSendingException ⇒ + case cause: GaveUpMessageException ⇒ log.debug("{} to {} failed. Restarting it. {}", streamName, remoteAddress, cause.getMessage) // restart unconditionally, without counting restarts lazyRestart() diff --git a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala index 957c432036..3e093fa5f8 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -3,6 +3,7 @@ */ package akka.remote.artery +import akka.util.PrettyDuration.PrettyPrintableDuration import java.util.ArrayDeque import scala.annotation.tailrec import scala.concurrent.duration._ @@ -26,6 +27,7 @@ import akka.actor.ActorRef import akka.remote.PriorityMessage import akka.actor.ActorSelectionMessage import akka.dispatch.sysmsg.SystemMessage +import scala.util.control.NoStackTrace /** * INTERNAL API @@ -38,11 +40,14 @@ private[akka] object SystemMessageDelivery { 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 + } /** @@ -71,6 +76,9 @@ private[akka] class SystemMessageDelivery( 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 @@ -109,13 +117,13 @@ private[akka] class SystemMessageDelivery( 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) - // FIXME give up resending after a long while, i.e. config property quarantine-after-silence } // ControlMessageObserver, external call @@ -141,6 +149,7 @@ private[akka] class SystemMessageDelivery( } private def ack(n: Long): Unit = { + ackTimestamp = System.nanoTime() if (n <= seqNo) clearUnacknowledged(n) } @@ -176,6 +185,10 @@ private[akka] class SystemMessageDelivery( 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) @@ -209,6 +222,13 @@ private[akka] class SystemMessageDelivery( } } + 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 diff --git a/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala index ead929855d..feeb58b2c4 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/AeronSinkSpec.scala @@ -10,7 +10,7 @@ import scala.concurrent.duration._ import scala.util.control.NoStackTrace import akka.actor.ExtendedActorSystem -import akka.remote.artery.AeronSink.GaveUpSendingException +import akka.remote.artery.AeronSink.GaveUpMessageException import akka.stream.ActorMaterializer import akka.stream.ActorMaterializerSettings import akka.stream.scaladsl.Sink @@ -75,7 +75,7 @@ class AeronSinkSpec extends AkkaSpec with ImplicitSender { .runWith(new AeronSink(channel, 1, aeron, taskRunner, pool, 500.millis, IgnoreEventSink)) // without the give up timeout the stream would not complete/fail - intercept[GaveUpSendingException] { + intercept[GaveUpMessageException] { Await.result(done, 5.seconds) } } From fa1d6d6f19a37f21b641797e40c3cc71634b7940 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Fri, 9 Sep 2016 17:54:13 +0200 Subject: [PATCH 131/186] Disable ClusterDeathWatchSpec for now (#21421) --- .../scala/akka/cluster/ClusterDeathWatchSpec.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) 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..e6e38df16a 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterDeathWatchSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterDeathWatchSpec.scala @@ -59,7 +59,11 @@ 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) { + "work with artery" in { + pending + } + +/* "receive Terminated when watched node becomes Down/Removed" in within(20 seconds) { awaitClusterUp(first, second, third, fourth) enterBarrier("cluster-up") @@ -262,7 +266,7 @@ abstract class ClusterDeathWatchSpec enterBarrier("after-4") } - } + }*/ } } From b0e03058b922cdd1f01907f438f74cab4bc56876 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Fri, 9 Sep 2016 17:55:02 +0200 Subject: [PATCH 132/186] Port and hostname config path was changed, cluster tests didn't get the change (#21427) --- .../src/multi-jvm/scala/akka/cluster/QuickRestartSpec.scala | 2 +- .../multi-jvm/scala/akka/cluster/RestartFirstSeedNodeSpec.scala | 2 +- .../src/multi-jvm/scala/akka/cluster/RestartNode3Spec.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) 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 5bc53ba4c7..ab973898d3 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/QuickRestartSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/QuickRestartSpec.scala @@ -72,7 +72,7 @@ abstract class QuickRestartSpec if (RARP(system).provider.remoteSettings.Artery.Enabled) s""" akka.cluster.roles = [round-$n] - akka.remote.artery.port = ${Cluster(restartingSystem).selfAddress.port.get} + akka.remote.artery.canonical.port = ${Cluster(restartingSystem).selfAddress.port.get} """ else s""" 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 adada79382..313bc3b245 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartFirstSeedNodeSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartFirstSeedNodeSpec.scala @@ -56,7 +56,7 @@ abstract class RestartFirstSeedNodeSpec system.name, ConfigFactory.parseString( if (RARP(system).provider.remoteSettings.Artery.Enabled) - "akka.remote.artery.port=" + seedNodes.head.port.get + "akka.remote.artery.canonical.port=" + seedNodes.head.port.get else "akka.remote.netty.tcp.port=" + seedNodes.head.port.get ).withFallback(system.settings.config)) 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 652923602c..895b3fb5bf 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNode3Spec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNode3Spec.scala @@ -52,7 +52,7 @@ abstract class RestartNode3Spec system.name, ConfigFactory.parseString( if (RARP(system).provider.remoteSettings.Artery.Enabled) - "akka.remote.artery.port=" + secondUniqueAddress.address.port.get + "akka.remote.artery.canonical.port=" + secondUniqueAddress.address.port.get else "akka.remote.netty.tcp.port=" + secondUniqueAddress.address.port.get ).withFallback(system.settings.config)) From 3502f0d72f7fd2f7e2248b39c91cc267c3b91ff9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Fri, 9 Sep 2016 18:12:35 +0200 Subject: [PATCH 133/186] One more missed canonical.port in cluster tests (#21428) --- .../multi-jvm/scala/akka/cluster/ClusterDeathWatchSpec.scala | 2 +- .../scala/akka/cluster/UnreachableNodeJoinsAgainSpec.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 e6e38df16a..9c7903f9bc 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterDeathWatchSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterDeathWatchSpec.scala @@ -63,7 +63,7 @@ abstract class ClusterDeathWatchSpec pending } -/* "receive Terminated when watched node becomes Down/Removed" in within(20 seconds) { + /* "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/UnreachableNodeJoinsAgainSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/UnreachableNodeJoinsAgainSpec.scala index a6a0aacb3a..5b7d26e320 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/UnreachableNodeJoinsAgainSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/UnreachableNodeJoinsAgainSpec.scala @@ -165,7 +165,7 @@ abstract class UnreachableNodeJoinsAgainSpec ConfigFactory.parseString( if (RARP(system).provider.remoteSettings.Artery.Enabled) s""" - akka.remote.artery { + akka.remote.artery.canonical { hostname = ${victimAddress.host.get} port = ${victimAddress.port.get} } From bffbef0caa3ff379f5c9623103450581272770d2 Mon Sep 17 00:00:00 2001 From: Johannes Rudolph Date: Mon, 12 Sep 2016 14:20:37 +0200 Subject: [PATCH 134/186] =rem remove test-only HeaderBuilder.bothWays --- .../scala/akka/remote/artery/BufferPool.scala | 8 +--- .../remote/artery/EnvelopeBufferSpec.scala | 37 +++++++++++++------ 2 files changed, 26 insertions(+), 19 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index 745417daa1..0c1f138269 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -80,15 +80,9 @@ private[remote] object HeaderBuilder { 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]) - - /** INTERNAL API, FOR TESTING ONLY */ - private[remote] def bothWays( - in: InboundCompressions, - outboundActorRefCompression: CompressionTable[ActorRef], - outboundClassManifestCompression: CompressionTable[String]): HeaderBuilder = - new HeaderBuilderImpl(in, outboundActorRefCompression, outboundClassManifestCompression) } /** diff --git a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala index 644b321491..1bebe48629 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala @@ -8,6 +8,7 @@ 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 } @@ -47,8 +48,11 @@ class EnvelopeBufferSpec extends AkkaSpec { } "EnvelopeBuffer" must { - val headerIn = HeaderBuilder.bothWays(TestCompressor, TestCompressor.outboundActorRefTable, TestCompressor.outboundClassManifestTable) - val headerOut = HeaderBuilder.bothWays(TestCompressor, TestCompressor.outboundActorRefTable, TestCompressor.outboundClassManifestTable) + 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) @@ -83,18 +87,21 @@ class EnvelopeBufferSpec extends AkkaSpec { } "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 minimalRef("uncompressable0") - headerIn setRecipientActorRef minimalRef("uncompressable11") + headerIn setSenderActorRef senderRef + headerIn setRecipientActorRef recipientRef headerIn setManifest "uncompressable3333" val expectedHeaderLength = EnvelopeBuffer.LiteralsSectionOffset + // Constant header part - 2 + headerIn.senderActorRefPath.get.length + // Length field + literal - 2 + headerIn.recipientActorRefPath.get.length + // Length field + literal - 2 + headerIn.manifest(originUid).length // Length field + literal + 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) @@ -113,17 +120,19 @@ class EnvelopeBufferSpec extends AkkaSpec { } "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 minimalRef("uncompressable1") + headerIn setRecipientActorRef recipientRef headerIn setManifest "manifest1" envelope.writeHeader(headerIn) envelope.byteBuffer.position() should ===( EnvelopeBuffer.LiteralsSectionOffset + - 2 + headerIn.recipientActorRefPath.get.length) + 2 + lengthOfSerializedActorRefPath(recipientRef)) envelope.byteBuffer.flip() envelope.parseHeader(headerOut) @@ -137,18 +146,20 @@ class EnvelopeBufferSpec extends AkkaSpec { headerOut.recipientActorRef(originUid) should ===(OptionVal.None) headerOut.manifest(originUid) should ===("manifest1") + val senderRef = minimalRef("uncompressable0") + headerIn setVersion 3 headerIn setUid Long.MinValue headerIn setSerializer -1 - headerIn setSenderActorRef minimalRef("uncompressable0") + headerIn setSenderActorRef senderRef headerIn setRecipientActorRef minimalRef("reallylongcompressablestring") headerIn setManifest "longlonglongliteralmanifest" envelope.writeHeader(headerIn) envelope.byteBuffer.position() should ===( EnvelopeBuffer.LiteralsSectionOffset + - 2 + headerIn.senderActorRefPath.get.length + - 2 + headerIn.manifest(originUid).length) + 2 + lengthOfSerializedActorRefPath(senderRef) + + 2 + "longlonglongliteralmanifest".length) envelope.byteBuffer.flip() envelope.parseHeader(headerOut) @@ -193,4 +204,6 @@ class EnvelopeBufferSpec extends AkkaSpec { } + def lengthOfSerializedActorRefPath(ref: ActorRef): Int = + Serialization.serializedActorPath(ref).length } From 0e253b42085e88c26aae4931e5ce7b12c9a5474a Mon Sep 17 00:00:00 2001 From: Johannes Rudolph Date: Mon, 12 Sep 2016 14:21:44 +0200 Subject: [PATCH 135/186] =rem,act move direct buffer cleaning util method to akka.io --- .../scala/akka/io/DirectByteBufferPool.scala | 38 ++++++++++++------- .../scala/akka/remote/artery/BufferPool.scala | 28 ++------------ 2 files changed, 28 insertions(+), 38 deletions(-) diff --git a/akka-actor/src/main/scala/akka/io/DirectByteBufferPool.scala b/akka-actor/src/main/scala/akka/io/DirectByteBufferPool.scala index 3f68e35ba9..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-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index 0c1f138269..8871c14eda 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -7,14 +7,14 @@ package akka.remote.artery import java.nio.charset.Charset import java.nio.{ ByteBuffer, ByteOrder } -import akka.actor.{ ActorPath, ChildActorPath, ActorRef, Address } +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.{ OptionVal, Unsafe } -import scala.util.control.NonFatal import akka.remote.artery.compress.NoInboundCompressions /** @@ -420,27 +420,5 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { } } - /** - * DirectByteBuffers are garbage collected by using a phantom reference and a - * reference queue. Every once a while, the JVM checks the reference queue and - * cleans the DirectByteBuffers. However, as this doesn't happen - * immediately after discarding all references to a DirectByteBuffer, it's - * easy to OutOfMemoryError yourself using DirectByteBuffers. This function - * explicitly calls the Cleaner method of a DirectByteBuffer. - * - * Utilizes reflection to avoid dependency to `sun.misc.Cleaner`. - */ - def tryCleanDirectByteBuffer(): Unit = try { - if (byteBuffer.isDirect) { - val cleanerMethod = byteBuffer.getClass.getMethod("cleaner") - cleanerMethod.setAccessible(true) - val cleaner = cleanerMethod.invoke(byteBuffer) - val cleanMethod = cleaner.getClass.getMethod("clean") - cleanMethod.setAccessible(true) - cleanMethod.invoke(cleaner) - } - } catch { - case NonFatal(_) ⇒ // attempt failed, ok - } - + def tryCleanDirectByteBuffer(): Unit = DirectByteBufferPool.tryCleanDirectByteBuffer(byteBuffer) } From acafe80cf10577489b32638e5586596a0b250f88 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 12 Sep 2016 19:46:16 +0200 Subject: [PATCH 136/186] rewrite TestStage to use thread-safe shared state, #21431 * The previous approach was based on sending the test commands to the active stages themselves and let them keep track of the state. * The problem with that is that Association/OutboundTestStage that is created afterwards will not have the right state. Similar problems can occur for restarts. * Instead using thread-safe mutable state that is updated directly and used by all test stages. --- .../akka/remote/artery/ArteryTransport.scala | 87 ++++----- .../akka/remote/artery/Association.scala | 57 ++---- .../scala/akka/remote/artery/TestStage.scala | 169 +++++++----------- 3 files changed, 112 insertions(+), 201 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 39a59806c6..534fd2e593 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -6,9 +6,9 @@ package akka.remote.artery import java.io.File import java.net.InetSocketAddress import java.nio.channels.{ DatagramChannel, FileChannel } -import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.{ AtomicLong, AtomicReference } +import java.util.concurrent.atomic.AtomicBoolean import scala.annotation.tailrec import scala.collection.JavaConverters._ @@ -18,7 +18,9 @@ import scala.concurrent.duration._ import scala.util.Failure import scala.util.Success import scala.util.Try +import scala.util.control.NoStackTrace import scala.util.control.NonFatal + import akka.Done import akka.NotUsed import akka.actor._ @@ -43,18 +45,19 @@ 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.ActorMaterializerSettings 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.Helpers.ConfigOps -import akka.util.Helpers.Requiring import akka.util.OptionVal import akka.util.WildcardIndex import io.aeron.Aeron @@ -65,13 +68,10 @@ import io.aeron.UnavailableImageHandler 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.stream.scaladsl.BroadcastHub -import scala.util.control.NoStackTrace -import io.aeron.exceptions.DriverTimeoutException -import java.util.concurrent.atomic.AtomicBoolean /** * INTERNAL API @@ -281,9 +281,9 @@ private[remote] class FlushOnShutdown(done: Promise[Done], timeout: FiniteDurati */ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: RemoteActorRefProvider) extends RemoteTransport(_system, _provider) with InboundContext { - import FlightRecorderEvents._ - import ArteryTransport.ShutdownSignal import ArteryTransport.AeronTerminated + import ArteryTransport.ShutdownSignal + import FlightRecorderEvents._ // these vars are initialized once in the start method @volatile private[this] var _localAddress: UniqueAddress = _ @@ -312,7 +312,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private[this] val streamCompletions = new AtomicReference(Map.empty[String, Future[Done]]) @volatile private[this] var _shutdown = false - private val testStages: CopyOnWriteArrayList[TestManagementApi] = new CopyOnWriteArrayList + private val testState = new SharedTestState private val inboundLanes = settings.Advanced.InboundLanes @@ -579,15 +579,12 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundControlStream(compression: InboundCompressions): Unit = { - val (testMgmt, ctrl, completed) = + val (ctrl, completed) = aeronSource(controlStreamId, envelopeBufferPool) .via(inboundFlow(compression)) .toMat(inboundControlSink)(Keep.right) .run()(materializer) - if (settings.Advanced.TestMode) - testStages.add(testMgmt) - controlSubject = ctrl controlSubject.attach(new ControlMessageObserver { @@ -648,16 +645,11 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private def runInboundOrdinaryMessagesStream(compression: InboundCompressions): Unit = { val completed = if (inboundLanes == 1) { - val (testMgmt, completed) = aeronSource(ordinaryStreamId, envelopeBufferPool) + aeronSource(ordinaryStreamId, envelopeBufferPool) .via(inboundFlow(compression)) .toMat(inboundSink(envelopeBufferPool))(Keep.right) .run()(materializer) - if (settings.Advanced.TestMode) - testStages.add(testMgmt) - - completed - } else { val hubKillSwitch = KillSwitches.shared("hubKillSwitch") val source = aeronSource(ordinaryStreamId, envelopeBufferPool) @@ -677,7 +669,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } } - val values: Vector[(TestManagementApi, Future[Done])] = + val completedValues: Vector[Future[Done]] = (0 until inboundLanes).map { i ⇒ broadcastHub.runWith( // TODO replace filter with "PartitionHub" when that is implemented @@ -688,11 +680,6 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .toMat(lane)(Keep.right))(materializer) }(collection.breakOut) - val (testMgmtValues, completedValues) = values.unzip - - if (settings.Advanced.TestMode) - testMgmtValues.foreach(testStages.add) - import system.dispatcher val completed = Future.sequence(completedValues).map(_ ⇒ Done) @@ -711,14 +698,11 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private def runInboundLargeMessagesStream(): Unit = { val disableCompression = NoInboundCompressions // no compression on large message stream for now - val (testMgmt, completed) = aeronSource(largeStreamId, largeEnvelopeBufferPool) + val completed = aeronSource(largeStreamId, largeEnvelopeBufferPool) .via(inboundLargeFlow(disableCompression)) .toMat(inboundSink(largeEnvelopeBufferPool))(Keep.right) .run()(materializer) - if (settings.Advanced.TestMode) - testStages.add(testMgmt) - attachStreamRestart("Inbound large message stream", completed, () ⇒ runInboundLargeMessagesStream()) } @@ -811,14 +795,13 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private[remote] def isShutdown: Boolean = _shutdown override def managementCommand(cmd: Any): Future[Boolean] = { - if (testStages.isEmpty) - Future.successful(false) - else { - import system.dispatcher - import scala.collection.JavaConverters._ - val allTestStages = testStages.asScala.toVector ++ associationRegistry.allAssociations.flatMap(_.testStages) - Future.sequence(allTestStages.map(_.send(cmd))).map(_ ⇒ true) + 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 @@ -898,7 +881,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .viaMat(createEncoder(bufferPool))(Keep.right) } - def outboundControl(outboundContext: OutboundContext): Sink[OutboundEnvelope, (TestManagementApi, OutboundControlIngress, Future[Done])] = { + def outboundControl(outboundContext: OutboundContext): Sink[OutboundEnvelope, (OutboundControlIngress, Future[Done])] = { Flow.fromGraph(killSwitch.flow[OutboundEnvelope]) .via(new OutboundHandshake(system, outboundContext, outboundEnvelopePool, settings.Advanced.HandshakeTimeout, @@ -906,14 +889,11 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .via(new SystemMessageDelivery(outboundContext, system.deadLetters, settings.Advanced.SystemMessageResendInterval, settings.Advanced.SysMsgBufferSize)) // note that System messages must not be dropped before the SystemMessageDelivery stage - .viaMat(outboundTestFlow(outboundContext))(Keep.right) - .viaMat(new OutboundControlJunction(outboundContext, outboundEnvelopePool))(Keep.both) + .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) - .mapMaterializedValue { - case ((a, b), c) ⇒ (a, b, c) - } // TODO we can also add scrubbing stage that would collapse sys msg acks/nacks and remove duplicate Quarantine messages } @@ -944,13 +924,13 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R def createDeserializer(bufferPool: EnvelopeBufferPool): Flow[InboundEnvelope, InboundEnvelope, NotUsed] = Flow.fromGraph(new Deserializer(this, system, bufferPool)) - def inboundSink(bufferPool: EnvelopeBufferPool): Sink[InboundEnvelope, (TestManagementApi, Future[Done])] = + def inboundSink(bufferPool: EnvelopeBufferPool): Sink[InboundEnvelope, Future[Done]] = Flow[InboundEnvelope] .via(createDeserializer(bufferPool)) - .viaMat(new InboundTestStage(this, settings.Advanced.TestMode))(Keep.right) + .via(new InboundTestStage(this, testState, settings.Advanced.TestMode)) .via(new InboundHandshake(this, inControlStream = false)) .via(new InboundQuarantineCheck(this)) - .toMat(messageDispatcherSink)(Keep.both) + .toMat(messageDispatcherSink)(Keep.right) def inboundFlow(compression: InboundCompressions): Flow[EnvelopeBuffer, InboundEnvelope, NotUsed] = { Flow[EnvelopeBuffer] @@ -964,18 +944,15 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .via(createDecoder(compression, largeEnvelopeBufferPool)) } - def inboundControlSink: Sink[InboundEnvelope, (TestManagementApi, ControlMessageSubject, Future[Done])] = { + def inboundControlSink: Sink[InboundEnvelope, (ControlMessageSubject, Future[Done])] = { Flow[InboundEnvelope] .via(createDeserializer(envelopeBufferPool)) - .viaMat(new InboundTestStage(this, settings.Advanced.TestMode))(Keep.right) + .via(new InboundTestStage(this, testState, settings.Advanced.TestMode)) .via(new InboundHandshake(this, inControlStream = true)) .via(new InboundQuarantineCheck(this)) - .viaMat(new InboundControlJunction)(Keep.both) + .viaMat(new InboundControlJunction)(Keep.right) .via(new SystemMessageAcker(this)) .toMat(messageDispatcherSink)(Keep.both) - .mapMaterializedValue { - case ((a, b), c) ⇒ (a, b, c) - } } private def initializeFlightRecorder(): Option[(FileChannel, File, FlightRecorder)] = { @@ -990,8 +967,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R None } - def outboundTestFlow(outboundContext: OutboundContext): Flow[OutboundEnvelope, OutboundEnvelope, TestManagementApi] = - Flow.fromGraph(new OutboundTestStage(outboundContext, settings.Advanced.TestMode)) + 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) = { diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 6894e02e49..3f39dd575b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -4,9 +4,9 @@ package akka.remote.artery import java.util.Queue -import java.util.concurrent.CopyOnWriteArrayList 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 @@ -21,34 +21,29 @@ 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 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.SendQueue.ProducerApi 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.remote.artery.compress.CompressionProtocol._ -import akka.stream.scaladsl.MergeHub -import akka.actor.DeadLetter -import java.util.concurrent.atomic.AtomicBoolean -import akka.stream.KillSwitches -import scala.util.Failure -import scala.util.Success -import akka.remote.artery.ArteryTransport.AeronTerminated /** * INTERNAL API @@ -168,19 +163,11 @@ private[remote] class Association( } private def timeoutAfter[T](f: Future[T], timeout: FiniteDuration, e: ⇒ Throwable): Future[T] = { - import akka.pattern.after import transport.system.dispatcher val f2 = after(timeout, transport.system.scheduler)(Future.failed(e)) Future.firstCompletedOf(List(f, f2)) } - private val _testStages: CopyOnWriteArrayList[TestManagementApi] = new CopyOnWriteArrayList - - def testStages(): List[TestManagementApi] = { - import scala.collection.JavaConverters._ - _testStages.asScala.toList - } - private def deadletters = transport.system.deadLetters def outboundControlIngress: OutboundControlIngress = { @@ -265,8 +252,10 @@ private[remote] class Association( } // OutboundContext - override def sendControl(message: ControlMessage): Unit = - outboundControlIngress.sendControlMessage(message) + override def sendControl(message: ControlMessage): Unit = { + if (!transport.isShutdown) + outboundControlIngress.sendControlMessage(message) + } def send(message: Any, sender: OptionVal[ActorRef], recipient: OptionVal[RemoteActorRef]): Unit = { @@ -434,14 +423,11 @@ private[remote] class Association( queues(ControlQueueIndex) = wrapper // use new underlying queue immediately for restarts queuesVisibility = true // volatile write for visibility of the queues array - val (queueValue, (testMgmt, control, completed)) = + val (queueValue, (control, completed)) = Source.fromGraph(new SendQueue[OutboundEnvelope]) .toMat(transport.outboundControl(this))(Keep.both) .run()(materializer) - if (advancedSettings.TestMode) - _testStages.add(testMgmt) - queueValue.inject(wrapper.queue) // replace with the materialized value, still same underlying queue queues(ControlQueueIndex) = queueValue @@ -475,9 +461,6 @@ private[remote] class Association( .toMat(transport.outbound(this))(Keep.both) .run()(materializer) - if (advancedSettings.TestMode) - _testStages.add(testMgmt) - queueValue.inject(wrapper.queue) // replace with the materialized value, still same underlying queue queues(queueIndex) = queueValue @@ -499,30 +482,25 @@ private[remote] class Association( val lane = Source.fromGraph(new SendQueue[OutboundEnvelope]) .via(laneKillSwitch.flow) - .viaMat(transport.outboundTestFlow(this))(Keep.both) + .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, m), c), w) ⇒ ((q, m), (c, w)) + case ((q, c), w) ⇒ (q, c, w) } val (mergeHub, aeronSinkCompleted) = MergeHub.source[EnvelopeBuffer] .via(laneKillSwitch.flow) .toMat(transport.aeronSink(this))(Keep.both).run()(materializer) - val values: Vector[((SendQueue.QueueValue[OutboundEnvelope], TestManagementApi), (Encoder.ChangeOutboundCompression, Future[Done]))] = + val values: Vector[(SendQueue.QueueValue[OutboundEnvelope], Encoder.ChangeOutboundCompression, Future[Done])] = (0 until outboundLanes).map { _ ⇒ lane.to(mergeHub).run()(materializer) }(collection.breakOut) - val (a, b) = values.unzip - val (queueValues, testMgmtValues) = a.unzip - val (changeCompressionValues, laneCompletedValues) = b.unzip - - if (advancedSettings.TestMode) - testMgmtValues.foreach(_testStages.add) + val (queueValues, changeCompressionValues, laneCompletedValues) = values.unzip3 import transport.system.dispatcher val completed = Future.sequence(laneCompletedValues).flatMap(_ ⇒ aeronSinkCompleted) @@ -552,14 +530,11 @@ private[remote] class Association( queues(LargeQueueIndex) = wrapper // use new underlying queue immediately for restarts queuesVisibility = true // volatile write for visibility of the queues array - val ((queueValue, testMgmt), completed) = Source.fromGraph(new SendQueue[OutboundEnvelope]) - .viaMat(transport.outboundTestFlow(this))(Keep.both) + val (queueValue, completed) = Source.fromGraph(new SendQueue[OutboundEnvelope]) + .via(transport.outboundTestFlow(this)) .toMat(transport.outboundLarge(this))(Keep.both) .run()(materializer) - if (advancedSettings.TestMode) - _testStages.add(testMgmt) - queueValue.inject(wrapper.queue) // replace with the materialized value, still same underlying queue queues(LargeQueueIndex) = queueValue diff --git a/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala b/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala index 45e975e0e3..a71bb341c0 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala @@ -3,105 +3,99 @@ */ package akka.remote.artery -import scala.concurrent.ExecutionContext -import scala.concurrent.Future -import scala.concurrent.Promise +import java.util.concurrent.atomic.AtomicReference + +import scala.annotation.tailrec import scala.concurrent.duration._ -import akka.Done + import akka.actor.Address -import akka.remote.transport.ThrottlerTransportAdapter.Blackhole import akka.remote.transport.ThrottlerTransportAdapter.Direction -import akka.remote.transport.ThrottlerTransportAdapter.SetThrottle -import akka.remote.transport.ThrottlerTransportAdapter.Unthrottled import akka.stream.Attributes import akka.stream.FlowShape import akka.stream.Inlet import akka.stream.Outlet -import akka.stream.stage.AsyncCallback -import akka.stream.stage.CallbackWrapper -import akka.stream.stage.GraphStageWithMaterializedValue +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.stream.stage.GraphStageLogic /** - * INTERNAL API + * INTERNAL API: Thread safe mutable state that is shared among + * the test stages. */ -private[remote] trait TestManagementApi { - def send(command: Any)(implicit ec: ExecutionContext): Future[Done] -} +private[remote] class SharedTestState { + private val state = new AtomicReference[TestState](TestState(Map.empty)) -/** - * INTERNAL API - */ -private[remote] class TestManagementApiImpl(stopped: Future[Done], callback: AsyncCallback[TestManagementMessage]) - extends TestManagementApi { - - override def send(command: Any)(implicit ec: ExecutionContext): Future[Done] = { - if (stopped.isCompleted) - Future.successful(Done) - else { - val done = Promise[Done]() - callback.invoke(TestManagementMessage(command, done)) - Future.firstCompletedOf(List(done.future, stopped)) + 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) } -} -private[remote] class DisabledTestManagementApi extends TestManagementApi { - override def send(command: Any)(implicit ec: ExecutionContext): Future[Done] = - Future.failed(new RuntimeException("TestStage is disabled, enable with MultiNodeConfig.testTransport")) } /** * INTERNAL API */ -private[remote] final case class TestManagementMessage(command: Any, done: Promise[Done]) +private[remote] final case class TestState(blackholes: Map[Address, Set[Address]]) /** * INTERNAL API */ -private[remote] class OutboundTestStage(outboundContext: OutboundContext, enabled: Boolean) - extends GraphStageWithMaterializedValue[FlowShape[OutboundEnvelope, OutboundEnvelope], TestManagementApi] { +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 createLogicAndMaterializedValue(inheritedAttributes: Attributes) = { + override def createLogic(inheritedAttributes: Attributes) = { if (enabled) { - val stoppedPromise = Promise[Done]() - - // FIXME see issue #20503 related to CallbackWrapper, we might implement this in a better way - val logic = new TimerGraphStageLogic(shape) with CallbackWrapper[TestManagementMessage] with InHandler with OutHandler with StageLogging { - - private var blackhole = Set.empty[Address] - - private val callback = getAsyncCallback[TestManagementMessage] { - case TestManagementMessage(command, done) ⇒ - command match { - case SetThrottle(address, Direction.Send | Direction.Both, Blackhole) ⇒ - log.info("blackhole outbound messages to {}", address) - blackhole += address - case SetThrottle(address, Direction.Send | Direction.Both, Unthrottled) ⇒ - log.info("accept outbound messages to {}", address) - blackhole -= address - case _ ⇒ // not interested - } - done.success(Done) - } - - override def preStart(): Unit = { - initCallback(callback.invoke) - } - - override def postStop(): Unit = stoppedPromise.success(Done) + new TimerGraphStageLogic(shape) with InHandler with OutHandler with StageLogging { // InHandler override def onPush(): Unit = { val env = grab(in) - if (blackhole(outboundContext.remoteAddress)) { + if (state.isBlackhole(outboundContext.localAddress.address, outboundContext.remoteAddress)) { log.debug( "dropping outbound message [{}] to [{}] because of blackhole", env.message.getClass.getName, outboundContext.remoteAddress) @@ -115,17 +109,12 @@ private[remote] class OutboundTestStage(outboundContext: OutboundContext, enable setHandlers(in, out, this) } - - val managementApi: TestManagementApi = new TestManagementApiImpl(stoppedPromise.future, logic) - - (logic, managementApi) } else { - val logic = new GraphStageLogic(shape) with InHandler with OutHandler { + 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) } - (logic, new DisabledTestManagementApi) } } @@ -134,40 +123,15 @@ private[remote] class OutboundTestStage(outboundContext: OutboundContext, enable /** * INTERNAL API */ -private[remote] class InboundTestStage(inboundContext: InboundContext, enabled: Boolean) - extends GraphStageWithMaterializedValue[FlowShape[InboundEnvelope, InboundEnvelope], TestManagementApi] { +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 createLogicAndMaterializedValue(inheritedAttributes: Attributes) = { + override def createLogic(inheritedAttributes: Attributes) = { if (enabled) { - val stoppedPromise = Promise[Done]() - - // FIXME see issue #20503 related to CallbackWrapper, we might implement this in a better way - val logic = new TimerGraphStageLogic(shape) with CallbackWrapper[TestManagementMessage] with InHandler with OutHandler with StageLogging { - - private var blackhole = Set.empty[Address] - - private val callback = getAsyncCallback[TestManagementMessage] { - case TestManagementMessage(command, done) ⇒ - command match { - case SetThrottle(address, Direction.Receive | Direction.Both, Blackhole) ⇒ - log.info("blackhole inbound messages from {}", address) - blackhole += address - case SetThrottle(address, Direction.Receive | Direction.Both, Unthrottled) ⇒ - log.info("accept inbound messages from {}", address) - blackhole -= address - case _ ⇒ // not interested - } - done.success(Done) - } - - override def preStart(): Unit = { - initCallback(callback.invoke) - } - - override def postStop(): Unit = stoppedPromise.success(Done) + new TimerGraphStageLogic(shape) with InHandler with OutHandler with StageLogging { // InHandler override def onPush(): Unit = { @@ -177,7 +141,7 @@ private[remote] class InboundTestStage(inboundContext: InboundContext, enabled: // unknown, handshake not completed push(out, env) case OptionVal.Some(association) ⇒ - if (blackhole(association.remoteAddress)) { + if (state.isBlackhole(inboundContext.localAddress.address, association.remoteAddress)) { log.debug( "dropping inbound message [{}] from [{}] with UID [{}] because of blackhole", env.message.getClass.getName, association.remoteAddress, env.originUid) @@ -192,17 +156,12 @@ private[remote] class InboundTestStage(inboundContext: InboundContext, enabled: setHandlers(in, out, this) } - - val managementApi: TestManagementApi = new TestManagementApiImpl(stoppedPromise.future, logic) - - (logic, managementApi) } else { - val logic = new GraphStageLogic(shape) with InHandler with OutHandler { + 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) } - (logic, new DisabledTestManagementApi) } } From bf151e979385928f3c9783982f16a1d27bc30bb8 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 13 Sep 2016 08:01:58 +0200 Subject: [PATCH 137/186] don't quarantine back, #21450 * Don't quarantine the other system when receiving the Quarantined message, 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. --- .../cluster/SurviveNetworkInstabilitySpec.scala | 15 ++++----------- .../akka/remote/artery/ArteryTransport.scala | 6 ++++-- 2 files changed, 8 insertions(+), 13 deletions(-) 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 09392afe35..f69be2b65b 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/SurviveNetworkInstabilitySpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/SurviveNetworkInstabilitySpec.scala @@ -366,17 +366,10 @@ abstract class SurviveNetworkInstabilitySpec } runOn(side2: _*) { - if (RARP(system).provider.remoteSettings.Artery.Enabled) { - // with artery the other side stays quarantined - val expected = (side2 map address).toSet - clusterView.members.map(_.address) should ===(expected) - - } else { - // with the old remoting side2 comes back but stays unreachable - val expected = ((side2 ++ side1) map address).toSet - clusterView.members.map(_.address) should ===(expected) - assertUnreachable(side1: _*) - } + // side2 comes back but stays unreachable + val expected = ((side2 ++ side1) map address).toSet + clusterView.members.map(_.address) should ===(expected) + assertUnreachable(side1: _*) } enterBarrier("after-7") diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 534fd2e593..4e6ce2efa0 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -623,10 +623,12 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } 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) publishLifecycleEvent(lifecycleEvent) - // quarantine the other system from here - association(from.address).quarantine(lifecycleEvent.toString, Some(from.uid)) case _: ActorSystemTerminating ⇒ inboundEnvelope.sender match { From 211537f485bea8a32293825d50127a14c52db5b6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Dubiel?= Date: Tue, 13 Sep 2016 10:48:37 +0200 Subject: [PATCH 138/186] =art Count min sketch benchmark and performance improvments. (#21037) --- .../compress/CountMinSketchBenchmark.scala | 50 +++ .../artery/compress/CountMinSketch.java | 357 ++++++------------ .../artery/compress/InboundCompressions.scala | 2 +- 3 files changed, 175 insertions(+), 234 deletions(-) create mode 100644 akka-bench-jmh/src/main/scala/akka/remote/artery/compress/CountMinSketchBenchmark.scala 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-remote/src/main/java/akka/remote/artery/compress/CountMinSketch.java b/akka-remote/src/main/java/akka/remote/artery/compress/CountMinSketch.java index 6041cb8c78..7cd729da60 100644 --- a/akka-remote/src/main/java/akka/remote/artery/compress/CountMinSketch.java +++ b/akka-remote/src/main/java/akka/remote/artery/compress/CountMinSketch.java @@ -6,35 +6,32 @@ package akka.remote.artery.compress; import akka.actor.ActorRef; -import java.util.Random; - /** * 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 { - public static final long PRIME_MODULUS = (1L << 31) - 1; - private int depth; private int width; private long[][] table; - private long[] hashA; 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; @@ -43,46 +40,14 @@ public class CountMinSketch { initTablesWith(depth, width, seed); } - @SuppressWarnings("unused") - public CountMinSketch(double epsOfTotalCount, double confidence, int seed) { - // 2/w = eps ; w = 2/eps - // 1/2^depth <= 1-confidence ; depth >= -log2 (1-confidence) - this.eps = epsOfTotalCount; - this.confidence = confidence; - this.width = (int) Math.ceil(2 / epsOfTotalCount); - this.depth = (int) Math.ceil(-Math.log(1 - confidence) / Math.log(2)); - recyclableCMSHashBuckets = preallocateHashBucketsArray(depth); - initTablesWith(depth, width, seed); - } - - @SuppressWarnings("unused") - public CountMinSketch(int depth, int width, int size, long[] hashA, long[][] table) { - this.depth = depth; - this.width = width; - this.eps = 2.0 / width; - this.confidence = 1 - 1 / Math.pow(2, depth); - this.hashA = hashA; - this.table = table; - this.size = size; - recyclableCMSHashBuckets = preallocateHashBucketsArray(depth); - } private void initTablesWith(int depth, int width, int seed) { this.table = new long[depth][width]; - this.hashA = new long[depth]; - Random r = new Random(seed); - // We're using a linear hash functions - // of the form (a*x+b) mod p. - // a,b are chosen independently for each hash function. - // However we can set b = 0 as all it does is shift the results - // without compromising their uniformity or independence with - // the other hashes. - for (int i = 0; i < depth; ++i) { - hashA[i] = r.nextInt(Integer.MAX_VALUE); - } } - /** Referred to as {@code epsilon} in the whitepaper */ + /** + * Referred to as {@code epsilon} in the whitepaper + */ public double relativeError() { return eps; } @@ -91,46 +56,6 @@ public class CountMinSketch { return confidence; } - private int hash(long item, int i) { - long hash = hashA[i] * item; - // A super fast way of computing x mod 2^p-1 - // See http://www.cs.princeton.edu/courses/archive/fall09/cos521/Handouts/universalclasses.pdf - // page 149, right after Proposition 7. - hash += hash >> 32; - hash &= PRIME_MODULUS; - // Doing "%" after (int) conversion is ~2x faster than %'ing longs. - return ((int) hash) % width; - } - - public void add(long 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"); - } - for (int i = 0; i < depth; ++i) { - table[i][hash(item, i)] += count; - } - size += count; - } - - public void addObject(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"); - } - MurmurHash.hashBuckets(item, recyclableCMSHashBuckets, width); - for (int i = 0; i < depth; ++i) { - table[i][recyclableCMSHashBuckets[i]] += count; - } - size += count; - } - /** * 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. @@ -143,34 +68,31 @@ public class CountMinSketch { // parameter to constructor. throw new IllegalArgumentException("Negative increments not implemented"); } - MurmurHash.hashBuckets(item, recyclableCMSHashBuckets, width); + 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(long item) { - long res = Long.MAX_VALUE; - for (int i = 0; i < depth; ++i) { - res = Math.min(res, table[i][hash(item, i)]); - } - return res; - } /** * 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) { @@ -180,173 +102,142 @@ public class CountMinSketch { } return res; } - + + /** - * This is copied from https://github.com/addthis/stream-lib/blob/master/src/main/java/com/clearspring/analytics/hash/MurmurHash.java - * Which is Apache V2 licensed. - *

- * This is a very fast, non-cryptographic hash suitable for general hash-based - * lookup. See http://murmurhash.googlepages.com/ for more details. - *

- *

- * The C version of MurmurHash 2.0 found at that site was ported to Java by - * Andrzej Bialecki (ab at getopt org). - *

+ * 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 */ - // TODO replace with Scala's Murmur3, it's much faster - private static class MurmurHash { - + 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, + // 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); + return hashLong((Long) o, 0); } if (o instanceof Integer) { - return hashLong((Integer) o); + return hashLong((Integer) o, 0); } if (o instanceof Double) { - return hashLong(Double.doubleToRawLongBits((Double) o)); + return hashLong(Double.doubleToRawLongBits((Double) o), 0); } if (o instanceof Float) { - return hashLong(Float.floatToRawIntBits((Float) o)); + return hashLong(Float.floatToRawIntBits((Float) o), 0); } if (o instanceof byte[]) { - return hash((byte[]) o); + return bytesHash((byte[]) o, 0); } return hash(o.toString()); } - - public static int hash(byte[] data) { - return hash(data, data.length, -1); + + 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); } - - public static int hash(byte[] data, int seed) { - return hash(data, data.length, seed); - } - - public static int hash(byte[] data, int length, int seed) { - int m = 0x5bd1e995; - int r = 24; - - int h = seed ^ length; - - int len_4 = length >> 2; - - for (int i = 0; i < len_4; i++) { - int i_4 = i << 2; - int k = data[i_4 + 3]; - k = k << 8; - k = k | (data[i_4 + 2] & 0xff); - k = k << 8; - k = k | (data[i_4 + 1] & 0xff); - k = k << 8; - k = k | (data[i_4 + 0] & 0xff); - k *= m; - k ^= k >>> r; - k *= m; - h *= m; - h ^= k; + + 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; } - - // avoid calculating modulo - int len_m = len_4 << 2; - int left = length - len_m; - - if (left != 0) { - if (left >= 3) { - h ^= (int) data[length - 3] << 16; - } - if (left >= 2) { - h ^= (int) data[length - 2] << 8; - } - if (left >= 1) { - h ^= (int) data[length - 1]; - } - - h *= m; + + // 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); } - - h ^= h >>> 13; - h *= m; - h ^= h >>> 15; - - return h; + + // Finalization + return avalanche(h ^ data.length); } - - public static int hashLong(long data) { - return hashLong(data, 0); - } - public static int hashLong(long data, int seed) { - int m = 0x5bd1e995; - int r = 24; - - int h = seed; - // int h = seed ^ length; - - int k = (int) data * m; - k ^= k >>> r; - h ^= k * m; - - k = (int) (data >> 32) * m; - k ^= k >>> r; - h *= m; - h ^= k * m; - - h ^= h >>> 13; - h *= m; - h ^= h >>> 15; - - return h; - } - - // Murmur is faster than an SHA-based approach and provides as-good collision - // resistance. The combinatorial generation approach described in - // http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf - // does prove to work in actual tests, and is obviously faster - // than performing further iterations of murmur. -// public static int[] hashBuckets(String key, int hashCount, int max) { -// byte[] b; -// try { -// b = key.getBytes("UTF-16");// TODO Use the Unsafe trick @patriknw used to access the backing array directly -- via Endre -// } catch (UnsupportedEncodingException e) { -// throw new RuntimeException(e); -// } -// return hashBuckets(b, hashCount, max); -// } - -// static int[] hashBuckets(byte[] b, int hashCount, int max) { -// // TODO we could reuse the arrays -// int[] result = preallocateHashBucketsArray(hashCount); -// int hash1 = hash(b, b.length, 0); -// int hash2 = hash(b, b.length, hash1); -// for (int i = 0; i < hashCount; i++) { -// result[i] = Math.abs((hash1 + i * hash2) % max); -// } -// return result; -// } - - /** Mutates passed in {@code hashBuckets} */ - static void hashBuckets(Object item, int[] hashBuckets, int max) { - int hash1 = hash(item); // specialized hash for ActorRef and Strings - int hash2 = hashLong(hash1, hash1); + + /** + * 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; - for (int i = 0; i < depth; i++) - hashBuckets[i] = Math.abs((hash1 + i * hash2) % max); + 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 + } } + } - public int[] preallocateHashBucketsArray(int depth) { + private int[] preallocateHashBucketsArray(int depth) { return new int[depth]; } 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 index b55c051dba..136bedbe76 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala @@ -205,7 +205,7 @@ private[remote] abstract class InboundCompression[T >: Null]( private[this] val state: AtomicReference[InboundCompression.State[T]] = new AtomicReference(InboundCompression.State.empty) // TODO calibrate properly (h/w have direct relation to preciseness and max capacity) - private[this] val cms = new CountMinSketch(100, 100, System.currentTimeMillis().toInt) + private[this] val cms = new CountMinSketch(16, 1024, System.currentTimeMillis().toInt) /* ==== COMPRESSION ==== */ From 3b57947b1f85b83f36bbf86237dcffc8663f826a Mon Sep 17 00:00:00 2001 From: Konrad Malawski Date: Tue, 13 Sep 2016 10:49:19 +0200 Subject: [PATCH 139/186] =art remove addressed todo in compressions --- .../scala/akka/remote/artery/compress/InboundCompressions.scala | 1 - 1 file changed, 1 deletion(-) 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 index 136bedbe76..01d7928b7a 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala @@ -204,7 +204,6 @@ private[remote] abstract class InboundCompression[T >: Null]( private[this] val state: AtomicReference[InboundCompression.State[T]] = new AtomicReference(InboundCompression.State.empty) - // TODO calibrate properly (h/w have direct relation to preciseness and max capacity) private[this] val cms = new CountMinSketch(16, 1024, System.currentTimeMillis().toInt) /* ==== COMPRESSION ==== */ From 848d56cc2f5456ba6a390ee8e7adcecc4644c098 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andre=CC=81n?= Date: Wed, 14 Sep 2016 11:40:42 +0200 Subject: [PATCH 140/186] More tests working on artery * non-multi-jvm tests from akka-cluster * akka-cluster-metrics * akka-cluster-tools * akka-cluster-sharding --- .../scala/akka/cluster/metrics/TestUtil.scala | 5 +- .../cluster/client/ClusterClientSpec.scala | 1 + .../pubsub/DistributedPubSubRestartSpec.scala | 16 ++++-- .../singleton/ClusterSingletonProxySpec.scala | 42 +++++++-------- .../ClusterSingletonRestartSpec.scala | 21 ++++++-- .../scala/akka/cluster/AutoDownSpec.scala | 19 ++++--- .../ClusterDomainEventPublisherSpec.scala | 15 ++++-- .../cluster/routing/WeightedRouteesSpec.scala | 13 +++-- .../cluster/ddata/WriteAggregatorSpec.scala | 7 ++- .../akka/remote/RemoteDeathWatchSpec.scala | 12 +++-- .../scala/akka/remote/RemoteRouterSpec.scala | 51 ++++++++++--------- 11 files changed, 127 insertions(+), 75 deletions(-) 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 243915a8a8..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) { } 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 26d549e40c..8471231ff6 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 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/test/scala/akka/cluster/singleton/ClusterSingletonProxySpec.scala b/akka-cluster-tools/src/test/scala/akka/cluster/singleton/ClusterSingletonProxySpec.scala index 0c32cd051a..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 = "cluster" - - 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/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/ClusterDomainEventPublisherSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala index c3232097df..be3db51745 100644 --- a/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala @@ -17,6 +17,7 @@ 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 { @@ -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/routing/WeightedRouteesSpec.scala b/akka-cluster/src/test/scala/akka/cluster/routing/WeightedRouteesSpec.scala index 49ae6f32fc..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,6 +7,7 @@ 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 @@ -16,10 +17,14 @@ class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString(""" 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/test/scala/akka/cluster/ddata/WriteAggregatorSpec.scala b/akka-distributed-data/src/test/scala/akka/cluster/ddata/WriteAggregatorSpec.scala index 9b8cb45a9d..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 { @@ -55,7 +56,11 @@ class WriteAggregatorSpec extends AkkaSpec(""" """) 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-remote/src/test/scala/akka/remote/RemoteDeathWatchSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteDeathWatchSpec.scala index 2528408023..fcdeab7b30 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteDeathWatchSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteDeathWatchSpec.scala @@ -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/RemoteRouterSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala index 1b51e68455..43976d8aa0 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala @@ -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 { From 9acb1feba3bd4b0b55a023a1cc673980323f0f54 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andre=CC=81n?= Date: Fri, 16 Sep 2016 12:29:49 +0200 Subject: [PATCH 141/186] ClusterClientSpec workin with artery #21477 --- .../scala/akka/cluster/client/ClusterClientSpec.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) 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 8471231ff6..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 @@ -431,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) From 2acfb8cfac34a2f0a003f35c2b284cf6a9bd9f67 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Endre=20S=C3=A1ndor=20Varga?= Date: Fri, 16 Sep 2016 11:47:29 +0200 Subject: [PATCH 142/186] #21466 Wrong protocol name expected in tests when using Artery --- .../ddata/protobuf/ReplicatedDataSerializerSpec.scala | 9 ++++++--- .../ddata/protobuf/ReplicatorMessageSerializerSpec.scala | 9 ++++++--- 2 files changed, 12 insertions(+), 6 deletions(-) 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 e96cca39e1..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,6 +23,7 @@ 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( @@ -34,9 +35,11 @@ class ReplicatedDataSerializerSpec extends TestKit(ActorSystem( 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 314e1f34b9..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,6 +21,7 @@ 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( @@ -32,9 +33,11 @@ class ReplicatorMessageSerializerSpec extends TestKit(ActorSystem( 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") From 835125de3d250fabbbf3596f1b3c92047e23d8dd Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 14 Sep 2016 09:57:44 +0200 Subject: [PATCH 143/186] make cluster.StressSpec pass with Artery, #21458 * need to use a shared media driver to get the cpu usage at a reasonable level * also changed to SleepingIdleStrategy(1 ms) when cpu-level=1 not needed for the test to pass, but can be good to make level 1 more extreme --- .../akka/cluster/MultiNodeClusterSpec.scala | 6 +- .../cluster/SharedMediaDriverSupport.scala | 105 +++++ .../scala/akka/cluster/StressSpec.scala | 415 +++++++++--------- .../akka/remote/testkit/MultiNodeSpec.scala | 2 +- .../akka/remote/artery/ArteryTransport.scala | 2 +- .../scala/akka/remote/artery/TaskRunner.scala | 25 +- 6 files changed, 344 insertions(+), 211 deletions(-) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/SharedMediaDriverSupport.scala 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 ed5ac063ff..a819090154 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala @@ -83,12 +83,14 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec with WatchedByCoro 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() } 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..925bfbc4d5 --- /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.ImageLivenessTimeoutNs.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 47d2fff641..1cbe452bcf 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 @@ -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 @@ -1129,200 +1153,199 @@ abstract class StressSpec "A cluster under stress" must { - "TODO work with artery" in (pending) - // "log settings" taggedAs LongRunningTest in { - // if (infolog) { - // log.info("StressSpec JVM:\n{}", jvmInfo) - // runOn(roles.head) { - // log.info("StressSpec settings:\n{}", settings) - // } - // } - // enterBarrier("after-" + step) - // } - // - // "join seed nodes" taggedAs LongRunningTest in within(30 seconds) { - // - // val otherNodesJoiningSeedNodes = roles.slice(numberOfSeedNodes, numberOfSeedNodes + numberOfNodesJoiningToSeedNodesInitially) - // val size = seedNodes.size + otherNodesJoiningSeedNodes.size - // - // createResultAggregator("join seed nodes", expectedResults = size, includeInHistory = true) - // - // runOn((seedNodes ++ otherNodesJoiningSeedNodes): _*) { - // reportResult { - // cluster.joinSeedNodes(seedNodes.toIndexedSeq map address) - // awaitMembersUp(size, timeout = remainingOrDefault) - // } - // } - // - // awaitClusterResult() - // - // nbrUsedRoles += size - // enterBarrier("after-" + step) - // } - // - // "start routers that are running while nodes are joining" taggedAs LongRunningTest in { - // runOn(roles.take(3): _*) { - // system.actorOf( - // Props(classOf[Master], settings, settings.workBatchInterval, false).withDeploy(Deploy.local), - // name = masterName) ! Begin - // } - // } - // - // "join nodes one-by-one to small cluster" taggedAs LongRunningTest in { - // joinOneByOne(numberOfNodesJoiningOneByOneSmall) - // enterBarrier("after-" + step) - // } - // - // "join several nodes to one node" taggedAs LongRunningTest in { - // joinSeveral(numberOfNodesJoiningToOneNode, toSeedNodes = false) - // nbrUsedRoles += numberOfNodesJoiningToOneNode - // enterBarrier("after-" + step) - // } - // - // "join several nodes to seed nodes" taggedAs LongRunningTest in { - // if (numberOfNodesJoiningToSeedNodes > 0) { - // joinSeveral(numberOfNodesJoiningToSeedNodes, toSeedNodes = true) - // nbrUsedRoles += numberOfNodesJoiningToSeedNodes - // } - // enterBarrier("after-" + step) - // } - // - // "join nodes one-by-one to large cluster" taggedAs LongRunningTest in { - // joinOneByOne(numberOfNodesJoiningOneByOneLarge) - // enterBarrier("after-" + step) - // } - // - // "end routers that are running while nodes are joining" taggedAs LongRunningTest in within(30.seconds) { - // if (exerciseActors) { - // runOn(roles.take(3): _*) { - // master match { - // case Some(m) ⇒ - // m.tell(End, testActor) - // val workResult = awaitWorkResult(m) - // workResult.retryCount should ===(0) - // workResult.sendCount should be > (0L) - // workResult.ackCount should be > (0L) - // case None ⇒ fail("master not running") - // } - // } - // } - // enterBarrier("after-" + step) - // } - // - // "use routers with normal throughput" taggedAs LongRunningTest in { - // if (exerciseActors) { - // exerciseRouters("use routers with normal throughput", normalThroughputDuration, - // batchInterval = workBatchInterval, expectDroppedMessages = false, tree = false) - // } - // enterBarrier("after-" + step) - // } - // - // "use routers with high throughput" taggedAs LongRunningTest in { - // if (exerciseActors) { - // exerciseRouters("use routers with high throughput", highThroughputDuration, - // batchInterval = Duration.Zero, expectDroppedMessages = false, tree = false) - // } - // enterBarrier("after-" + step) - // } - // - // "use many actors with normal throughput" taggedAs LongRunningTest in { - // if (exerciseActors) { - // exerciseRouters("use many actors with normal throughput", normalThroughputDuration, - // batchInterval = workBatchInterval, expectDroppedMessages = false, tree = true) - // } - // enterBarrier("after-" + step) - // } - // - // "use many actors with high throughput" taggedAs LongRunningTest in { - // if (exerciseActors) { - // exerciseRouters("use many actors with high throughput", highThroughputDuration, - // batchInterval = Duration.Zero, expectDroppedMessages = false, tree = true) - // } - // enterBarrier("after-" + step) - // } - // - // "exercise join/remove/join/remove" taggedAs LongRunningTest in { - // exerciseJoinRemove("exercise join/remove", joinRemoveDuration) - // enterBarrier("after-" + step) - // } - // - // "exercise supervision" taggedAs LongRunningTest in { - // if (exerciseActors) { - // exerciseSupervision("exercise supervision", supervisionDuration, supervisionOneIteration) - // } - // enterBarrier("after-" + step) - // } - // - // "gossip when idle" taggedAs LongRunningTest in { - // idleGossip("idle gossip") - // enterBarrier("after-" + step) - // } - // - // "start routers that are running while nodes are removed" taggedAs LongRunningTest in { - // if (exerciseActors) { - // runOn(roles.take(3): _*) { - // system.actorOf( - // Props(classOf[Master], settings, settings.workBatchInterval, false).withDeploy(Deploy.local), - // name = masterName) ! Begin - // } - // } - // enterBarrier("after-" + step) - // } - // - // "leave nodes one-by-one from large cluster" taggedAs LongRunningTest in { - // removeOneByOne(numberOfNodesLeavingOneByOneLarge, shutdown = false) - // enterBarrier("after-" + step) - // } - // - // "shutdown nodes one-by-one from large cluster" taggedAs LongRunningTest in { - // removeOneByOne(numberOfNodesShutdownOneByOneLarge, shutdown = true) - // enterBarrier("after-" + step) - // } - // - // "leave several nodes" taggedAs LongRunningTest in { - // removeSeveral(numberOfNodesLeaving, shutdown = false) - // nbrUsedRoles -= numberOfNodesLeaving - // enterBarrier("after-" + step) - // } - // - // "shutdown several nodes" taggedAs LongRunningTest in { - // removeSeveral(numberOfNodesShutdown, shutdown = true) - // nbrUsedRoles -= numberOfNodesShutdown - // enterBarrier("after-" + step) - // } - // - // "shutdown nodes one-by-one from small cluster" taggedAs LongRunningTest in { - // removeOneByOne(numberOfNodesShutdownOneByOneSmall, shutdown = true) - // enterBarrier("after-" + step) - // } - // - // "leave nodes one-by-one from small cluster" taggedAs LongRunningTest in { - // removeOneByOne(numberOfNodesLeavingOneByOneSmall, shutdown = false) - // enterBarrier("after-" + step) - // } - // - // "end routers that are running while nodes are removed" taggedAs LongRunningTest in within(30.seconds) { - // if (exerciseActors) { - // runOn(roles.take(3): _*) { - // master match { - // case Some(m) ⇒ - // m.tell(End, testActor) - // val workResult = awaitWorkResult(m) - // workResult.sendCount should be > (0L) - // workResult.ackCount should be > (0L) - // case None ⇒ fail("master not running") - // } - // } - // } - // enterBarrier("after-" + step) - // } - // - // "log jvm info" taggedAs LongRunningTest in { - // if (infolog) { - // log.info("StressSpec JVM:\n{}", jvmInfo) - // } - // enterBarrier("after-" + step) - // } + "log settings" taggedAs LongRunningTest in { + if (infolog) { + log.info("StressSpec JVM:\n{}", jvmInfo) + runOn(roles.head) { + log.info("StressSpec settings:\n{}", settings) + } + } + enterBarrier("after-" + step) + } + + "join seed nodes" taggedAs LongRunningTest in within(30 seconds) { + + val otherNodesJoiningSeedNodes = roles.slice(numberOfSeedNodes, numberOfSeedNodes + numberOfNodesJoiningToSeedNodesInitially) + val size = seedNodes.size + otherNodesJoiningSeedNodes.size + + createResultAggregator("join seed nodes", expectedResults = size, includeInHistory = true) + + runOn((seedNodes ++ otherNodesJoiningSeedNodes): _*) { + reportResult { + cluster.joinSeedNodes(seedNodes.toIndexedSeq map address) + awaitMembersUp(size, timeout = remainingOrDefault) + } + } + + awaitClusterResult() + + nbrUsedRoles += size + enterBarrier("after-" + step) + } + + "start routers that are running while nodes are joining" taggedAs LongRunningTest in { + runOn(roles.take(3): _*) { + system.actorOf( + Props(classOf[Master], settings, settings.workBatchInterval, false).withDeploy(Deploy.local), + name = masterName) ! Begin + } + } + + "join nodes one-by-one to small cluster" taggedAs LongRunningTest in { + joinOneByOne(numberOfNodesJoiningOneByOneSmall) + enterBarrier("after-" + step) + } + + "join several nodes to one node" taggedAs LongRunningTest in { + joinSeveral(numberOfNodesJoiningToOneNode, toSeedNodes = false) + nbrUsedRoles += numberOfNodesJoiningToOneNode + enterBarrier("after-" + step) + } + + "join several nodes to seed nodes" taggedAs LongRunningTest in { + if (numberOfNodesJoiningToSeedNodes > 0) { + joinSeveral(numberOfNodesJoiningToSeedNodes, toSeedNodes = true) + nbrUsedRoles += numberOfNodesJoiningToSeedNodes + } + enterBarrier("after-" + step) + } + + "join nodes one-by-one to large cluster" taggedAs LongRunningTest in { + joinOneByOne(numberOfNodesJoiningOneByOneLarge) + enterBarrier("after-" + step) + } + + "end routers that are running while nodes are joining" taggedAs LongRunningTest in within(30.seconds) { + if (exerciseActors) { + runOn(roles.take(3): _*) { + master match { + case Some(m) ⇒ + m.tell(End, testActor) + val workResult = awaitWorkResult(m) + workResult.retryCount should ===(0) + workResult.sendCount should be > (0L) + workResult.ackCount should be > (0L) + case None ⇒ fail("master not running") + } + } + } + enterBarrier("after-" + step) + } + + "use routers with normal throughput" taggedAs LongRunningTest in { + if (exerciseActors) { + exerciseRouters("use routers with normal throughput", normalThroughputDuration, + batchInterval = workBatchInterval, expectDroppedMessages = false, tree = false) + } + enterBarrier("after-" + step) + } + + "use routers with high throughput" taggedAs LongRunningTest in { + if (exerciseActors) { + exerciseRouters("use routers with high throughput", highThroughputDuration, + batchInterval = Duration.Zero, expectDroppedMessages = false, tree = false) + } + enterBarrier("after-" + step) + } + + "use many actors with normal throughput" taggedAs LongRunningTest in { + if (exerciseActors) { + exerciseRouters("use many actors with normal throughput", normalThroughputDuration, + batchInterval = workBatchInterval, expectDroppedMessages = false, tree = true) + } + enterBarrier("after-" + step) + } + + "use many actors with high throughput" taggedAs LongRunningTest in { + if (exerciseActors) { + exerciseRouters("use many actors with high throughput", highThroughputDuration, + batchInterval = Duration.Zero, expectDroppedMessages = false, tree = true) + } + enterBarrier("after-" + step) + } + + "exercise join/remove/join/remove" taggedAs LongRunningTest in { + exerciseJoinRemove("exercise join/remove", joinRemoveDuration) + enterBarrier("after-" + step) + } + + "exercise supervision" taggedAs LongRunningTest in { + if (exerciseActors) { + exerciseSupervision("exercise supervision", supervisionDuration, supervisionOneIteration) + } + enterBarrier("after-" + step) + } + + "gossip when idle" taggedAs LongRunningTest in { + idleGossip("idle gossip") + enterBarrier("after-" + step) + } + + "start routers that are running while nodes are removed" taggedAs LongRunningTest in { + if (exerciseActors) { + runOn(roles.take(3): _*) { + system.actorOf( + Props(classOf[Master], settings, settings.workBatchInterval, false).withDeploy(Deploy.local), + name = masterName) ! Begin + } + } + enterBarrier("after-" + step) + } + + "leave nodes one-by-one from large cluster" taggedAs LongRunningTest in { + removeOneByOne(numberOfNodesLeavingOneByOneLarge, shutdown = false) + enterBarrier("after-" + step) + } + + "shutdown nodes one-by-one from large cluster" taggedAs LongRunningTest in { + removeOneByOne(numberOfNodesShutdownOneByOneLarge, shutdown = true) + enterBarrier("after-" + step) + } + + "leave several nodes" taggedAs LongRunningTest in { + removeSeveral(numberOfNodesLeaving, shutdown = false) + nbrUsedRoles -= numberOfNodesLeaving + enterBarrier("after-" + step) + } + + "shutdown several nodes" taggedAs LongRunningTest in { + removeSeveral(numberOfNodesShutdown, shutdown = true) + nbrUsedRoles -= numberOfNodesShutdown + enterBarrier("after-" + step) + } + + "shutdown nodes one-by-one from small cluster" taggedAs LongRunningTest in { + removeOneByOne(numberOfNodesShutdownOneByOneSmall, shutdown = true) + enterBarrier("after-" + step) + } + + "leave nodes one-by-one from small cluster" taggedAs LongRunningTest in { + removeOneByOne(numberOfNodesLeavingOneByOneSmall, shutdown = false) + enterBarrier("after-" + step) + } + + "end routers that are running while nodes are removed" taggedAs LongRunningTest in within(30.seconds) { + if (exerciseActors) { + runOn(roles.take(3): _*) { + master match { + case Some(m) ⇒ + m.tell(End, testActor) + val workResult = awaitWorkResult(m) + workResult.sendCount should be > (0L) + workResult.ackCount should be > (0L) + case None ⇒ fail("master not running") + } + } + } + enterBarrier("after-" + step) + } + + "log jvm info" taggedAs LongRunningTest in { + if (infolog) { + log.info("StressSpec JVM:\n{}", jvmInfo) + } + enterBarrier("after-" + step) + } } } 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 2ba21c2207..079de2f5f1 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 @@ -97,7 +97,7 @@ abstract class MultiNodeConfig { _roles(MultiNodeSpec.selfIndex) } - private[testkit] def config: Config = { + private[akka] def config: Config = { val transportConfig = if (_testTransport) ConfigFactory.parseString( """ diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 4e6ce2efa0..d9be34a281 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -462,7 +462,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } val driver = MediaDriver.launchEmbedded(driverContext) - log.debug("Started embedded media driver in directory [{}]", driver.aeronDirectoryName) + log.info("Started embedded media driver in directory [{}]", driver.aeronDirectoryName) topLevelFREvents.loFreq(Transport_MediaDriverStarted, driver.aeronDirectoryName().getBytes("US-ASCII")) Runtime.getRuntime.addShutdownHook(stopMediaDriverShutdownHook) if (!mediaDriver.compareAndSet(None, Some(driver))) { diff --git a/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala b/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala index a8e1d67750..635c7ee517 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala @@ -4,16 +4,20 @@ package akka.remote.artery import java.util.concurrent.TimeUnit.MICROSECONDS -import scala.util.control.NonFatal -import akka.actor.ExtendedActorSystem -import akka.dispatch.AbstractNodeQueue -import akka.event.Logging -import org.agrona.concurrent.BackoffIdleStrategy +import java.util.concurrent.TimeUnit.MILLISECONDS + import scala.annotation.tailrec import scala.reflect.ClassTag -import org.agrona.concurrent.IdleStrategy -import org.agrona.concurrent.BusySpinIdleStrategy +import scala.util.control.NonFatal + +import akka.actor.ExtendedActorSystem +import akka.dispatch.AbstractNodeQueue import akka.dispatch.MonitorableThreadFactory +import akka.event.Logging +import org.agrona.concurrent.BackoffIdleStrategy +import org.agrona.concurrent.BusySpinIdleStrategy +import org.agrona.concurrent.IdleStrategy +import org.agrona.concurrent.SleepingIdleStrategy /** * INTERNAL API @@ -82,10 +86,9 @@ private[akka] object TaskRunner { } def createIdleStrategy(idleCpuLevel: Int): IdleStrategy = { - if (idleCpuLevel == 1) { - val maxParkMicros = 400 - new BackoffIdleStrategy(100, 1, MICROSECONDS.toNanos(1), MICROSECONDS.toNanos(maxParkMicros)) - } else if (idleCpuLevel == 10) + if (idleCpuLevel == 1) + new SleepingIdleStrategy(MILLISECONDS.toNanos(1)) + else if (idleCpuLevel == 10) new BusySpinIdleStrategy else { // spin between 100 to 10000 depending on idleCpuLevel From 392ca5ecce5a106588ec0a502bc732f87b907b90 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Fri, 16 Sep 2016 15:12:40 +0200 Subject: [PATCH 144/186] Enable flight recorder in tests #21205 * Setting to configure where the flight recorder puts its file * Run ArteryMultiNodeSpecs with flight recorder enabled * More cleanup in exit hook, wait for task runner to stop * Enable flight recorder for the cluster multi node tests * Enable flight recorder for multi node remoting tests * Toggle always-dump flight recorder output when akka.remote.artery.always-dump-flight-recorder is set --- .../akka/cluster/MultiNodeClusterSpec.scala | 25 ++++++-- .../testkit/FlightRecordingSupport.scala | 59 +++++++++++++++++++ .../remote/AttemptSysMsgRedeliverySpec.scala | 5 +- .../akka/remote/LookupRemoteActorSpec.scala | 6 +- .../akka/remote/MultiNodeRemotingSpec.scala | 48 +++++++++++++++ .../akka/remote/NewRemoteActorSpec.scala | 5 +- .../PiercingShouldKeepQuarantineSpec.scala | 6 +- .../akka/remote/RemoteDeliverySpec.scala | 5 +- .../RemoteDeploymentDeathWatchSpec.scala | 5 +- .../akka/remote/RemoteGatePiercingSpec.scala | 5 +- .../remote/RemoteNodeDeathWatchSpec.scala | 5 +- .../RemoteNodeRestartDeathWatchSpec.scala | 3 +- .../remote/RemoteNodeRestartGateSpec.scala | 3 +- .../RemoteNodeShutdownAndComesBackSpec.scala | 5 +- .../remote/RemoteQuarantinePiercingSpec.scala | 6 +- .../akka/remote/RemoteReDeploymentSpec.scala | 6 +- .../RemoteRestartedQuarantinedSpec.scala | 3 +- .../scala/akka/remote/Ticket15109Spec.scala | 4 +- .../akka/remote/artery/LatencySpec.scala | 6 +- .../remote/artery/MaxThroughputSpec.scala | 11 ++-- .../RemoteRestartedQuarantinedSpec.scala | 14 ++--- .../artery/SurviveNetworkPartitionSpec.scala | 9 +-- .../remote/routing/RemoteRandomSpec.scala | 9 +-- .../remote/routing/RemoteRoundRobinSpec.scala | 11 ++-- .../routing/RemoteScatterGatherSpec.scala | 9 +-- .../testconductor/TestConductorSpec.scala | 11 ++-- .../remote/testkit/MultiNodeSpecSpec.scala | 5 +- akka-remote/src/main/resources/reference.conf | 5 ++ .../akka/remote/artery/ArterySettings.scala | 3 + .../akka/remote/artery/ArteryTransport.scala | 39 ++++++------ .../akka/remote/artery/FlightRecorder.scala | 40 +++++++++++-- .../remote/artery/FlightRecorderReader.scala | 40 ++++++++++++- .../scala/akka/remote/artery/TaskRunner.scala | 27 +++++---- .../remote/artery/ArteryMultiNodeSpec.scala | 38 +++++++++++- .../remote/artery/FlightRecorderSpec.scala | 31 +++++++++- project/Dependencies.scala | 2 +- 36 files changed, 380 insertions(+), 134 deletions(-) create mode 100644 akka-multi-node-testkit/src/main/scala/akka/remote/testkit/FlightRecordingSupport.scala create mode 100644 akka-remote-tests/src/multi-jvm/scala/akka/remote/MultiNodeRemotingSpec.scala rename akka-remote/src/{test => main}/scala/akka/remote/artery/FlightRecorderReader.scala (86%) 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 a819090154..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,7 +37,7 @@ object MultiNodeClusterSpec { def clusterConfig(failureDetectorPuppet: Boolean): Config = if (failureDetectorPuppet) clusterConfigWithFailureDetectorPuppet else clusterConfig - def clusterConfig: Config = ConfigFactory.parseString(""" + def clusterConfig: Config = ConfigFactory.parseString(s""" akka.actor.provider = cluster akka.cluster { jmx.enabled = off @@ -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,7 +88,7 @@ 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 @@ -92,6 +103,10 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec with WatchedByCoro 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-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..c5aa24275b --- /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("Flight recorder dump:") + FlightRecorderReader.dumpToStdout(flightRecorderFile) + } + } + + private def destinationIsValidForDump() = { + val path = flightRecorderFile.toString + path != "" && path.endsWith(".afr") + } + +} 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 35badd1cc0..fd1d2d079b 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 @@ -24,7 +24,7 @@ class AttemptSysMsgRedeliveryMultiJvmSpec(artery: Boolean) extends MultiNodeConf commonConfig(debugConfig(on = false).withFallback( ConfigFactory.parseString(s""" akka.remote.artery.enabled = $artery - """))) + """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf)) testTransport(on = true) @@ -53,8 +53,7 @@ object AttemptSysMsgRedeliverySpec { } abstract class AttemptSysMsgRedeliverySpec(multiNodeConfig: AttemptSysMsgRedeliveryMultiJvmSpec) - extends MultiNodeSpec(multiNodeConfig) - with STMultiNodeSpec with ImplicitSender with DefaultTimeout { + extends MultiNodeRemotingSpec(multiNodeConfig) { import multiNodeConfig._ import AttemptSysMsgRedeliverySpec._ 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 67635e1b17..47fe16b294 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 @@ -18,7 +18,7 @@ class LookupRemoteActorMultiJvmSpec(artery: Boolean) extends MultiNodeConfig { commonConfig(debugConfig(on = false).withFallback( ConfigFactory.parseString(s""" akka.remote.artery.enabled = $artery - """))) + """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf)) val master = role("master") val slave = role("slave") @@ -39,8 +39,8 @@ object LookupRemoteActorSpec { } } -abstract class LookupRemoteActorSpec(multiNodeConfig: LookupRemoteActorMultiJvmSpec) extends MultiNodeSpec(multiNodeConfig) - with STMultiNodeSpec with ImplicitSender with DefaultTimeout { +abstract class LookupRemoteActorSpec(multiNodeConfig: LookupRemoteActorMultiJvmSpec) + extends MultiNodeRemotingSpec(multiNodeConfig) { import multiNodeConfig._ import LookupRemoteActorSpec._ diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/MultiNodeRemotingSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/MultiNodeRemotingSpec.scala new file mode 100644 index 0000000000..65e92bc68a --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/MultiNodeRemotingSpec.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 MultiNodeRemotingSpec { + + def arteryFlightRecordingConf = + ConfigFactory.parseString( + s""" + akka.remote.artery.advanced.flight-recorder { + enabled=on + destination=target/flight-recorder-${UUID.randomUUID().toString}.afr + } + """) + +} + +abstract class MultiNodeRemotingSpec(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/NewRemoteActorSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/NewRemoteActorSpec.scala index 91a959dacd..66025899da 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 @@ -20,7 +20,7 @@ class NewRemoteActorMultiJvmSpec(artery: Boolean) extends MultiNodeConfig { ConfigFactory.parseString(s""" akka.remote.log-remote-lifecycle-events = off akka.remote.artery.enabled = $artery - """))) + """).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf))) val master = role("master") val slave = role("slave") @@ -55,8 +55,7 @@ object NewRemoteActorSpec { } abstract class NewRemoteActorSpec(multiNodeConfig: NewRemoteActorMultiJvmSpec) - extends MultiNodeSpec(multiNodeConfig) - with STMultiNodeSpec with ImplicitSender with DefaultTimeout { + extends MultiNodeRemotingSpec(multiNodeConfig) { import multiNodeConfig._ import NewRemoteActorSpec._ 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 7a337fa57c..6c3b9df226 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 @@ -15,7 +15,7 @@ class PiercingShouldKeepQuarantineConfig(artery: Boolean) extends MultiNodeConfi ConfigFactory.parseString(s""" akka.remote.retry-gate-closed-for = 0.5s akka.remote.artery.enabled = $artery - """))) + """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf)) } @@ -38,9 +38,7 @@ object PiercingShouldKeepQuarantineSpec { } abstract class PiercingShouldKeepQuarantineSpec(multiNodeConfig: PiercingShouldKeepQuarantineConfig) - extends MultiNodeSpec(multiNodeConfig) - with STMultiNodeSpec - with ImplicitSender { + extends MultiNodeRemotingSpec(multiNodeConfig) { import multiNodeConfig._ import PiercingShouldKeepQuarantineSpec._ 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 3e59ee2e87..d54c90e4f8 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 @@ -26,7 +26,7 @@ class RemoteDeliveryConfig(artery: Boolean) extends MultiNodeConfig { commonConfig(debugConfig(on = false).withFallback( ConfigFactory.parseString(s""" akka.remote.artery.enabled = $artery - """))) + """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf)) } class RemoteDeliveryMultiJvmNode1 extends RemoteDeliverySpec(new RemoteDeliveryConfig(artery = false)) @@ -48,8 +48,7 @@ object RemoteDeliverySpec { } abstract class RemoteDeliverySpec(multiNodeConfig: RemoteDeliveryConfig) - extends MultiNodeSpec(multiNodeConfig) - with STMultiNodeSpec with ImplicitSender { + extends MultiNodeRemotingSpec(multiNodeConfig) { import multiNodeConfig._ import RemoteDeliverySpec._ 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 649e340b6e..a469a5baf1 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 @@ -27,7 +27,7 @@ class RemoteDeploymentDeathWatchMultiJvmSpec(artery: Boolean) extends MultiNodeC akka.loglevel = INFO akka.remote.log-remote-lifecycle-events = off akka.remote.artery.enabled = $artery - """))) + """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf)) deployOn(second, """/hello.remote = "@third@" """) @@ -69,8 +69,7 @@ object RemoteDeploymentDeathWatchSpec { } abstract class RemoteDeploymentDeathWatchSpec(multiNodeConfig: RemoteDeploymentDeathWatchMultiJvmSpec) - extends MultiNodeSpec(multiNodeConfig) - with STMultiNodeSpec with ImplicitSender { + extends MultiNodeRemotingSpec(multiNodeConfig) { import multiNodeConfig._ import RemoteDeploymentDeathWatchSpec._ 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..08f6269945 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 MultiNodeRemotingSpec(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 06dcfe4e8d..4fe5da25ce 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 @@ -31,7 +31,7 @@ class RemoteNodeDeathWatchConfig(artery: Boolean) extends MultiNodeConfig { ## 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(MultiNodeRemotingSpec.arteryFlightRecordingConf)) } @@ -92,8 +92,7 @@ object RemoteNodeDeathWatchSpec { } abstract class RemoteNodeDeathWatchSpec(multiNodeConfig: RemoteNodeDeathWatchConfig) - extends MultiNodeSpec(multiNodeConfig) - with STMultiNodeSpec with ImplicitSender { + extends MultiNodeRemotingSpec(multiNodeConfig) { import multiNodeConfig._ import RemoteNodeDeathWatchSpec._ import RemoteWatcher._ 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 2a7a838e69..63dc11987a 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 @@ -63,8 +63,7 @@ object RemoteNodeRestartDeathWatchSpec { } abstract class RemoteNodeRestartDeathWatchSpec(multiNodeConfig: RemoteNodeRestartDeathWatchConfig) - extends MultiNodeSpec(multiNodeConfig) - with STMultiNodeSpec with ImplicitSender { + extends MultiNodeRemotingSpec(multiNodeConfig) { import multiNodeConfig._ import RemoteNodeRestartDeathWatchSpec._ 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..ee7015602d 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 MultiNodeRemotingSpec(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 dc72e998a0..b396c6e402 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 MultiNodeRemotingSpec(RemoteNodeShutdownAndComesBackSpec) { import RemoteNodeShutdownAndComesBackSpec._ 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 170fdfa4f6..7790ea489e 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 @@ -27,7 +27,7 @@ class RemoteQuarantinePiercingConfig(artery: Boolean) extends MultiNodeConfig { akka.loglevel = INFO akka.remote.log-remote-lifecycle-events = INFO akka.remote.artery.enabled = $artery - """))) + """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf)) } @@ -51,9 +51,7 @@ object RemoteQuarantinePiercingSpec { } abstract class RemoteQuarantinePiercingSpec(multiNodeConfig: RemoteQuarantinePiercingConfig) - extends MultiNodeSpec(multiNodeConfig) - with STMultiNodeSpec - with ImplicitSender { + extends MultiNodeRemotingSpec(multiNodeConfig) { import multiNodeConfig._ import RemoteQuarantinePiercingSpec._ 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 456d8a262a..5e7253286f 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 @@ -35,7 +35,7 @@ class RemoteReDeploymentConfig(artery: Boolean) extends MultiNodeConfig { acceptable-heartbeat-pause=2.5s } akka.remote.artery.enabled = $artery - """))) + """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf)) testTransport(on = true) @@ -105,8 +105,8 @@ object RemoteReDeploymentMultiJvmSpec { def echoProps(target: ActorRef) = Props(new Echo(target)) } -abstract class RemoteReDeploymentMultiJvmSpec(multiNodeConfig: RemoteReDeploymentConfig) extends MultiNodeSpec(multiNodeConfig) - with STMultiNodeSpec with ImplicitSender { +abstract class RemoteReDeploymentMultiJvmSpec(multiNodeConfig: RemoteReDeploymentConfig) + extends MultiNodeRemotingSpec(multiNodeConfig) { def sleepAfterKill: FiniteDuration def expectQuarantine: Boolean 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 bf66b9795a..cd53abd011 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 MultiNodeRemotingSpec(RemoteRestartedQuarantinedSpec) { import RemoteRestartedQuarantinedSpec._ 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..b43610fea1 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 MultiNodeRemotingSpec(Ticket15109Spec) { import Ticket15109Spec._ 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 index f8e1afe19b..d7a5763be7 100644 --- 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 @@ -9,6 +9,7 @@ import java.util.concurrent.locks.LockSupport import scala.concurrent.duration._ import akka.actor._ +import akka.remote.MultiNodeRemotingSpec import akka.remote.testconductor.RoleName import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeSpec @@ -52,7 +53,7 @@ object LatencySpec extends MultiNodeConfig { } } } - """))) + """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf)) final case object Reset @@ -160,8 +161,7 @@ class LatencySpecMultiJvmNode1 extends LatencySpec class LatencySpecMultiJvmNode2 extends LatencySpec abstract class LatencySpec - extends MultiNodeSpec(LatencySpec) - with STMultiNodeSpec with ImplicitSender { + extends MultiNodeRemotingSpec(LatencySpec) { import LatencySpec._ 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 index 9488ddd898..6e2b29184d 100644 --- 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 @@ -6,9 +6,10 @@ 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.RemoteActorRefProvider +import akka.remote.{ MultiNodeRemotingSpec, RARP, RemoteActorRefProvider } import akka.remote.testconductor.RoleName import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeSpec @@ -19,7 +20,6 @@ import akka.serialization.SerializerWithStringManifest import akka.testkit._ import com.typesafe.config.ConfigFactory import akka.remote.artery.compress.CompressionProtocol.Events.ReceivedActorRefCompressionTable -import akka.remote.RARP object MaxThroughputSpec extends MultiNodeConfig { val first = role("first") @@ -67,7 +67,7 @@ object MaxThroughputSpec extends MultiNodeConfig { } } } - """))) + """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf)) case object Run sealed trait Echo extends DeadLetterSuppression with JavaSerializable @@ -277,10 +277,7 @@ object MaxThroughputSpec extends MultiNodeConfig { class MaxThroughputSpecMultiJvmNode1 extends MaxThroughputSpec class MaxThroughputSpecMultiJvmNode2 extends MaxThroughputSpec -abstract class MaxThroughputSpec - extends MultiNodeSpec(MaxThroughputSpec) - with STMultiNodeSpec with ImplicitSender - with PerfFlamesSupport { +abstract class MaxThroughputSpec extends MultiNodeRemotingSpec(MaxThroughputSpec) with PerfFlamesSupport { import MaxThroughputSpec._ 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 index b7bc87155a..333de1a87e 100644 --- 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 @@ -4,12 +4,13 @@ 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.{ ForceDisassociateExplicitly, ForceDisassociate, Direction } +import akka.remote.transport.ThrottlerTransportAdapter.{ Direction, ForceDisassociate, ForceDisassociateExplicitly } import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.STMultiNodeSpec @@ -17,10 +18,9 @@ import akka.testkit._ import akka.actor.ActorIdentity import akka.remote.testconductor.RoleName import akka.actor.Identify + import scala.concurrent.Await -import akka.remote.AddressUidExtension -import akka.remote.RARP -import akka.remote.ThisActorSystemQuarantinedEvent +import akka.remote.{ AddressUidExtension, MultiNodeRemotingSpec, RARP, ThisActorSystemQuarantinedEvent } object RemoteRestartedQuarantinedSpec extends MultiNodeConfig { val first = role("first") @@ -31,7 +31,7 @@ object RemoteRestartedQuarantinedSpec extends MultiNodeConfig { akka.loglevel = WARNING akka.remote.log-remote-lifecycle-events = WARNING akka.remote.artery.enabled = on - """))) + """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf)) class Subject extends Actor { def receive = { @@ -45,9 +45,7 @@ object RemoteRestartedQuarantinedSpec extends MultiNodeConfig { class RemoteRestartedQuarantinedSpecMultiJvmNode1 extends RemoteRestartedQuarantinedSpec class RemoteRestartedQuarantinedSpecMultiJvmNode2 extends RemoteRestartedQuarantinedSpec -abstract class RemoteRestartedQuarantinedSpec - extends MultiNodeSpec(RemoteRestartedQuarantinedSpec) - with STMultiNodeSpec with ImplicitSender { +abstract class RemoteRestartedQuarantinedSpec extends MultiNodeRemotingSpec(RemoteRestartedQuarantinedSpec) { import RemoteRestartedQuarantinedSpec._ 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 index 9eab4bab1b..3356111b5d 100644 --- 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 @@ -7,7 +7,7 @@ import scala.concurrent.duration._ import akka.actor._ import akka.actor.ActorIdentity import akka.actor.Identify -import akka.remote.RARP +import akka.remote.{ MultiNodeRemotingSpec, QuarantinedEvent, RARP } import akka.remote.testconductor.RoleName import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeSpec @@ -15,7 +15,6 @@ import akka.remote.testkit.STMultiNodeSpec import akka.remote.transport.ThrottlerTransportAdapter.Direction import akka.testkit._ import com.typesafe.config.ConfigFactory -import akka.remote.QuarantinedEvent object SurviveNetworkPartitionSpec extends MultiNodeConfig { val first = role("first") @@ -26,7 +25,7 @@ object SurviveNetworkPartitionSpec extends MultiNodeConfig { akka.loglevel = INFO akka.remote.artery.enabled = on akka.remote.artery.advanced.give-up-system-message-after = 4s - """))) + """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf)) testTransport(on = true) } @@ -34,9 +33,7 @@ object SurviveNetworkPartitionSpec extends MultiNodeConfig { class SurviveNetworkPartitionSpecMultiJvmNode1 extends SurviveNetworkPartitionSpec class SurviveNetworkPartitionSpecMultiJvmNode2 extends SurviveNetworkPartitionSpec -abstract class SurviveNetworkPartitionSpec - extends MultiNodeSpec(SurviveNetworkPartitionSpec) - with STMultiNodeSpec with ImplicitSender { +abstract class SurviveNetworkPartitionSpec extends MultiNodeRemotingSpec(SurviveNetworkPartitionSpec) { import SurviveNetworkPartitionSpec._ 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 318b71a4ad..27cce605b0 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,7 +9,8 @@ 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.MultiNodeRemotingSpec +import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec } import akka.routing.Broadcast import akka.routing.RandomPool import akka.routing.RoutedActorRef @@ -26,7 +27,7 @@ class RemoteRandomConfig(artery: Boolean) extends MultiNodeConfig { commonConfig(debugConfig(on = false).withFallback( ConfigFactory.parseString(s""" akka.remote.artery.enabled = $artery - """))) + """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf)) deployOnAll(""" /service-hello { @@ -55,8 +56,8 @@ object RemoteRandomSpec { } } -class RemoteRandomSpec(multiNodeConfig: RemoteRandomConfig) extends MultiNodeSpec(multiNodeConfig) - with STMultiNodeSpec with ImplicitSender with DefaultTimeout { +class RemoteRandomSpec(multiNodeConfig: RemoteRandomConfig) extends MultiNodeRemotingSpec(multiNodeConfig) + with DefaultTimeout { import multiNodeConfig._ import RemoteRandomSpec._ 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 157c8fb0e3..8c21c7c644 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 @@ -10,9 +10,11 @@ import akka.actor.ActorRef import akka.actor.Props import akka.actor.PoisonPill import akka.actor.Address + import scala.concurrent.Await import akka.pattern.ask -import akka.remote.testkit.{ STMultiNodeSpec, MultiNodeConfig, MultiNodeSpec } +import akka.remote.MultiNodeRemotingSpec +import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec } import akka.routing.Broadcast import akka.routing.GetRoutees import akka.routing.Routees @@ -23,6 +25,7 @@ import akka.routing.Resizer import akka.routing.Routee import akka.routing.FromConfig import akka.testkit._ + import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory @@ -36,7 +39,7 @@ class RemoteRoundRobinConfig(artery: Boolean) extends MultiNodeConfig { commonConfig(debugConfig(on = false).withFallback( ConfigFactory.parseString(s""" akka.remote.artery.enabled = $artery - """))) + """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf)) deployOnAll(""" /service-hello { @@ -83,8 +86,8 @@ object RemoteRoundRobinSpec { } } -class RemoteRoundRobinSpec(multiNodeConfig: RemoteRoundRobinConfig) extends MultiNodeSpec(multiNodeConfig) - with STMultiNodeSpec with ImplicitSender with DefaultTimeout { +class RemoteRoundRobinSpec(multiNodeConfig: RemoteRoundRobinConfig) extends MultiNodeRemotingSpec(multiNodeConfig) + with DefaultTimeout { import multiNodeConfig._ import RemoteRoundRobinSpec._ 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 351595d837..d671d5779b 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,7 +9,8 @@ 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.MultiNodeRemotingSpec +import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec } import akka.routing.Broadcast import akka.routing.ScatterGatherFirstCompletedPool import akka.routing.RoutedActorRef @@ -27,7 +28,7 @@ class RemoteScatterGatherConfig(artery: Boolean) extends MultiNodeConfig { commonConfig(debugConfig(on = false).withFallback( ConfigFactory.parseString(s""" akka.remote.artery.enabled = $artery - """))) + """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf)) deployOnAll(""" /service-hello { @@ -56,8 +57,8 @@ object RemoteScatterGatherSpec { } } -class RemoteScatterGatherSpec(multiNodeConfig: RemoteScatterGatherConfig) extends MultiNodeSpec(multiNodeConfig) - with STMultiNodeSpec with ImplicitSender with DefaultTimeout { +class RemoteScatterGatherSpec(multiNodeConfig: RemoteScatterGatherConfig) extends MultiNodeRemotingSpec(multiNodeConfig) + with DefaultTimeout { import multiNodeConfig._ import RemoteScatterGatherSpec._ 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..c5cb873d65 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.MultiNodeRemotingSpec +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(MultiNodeRemotingSpec.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 MultiNodeRemotingSpec(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..99fc3b6add 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.MultiNodeRemotingSpec import akka.testkit.LongRunningTest object MultiNodeSpecMultiJvmSpec extends MultiNodeConfig { - commonConfig(debugConfig(on = false)) + commonConfig(debugConfig(on = false).withFallback(MultiNodeRemotingSpec.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 MultiNodeRemotingSpec(MultiNodeSpecMultiJvmSpec) { import MultiNodeSpecMultiJvmSpec._ diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 2a210b6ee5..9b8c83ce5d 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -285,6 +285,11 @@ akka { 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 diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala index 0d3c6cc1a8..eb14e7b506 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala @@ -11,9 +11,11 @@ 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 */ @@ -108,6 +110,7 @@ private[akka] final class ArterySettings private (config: Config) { 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 = 1024 * 1024 diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index d9be34a281..9f7368a73c 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -6,21 +6,19 @@ 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.concurrent.TimeUnit import java.util.concurrent.atomic.{ AtomicLong, AtomicReference } import java.util.concurrent.atomic.AtomicBoolean import scala.annotation.tailrec -import scala.collection.JavaConverters._ -import scala.concurrent.Future -import scala.concurrent.Promise +import scala.concurrent.{ Await, Future, Promise } import scala.concurrent.duration._ import scala.util.Failure import scala.util.Success import scala.util.Try import scala.util.control.NoStackTrace import scala.util.control.NonFatal - import akka.Done import akka.NotUsed import akka.actor._ @@ -426,8 +424,13 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R log.info("Remoting started; listening on address: {}", defaultAddress) } - private lazy val stopMediaDriverShutdownHook = new Thread { - override def run(): Unit = stopMediaDriver() + private lazy val shutdownHook = new Thread { + override def run(): Unit = { + if (!_shutdown) { + internalShutdown() + + } + } } private def startMediaDriver(): Unit = { @@ -464,7 +467,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R val driver = MediaDriver.launchEmbedded(driverContext) log.info("Started embedded media driver in directory [{}]", driver.aeronDirectoryName) topLevelFREvents.loFreq(Transport_MediaDriverStarted, driver.aeronDirectoryName().getBytes("US-ASCII")) - Runtime.getRuntime.addShutdownHook(stopMediaDriverShutdownHook) + Runtime.getRuntime.addShutdownHook(shutdownHook) if (!mediaDriver.compareAndSet(None, Some(driver))) { throw new IllegalStateException("media driver started more than once") } @@ -493,7 +496,6 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R "Couldn't delete Aeron embedded media driver files in [{}] due to [{}]", driver.aeronDirectoryName, e.getMessage) } - Try(Runtime.getRuntime.removeShutdownHook(stopMediaDriverShutdownHook)) } } @@ -742,14 +744,18 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R flushingPromise.future } implicit val ec = remoteDispatcher + flushing.recover { case _ ⇒ Done }.flatMap(_ ⇒ internalShutdown()) + } + private def internalShutdown(): Future[Done] = { + import system.dispatcher + + killSwitch.abort(ShutdownSignal) + topLevelFREvents.loFreq(Transport_KillSwitchPulled, NoMetaData) for { - _ ← flushing.recover { case _ ⇒ Done } - _ = killSwitch.abort(ShutdownSignal) _ ← streamsCompleted + _ ← taskRunner.stop() } yield { - topLevelFREvents.loFreq(Transport_KillSwitchPulled, NoMetaData) - taskRunner.stop() topLevelFREvents.loFreq(Transport_Stopped, NoMetaData) if (aeronErrorLogTask != null) { @@ -767,8 +773,6 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R flightRecorder.foreach(_.close()) afrFileChannel.foreach(_.force(true)) afrFileChannel.foreach(_.close()) - // TODO: Be smarter about this in tests and make it always-on-for prod - afrFile.foreach(_.delete()) Done } } @@ -957,11 +961,10 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .toMat(messageDispatcherSink)(Keep.both) } - private def initializeFlightRecorder(): Option[(FileChannel, File, FlightRecorder)] = { + private def initializeFlightRecorder(): Option[(FileChannel, Path, FlightRecorder)] = { if (settings.Advanced.FlightRecorderEnabled) { - // TODO: Figure out where to put it, currently using temporary files - val afrFile = File.createTempFile("artery", ".afr") - afrFile.deleteOnExit() + 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))) diff --git a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala index c6583e6781..c6d63817a1 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala @@ -3,9 +3,9 @@ */ package akka.remote.artery -import java.io.{ File, RandomAccessFile } +import java.io.RandomAccessFile import java.nio.channels.FileChannel -import java.nio.file.StandardOpenOption +import java.nio.file._ import java.nio.{ ByteBuffer, ByteOrder } import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.{ CountDownLatch, TimeUnit } @@ -184,13 +184,43 @@ private[remote] class RollingEventLogSection( */ private[remote] object FlightRecorder { - def prepareFileForFlightRecorder(file: File): FileChannel = { + /** + * @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(file, "rwd") + val randomAccessFile = new RandomAccessFile(path.toFile, "rwd") randomAccessFile.setLength(FlightRecorder.TotalSize) randomAccessFile.close() - FileChannel.open(file.toPath, StandardOpenOption.CREATE, StandardOpenOption.WRITE, StandardOpenOption.READ) + FileChannel.open(path, StandardOpenOption.CREATE, StandardOpenOption.WRITE, StandardOpenOption.READ) } val Alignment = 64 * 1024 // Windows is picky about mapped section alignments diff --git a/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderReader.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderReader.scala similarity index 86% rename from akka-remote/src/test/scala/akka/remote/artery/FlightRecorderReader.scala rename to akka-remote/src/main/scala/akka/remote/artery/FlightRecorderReader.scala index fd5cbb805d..71682da8b3 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderReader.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderReader.scala @@ -1,14 +1,18 @@ package akka.remote.artery -import java.io.IOException +import java.io.{ IOException, RandomAccessFile } import java.nio.channels.FileChannel +import java.nio.file.Path import java.time.Instant import org.agrona.concurrent.MappedResizeableBuffer import scala.collection.immutable -object FlightRecorderReader { +/** + * Internal API + */ +private[akka] object FlightRecorderReader { import FlightRecorder._ sealed trait LogState @@ -59,9 +63,39 @@ object FlightRecorderReader { 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) + println(reader.structure) + + println("--- ALERT ENTRIES") + reader.structure.alertLog.logs.foreach(log ⇒ println(log.richEntries.mkString("\n"))) + + println("--- HI FREQUENCY ENTRIES") + reader.structure.hiFreqLog.logs.foreach(log ⇒ println(log.compactEntries.mkString("\n"))) + + println("--- LO FREQUENCY ENTRIES") + reader.structure.loFreqLog.logs.foreach(log ⇒ println(log.richEntries.mkString("\n"))) + + } finally { + if (reader ne null) reader.close() + if (channel ne null) channel.close() + if (raFile ne null) raFile.close() + } + } + } -class FlightRecorderReader(fileChannel: FileChannel) { +/** + * Internal API + */ +private[akka] final class FlightRecorderReader(fileChannel: FileChannel) { import FlightRecorder._ import FlightRecorderReader._ diff --git a/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala b/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala index 635c7ee517..922672bf6b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala @@ -3,22 +3,19 @@ */ package akka.remote.artery -import java.util.concurrent.TimeUnit.MICROSECONDS -import java.util.concurrent.TimeUnit.MILLISECONDS +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 -import akka.actor.ExtendedActorSystem -import akka.dispatch.AbstractNodeQueue -import akka.dispatch.MonitorableThreadFactory -import akka.event.Logging -import org.agrona.concurrent.BackoffIdleStrategy -import org.agrona.concurrent.BusySpinIdleStrategy -import org.agrona.concurrent.IdleStrategy -import org.agrona.concurrent.SleepingIdleStrategy - /** * INTERNAL API */ @@ -112,6 +109,7 @@ private[akka] class TaskRunner(system: ExtendedActorSystem, val idleCpuLevel: In 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 @@ -126,8 +124,9 @@ private[akka] class TaskRunner(system: ExtendedActorSystem, val idleCpuLevel: In thread.start() } - def stop(): Unit = { + def stop(): Future[Done] = { command(Shutdown) + shutdown.future } def command(cmd: Command): Unit = { @@ -177,7 +176,9 @@ private[akka] class TaskRunner(system: ExtendedActorSystem, val idleCpuLevel: In case null ⇒ // no command case Add(task) ⇒ tasks.add(task) case Remove(task) ⇒ tasks.remove(task) - case Shutdown ⇒ running = false + case Shutdown ⇒ + running = false + shutdown.trySuccess(Done) } } diff --git a/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala index 908bf25f3d..4e27bcc44d 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala @@ -3,14 +3,19 @@ */ package akka.remote.artery -import akka.actor.{ ActorSystem, ExtendedActorSystem, RootActorPath } +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(""" + ConfigFactory.parseString(s""" akka { actor.provider = remote actor.warn-about-java-serializer-usage = off @@ -20,6 +25,10 @@ object ArteryMultiNodeSpec { hostname = localhost port = 0 } + advanced.flight-recorder { + enabled=on + destination=target/flight-recorder-${UUID.randomUUID().toString}.afr + } } } """) @@ -41,6 +50,8 @@ abstract class ArteryMultiNodeSpec(config: Config) extends AkkaSpec(config.withF 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 @@ -62,9 +73,32 @@ abstract class ArteryMultiNodeSpec(config: Config) extends AkkaSpec(config.withF 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 protected def beforeTermination(): Unit = { + handleFlightRecorderFile() + } + override def afterTermination(): Unit = { remoteSystems.foreach(sys ⇒ shutdown(sys)) remoteSystems = Vector.empty } + 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/FlightRecorderSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala index 3dd3030fb6..4a04660b9b 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala @@ -6,12 +6,13 @@ package akka.remote.artery import java.io.{ File, IOException, RandomAccessFile } import java.nio.channels.FileChannel -import java.nio.file.StandardOpenOption +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.Jimfs class FlightRecorderSpec extends AkkaSpec { import FlightRecorderReader._ @@ -372,6 +373,34 @@ class FlightRecorderSpec extends AkkaSpec { } } + "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() + + 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 = { diff --git a/project/Dependencies.scala b/project/Dependencies.scala index c5935a8b2f..247a42e198 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -140,7 +140,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, aeronDriver, aeronClient, 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) From c20b6088cd04cf84abdc2e9c470fc26d85ab80d9 Mon Sep 17 00:00:00 2001 From: Konrad Malawski Date: Mon, 5 Sep 2016 22:44:22 +0200 Subject: [PATCH 145/186] =art add metadata field to the header, #20703 --- .../src/main/scala/akka/util/ByteString.scala | 6 +- .../akka/remote/artery/CodecBenchmark.scala | 4 +- akka-remote/src/main/resources/reference.conf | 11 + .../akka/remote/artery/ArteryTransport.scala | 2 +- .../scala/akka/remote/artery/BufferPool.scala | 129 ++++++++++-- .../scala/akka/remote/artery/Codecs.scala | 67 ++++++- .../akka/remote/artery/InboundEnvelope.scala | 11 +- .../artery/MetadataEnvelopeSerializer.scala | 173 ++++++++++++++++ .../akka/remote/artery/OutboundEnvelope.scala | 4 +- .../akka/remote/artery/RemoteInstrument.scala | 189 ++++++++++++++++++ .../scala/akka/remote/artery/TaskRunner.scala | 8 +- .../remote/artery/EnvelopeBufferSpec.scala | 8 +- .../artery/MetaMetadataSerializerSpec.scala | 73 +++++++ .../remote/artery/MetadataCarryingSpec.scala | 102 ++++++++++ .../remote/artery/MetadataContainerSpec.scala | 65 ++++++ .../akka/remote/artery/MetadataMapSpec.scala | 47 +++++ 16 files changed, 857 insertions(+), 42 deletions(-) create mode 100644 akka-remote/src/main/scala/akka/remote/artery/MetadataEnvelopeSerializer.scala create mode 100644 akka-remote/src/main/scala/akka/remote/artery/RemoteInstrument.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/MetaMetadataSerializerSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/MetadataCarryingSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/MetadataContainerSpec.scala create mode 100644 akka-remote/src/test/scala/akka/remote/artery/MetadataMapSpec.scala diff --git a/akka-actor/src/main/scala/akka/util/ByteString.scala b/akka-actor/src/main/scala/akka/util/ByteString.scala index d2600320d8..9cb952efad 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-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala index be9f3efd90..0e6927c705 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -140,7 +140,7 @@ class CodecBenchmark { val N = 100000 val encoder: Flow[OutboundEnvelope, EnvelopeBuffer, Encoder.ChangeOutboundCompression] = - Flow.fromGraph(new Encoder(uniqueLocalAddress, system, outboundEnvelopePool, envelopePool)) + Flow.fromGraph(new Encoder(uniqueLocalAddress, system.asInstanceOf[ExtendedActorSystem], outboundEnvelopePool, envelopePool)) Source.fromGraph(new BenchTestSourceSameElement(N, "elem")) .map(msg ⇒ outboundEnvelopePool.acquire().init(OptionVal.None, payload, OptionVal.Some(remoteRefB))) @@ -197,7 +197,7 @@ class CodecBenchmark { val N = 100000 val encoder: Flow[OutboundEnvelope, EnvelopeBuffer, Encoder.ChangeOutboundCompression] = - Flow.fromGraph(new Encoder(uniqueLocalAddress, system, outboundEnvelopePool, envelopePool)) + Flow.fromGraph(new Encoder(uniqueLocalAddress, system.asInstanceOf[ExtendedActorSystem], outboundEnvelopePool, envelopePool)) val localRecipient = resolvedRef.path.toSerializationFormatWithAddress(uniqueLocalAddress.address) val provider = RARP(system).provider diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 9b8c83ce5d..3d6a596822 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -320,6 +320,17 @@ akka { 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} [] } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 9f7368a73c..b4b98c34ca 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -995,7 +995,7 @@ private[remote] object ArteryTransport { val ProtocolName = "akka" - val Version = 0 + val Version: Byte = 0 class AeronTerminated(e: Throwable) extends RuntimeException(e) diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index 8871c14eda..1786f527c9 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -13,9 +13,12 @@ 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.{ OptionVal, Unsafe } +import akka.util.{ ByteString, CompactByteString, OptionVal, Unsafe } import akka.remote.artery.compress.NoInboundCompressions +import akka.util.ByteString.ByteString1C + +import scala.annotation.tailrec /** * INTERNAL API @@ -44,6 +47,19 @@ private[remote] class EnvelopeBufferPool(maximumPayload: Int, maximumBuffers: In } +/** 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)})" +} +object ByteFlag { + def binaryLeftPad(byte: Byte): String = { + val string = Integer.toBinaryString(byte) + val pad = "0" * (8 - string.length) // leftPad + pad + string + } +} + /** * INTERNAL API */ @@ -52,17 +68,24 @@ private[remote] object EnvelopeBuffer { val TagTypeMask = 0xFF000000 val TagValueMask = 0x0000FFFF - val VersionOffset = 0 // Int + // Flags (1 byte allocated for them) + val MetadataPresentFlag = new ByteFlag(0x1) + + val VersionOffset = 0 // Byte + val FlagsOffset = 1 // Byte + // 2 bytes free // TODO re-align values to not have this empty space val UidOffset = 4 // Long val SerializerOffset = 12 // Int val SenderActorRefTagOffset = 16 // Int val RecipientActorRefTagOffset = 20 // Int val ClassManifestTagOffset = 24 // Int - val ActorRefCompressionTableVersionTagOffset = 28 // Int - val ClassManifestCompressionTableVersionTagOffset = 32 // Int + val ActorRefCompressionTableVersionTagOffset = 28 // Int // TODO handle roll-over and move to Short + val ClassManifestCompressionTableVersionTagOffset = 32 // Int // TODO handle roll-over and move to Short - val LiteralsSectionOffset = 36 + // EITHER metadata followed by literals directly OR literals directly in this spot. + // Mode depends on the `MetadataPresentFlag`. + val MetadataContainerAndLiteralSectionOffset = 36 // Int val UsAscii = Charset.forName("US-ASCII") @@ -89,8 +112,13 @@ private[remote] object HeaderBuilder { * INTERNAL API */ private[remote] sealed trait HeaderBuilder { - def setVersion(v: Int): Unit - def version: Int + 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: Int def inboundClassManifestCompressionTableVersion: Int @@ -104,6 +132,9 @@ private[remote] sealed trait HeaderBuilder { 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. @@ -134,6 +165,9 @@ private[remote] sealed trait HeaderBuilder { */ def recipientActorRefPath: OptionVal[String] + def setMetadataContainer(container: ByteString): Unit + def clearMetadataContainer(): Unit + def setSerializer(serializer: Int): Unit def serializer: Int @@ -167,7 +201,8 @@ private[remote] final class HeaderBuilderImpl( private[this] val toSerializationFormat: SerializationFormatCache = new SerializationFormatCache // Fields only available for EnvelopeBuffer - var _version: Int = _ + var _version: Byte = _ + var _flags: Byte = _ var _uid: Long = _ var _inboundActorRefCompressionTableVersion: Int = 0 var _inboundClassManifestCompressionTableVersion: Int = 0 @@ -181,9 +216,18 @@ private[remote] final class HeaderBuilderImpl( var _manifest: String = null var _manifestIdx: Int = -1 - override def setVersion(v: Int) = _version = v + var _metadataContainer: ByteString = null + + 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 @@ -257,9 +301,23 @@ private[remote] final class HeaderBuilderImpl( } } + /** 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 + ", " + @@ -267,7 +325,8 @@ private[remote] final class HeaderBuilderImpl( "_recipientActorRefIdx:" + _recipientActorRefIdx + ", " + "_serializer:" + _serializer + ", " + "_manifest:" + _manifest + ", " + - "_manifestIdx:" + _manifestIdx + ")" + "_manifestIdx:" + _manifestIdx + ", " + + "_metadataContainer:" + _metadataContainer + ")" } @@ -286,17 +345,30 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { byteBuffer.clear() // Write fixed length parts - byteBuffer.putInt(header.version) + byteBuffer.put(header.version) + byteBuffer.put(header.flags) + // 1 empty byte slot // TODO avoid having these empty slots + // 1 empty byte slot + byteBuffer.position(UidOffset) // skips the above 2 empty slots byteBuffer.putLong(header.uid) byteBuffer.putInt(header.serializer) // compression table version numbers byteBuffer.putInt(ActorRefCompressionTableVersionTagOffset, header.outboundActorRefCompression.version | TagTypeMask) byteBuffer.putInt(ClassManifestCompressionTableVersionTagOffset, header.outboundClassManifestCompression.version | TagTypeMask) + byteBuffer.putInt(SenderActorRefTagOffset, header._senderActorRefIdx | TagTypeMask) - // Write compressable, variable-length parts always to the actual position of the buffer - // Write tag values explicitly in their proper offset - byteBuffer.position(LiteralsSectionOffset) + if (header.flag(MetadataPresentFlag)) { + // tag if we have metadata or not, as the layout next follows different patterns depending on that + byteBuffer.position(MetadataContainerAndLiteralSectionOffset) + + header.metadataContainer.copyToBuffer(byteBuffer) + // after metadata is written, buffer is at correct position to continue writing literals (they "moved forward") + } else { + // Write compressable, variable-length parts always to the actual position of the buffer + // Write tag values explicitly in their proper offset + byteBuffer.position(MetadataContainerAndLiteralSectionOffset) + } // Serialize sender if (header._senderActorRefIdx != -1) @@ -315,13 +387,17 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { 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.getInt + header setVersion byteBuffer.get() + header setFlags byteBuffer.get() + byteBuffer.get() // skip 1 byte + byteBuffer.get() // skip 1 byte header setUid byteBuffer.getLong header setSerializer byteBuffer.getInt @@ -335,9 +411,21 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { header._inboundClassManifestCompressionTableVersion = manifestCompressionVersionTag & TagValueMask } - // Read compressable, variable-length parts always from the actual position of the buffer - // Read tag values explicitly from their proper offset - byteBuffer.position(LiteralsSectionOffset) + 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) @@ -372,8 +460,7 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { private def readLiteral(): String = { val length = byteBuffer.getShort - if (length == 0) - "" + if (length == 0) "" else { ensureLiteralCharsLength(length) val chars = literalChars @@ -399,7 +486,7 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { if (length == 0) { byteBuffer.putShort(0) } else { - byteBuffer.putShort(literal.length.toShort) + byteBuffer.putShort(length.toShort) ensureLiteralCharsLength(length) val bytes = literalBytes val chars = Unsafe.instance.getObject(literal, StringValueFieldOffset).asInstanceOf[Array[Char]] diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index e2ee25c58e..2ef13b9b3f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -23,7 +23,8 @@ import akka.Done import akka.stream.stage.GraphStageWithMaterializedValue import scala.concurrent.Promise -import java.util.concurrent.atomic.AtomicInteger + +import scala.annotation.switch /** * INTERNAL API @@ -45,7 +46,7 @@ private[remote] object Encoder { */ private[remote] class Encoder( uniqueLocalAddress: UniqueAddress, - system: ActorSystem, + system: ExtendedActorSystem, outboundEnvelopePool: ObjectPool[ReusableOutboundEnvelope], bufferPool: EnvelopeBufferPool) extends GraphStageWithMaterializedValue[FlowShape[OutboundEnvelope, EnvelopeBuffer], Encoder.ChangeOutboundCompression] { @@ -65,6 +66,10 @@ private[remote] class Encoder( 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) @@ -106,6 +111,7 @@ private[remote] class Encoder( case OptionVal.Some(s) ⇒ headerBuilder setSenderActorRef s } + applyAndRenderRemoteMessageSentMetadata(instruments, outboundEnvelope, headerBuilder) MessageSerializer.serializeForArtery(serialization, outboundEnvelope.message, headerBuilder, envelope) } finally Serialization.currentTransportInformation.value = oldValue @@ -131,14 +137,43 @@ private[remote] class Encoder( } finally { outboundEnvelope match { case r: ReusableOutboundEnvelope ⇒ outboundEnvelopePool.release(r) - case _ ⇒ + 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 */ @@ -319,6 +354,7 @@ private[remote] class Decoder( originUid, headerBuilder.serializer, classManifest, + headerBuilder.flags, envelope, association) @@ -421,6 +457,7 @@ private[remote] class Deserializer( 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] @@ -432,7 +469,11 @@ private[remote] class Deserializer( val deserializedMessage = MessageSerializer.deserializeForArtery( system, envelope.originUid, serialization, envelope.serializer, envelope.classManifest, envelope.envelopeBuffer) - push(out, envelope.withMessage(deserializedMessage)) + val envelopeWithMessage = envelope.withMessage(deserializedMessage) + + applyIncomingInstruments(envelopeWithMessage) + + push(out, envelopeWithMessage) } catch { case NonFatal(e) ⇒ log.warning( @@ -448,6 +489,22 @@ private[remote] class Deserializer( 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/InboundEnvelope.scala b/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala index bd15681ef4..c017d73107 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala @@ -23,7 +23,7 @@ private[remote] object InboundEnvelope { originUid: Long, association: OptionVal[OutboundContext]): InboundEnvelope = { val env = new ReusableInboundEnvelope - env.init(recipient, recipientAddress, sender, originUid, -1, "", null, association) + env.init(recipient, recipientAddress, sender, originUid, -1, "", 0, null, association) .withMessage(message) } @@ -44,6 +44,9 @@ private[remote] trait InboundEnvelope { def message: AnyRef def envelopeBuffer: EnvelopeBuffer + def flags: Byte + def flag(byteFlag: ByteFlag): Boolean + def withMessage(message: AnyRef): InboundEnvelope def releaseEnvelopeBuffer(): InboundEnvelope @@ -71,6 +74,7 @@ private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { 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 @@ -84,6 +88,9 @@ private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { 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 @@ -115,6 +122,7 @@ private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { originUid: Long, serializer: Int, classManifest: String, + flags: Byte, envelopeBuffer: EnvelopeBuffer, association: OptionVal[OutboundContext]): InboundEnvelope = { _recipient = recipient @@ -123,6 +131,7 @@ private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { _originUid = originUid _serializer = serializer _classManifest = classManifest + _flags = flags _envelopeBuffer = envelopeBuffer _association = association this 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/OutboundEnvelope.scala b/akka-remote/src/main/scala/akka/remote/artery/OutboundEnvelope.scala index 6a1e9ba84c..3625f612dd 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/OutboundEnvelope.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/OutboundEnvelope.scala @@ -3,11 +3,9 @@ */ package akka.remote.artery -import akka.actor.InternalActorRef -import akka.util.OptionVal -import akka.actor.Address import akka.actor.ActorRef import akka.remote.RemoteActorRef +import akka.util.OptionVal /** * INTERNAL API 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..e9a8764e35 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/artery/RemoteInstrument.scala @@ -0,0 +1,189 @@ +/* + * 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("[", ",", "]")})" +} + +object MetadataMap { + def apply[T >: Null]() = new MetadataMap[T] +} diff --git a/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala b/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala index 922672bf6b..e3f093e0f6 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala @@ -3,16 +3,16 @@ */ package akka.remote.artery -import java.util.concurrent.TimeUnit.{MICROSECONDS, MILLISECONDS} +import java.util.concurrent.TimeUnit.{ MICROSECONDS, MILLISECONDS } import akka.Done import akka.actor.ExtendedActorSystem -import akka.dispatch.{AbstractNodeQueue, MonitorableThreadFactory} +import akka.dispatch.{ AbstractNodeQueue, MonitorableThreadFactory } import akka.event.Logging -import org.agrona.concurrent.{BackoffIdleStrategy, BusySpinIdleStrategy, IdleStrategy, SleepingIdleStrategy} +import org.agrona.concurrent.{ BackoffIdleStrategy, BusySpinIdleStrategy, IdleStrategy, SleepingIdleStrategy } import scala.annotation.tailrec -import scala.concurrent.{Future, Promise} +import scala.concurrent.{ Future, Promise } import scala.reflect.ClassTag import scala.util.control.NonFatal diff --git a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala index 1bebe48629..c6f85a2e98 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala @@ -69,7 +69,7 @@ class EnvelopeBufferSpec extends AkkaSpec { headerIn setManifest "manifest1" envelope.writeHeader(headerIn) - envelope.byteBuffer.position() should ===(EnvelopeBuffer.LiteralsSectionOffset) // Fully compressed header + envelope.byteBuffer.position() should ===(EnvelopeBuffer.MetadataContainerAndLiteralSectionOffset) // Fully compressed header envelope.byteBuffer.flip() envelope.parseHeader(headerOut) @@ -98,7 +98,7 @@ class EnvelopeBufferSpec extends AkkaSpec { headerIn setManifest "uncompressable3333" val expectedHeaderLength = - EnvelopeBuffer.LiteralsSectionOffset + // Constant header part + EnvelopeBuffer.MetadataContainerAndLiteralSectionOffset + // Constant header part 2 + lengthOfSerializedActorRefPath(senderRef) + // Length field + literal 2 + lengthOfSerializedActorRefPath(recipientRef) + // Length field + literal 2 + "uncompressable3333".length // Length field + literal @@ -131,7 +131,7 @@ class EnvelopeBufferSpec extends AkkaSpec { envelope.writeHeader(headerIn) envelope.byteBuffer.position() should ===( - EnvelopeBuffer.LiteralsSectionOffset + + EnvelopeBuffer.MetadataContainerAndLiteralSectionOffset + 2 + lengthOfSerializedActorRefPath(recipientRef)) envelope.byteBuffer.flip() @@ -157,7 +157,7 @@ class EnvelopeBufferSpec extends AkkaSpec { envelope.writeHeader(headerIn) envelope.byteBuffer.position() should ===( - EnvelopeBuffer.LiteralsSectionOffset + + EnvelopeBuffer.MetadataContainerAndLiteralSectionOffset + 2 + lengthOfSerializedActorRefPath(senderRef) + 2 + "longlonglongliteralmanifest".length) 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) + } + } + +} From 8e97ecee8f826558307539687e2960c87e892238 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Mon, 19 Sep 2016 11:00:34 +0200 Subject: [PATCH 146/186] Textual representations of event codes in flight recorder dump (#21489) --- .../akka/remote/artery/ArteryTransport.scala | 3 +- .../remote/artery/FlightRecorderEvents.scala | 43 +++++++++++++++++++ .../remote/artery/FlightRecorderReader.scala | 11 ++++- 3 files changed, 54 insertions(+), 3 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index b4b98c34ca..c0d3e02ce0 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -489,6 +489,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R try { if (settings.Advanced.DeleteAeronDirectory) { IoUtil.delete(new File(driver.aeronDirectoryName), false) + topLevelFREvents.loFreq(Transport_MediaFileDeleted, NoMetaData) } } catch { case NonFatal(e) ⇒ @@ -766,7 +767,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R if (areonErrorLog != null) areonErrorLog.close() if (mediaDriver.get.isDefined) { stopMediaDriver() - topLevelFREvents.loFreq(Transport_MediaFileDeleted, NoMetaData) + } topLevelFREvents.loFreq(Transport_FlightRecorderClose, NoMetaData) diff --git a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala index e800e5493f..ea0ccff368 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala @@ -2,6 +2,8 @@ package akka.remote.artery object FlightRecorderEvents { + // Note: Remember to update dictionary when adding new events! + val NoMetaData = Array.empty[Byte] // Top level remoting events @@ -44,4 +46,45 @@ object FlightRecorderEvents { 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 index 71682da8b3..762accc4c7 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderReader.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderReader.scala @@ -197,11 +197,18 @@ private[akka] final class FlightRecorderReader(fileChannel: FileChannel) { } case class RichEntry(timeStamp: Instant, dirty: Boolean, code: Long, metadata: Array[Byte]) { - override def toString: String = s"[$timeStamp] ${if (dirty) "#" else ""} \t $code | ${metadata.mkString(",")}" + override def toString: String = { + val textualCode = FlightRecorderEvents.eventDictionary.getOrElse(code, "").take(34) + val metadataString = new String(metadata, "UTF-8") + f"[$timeStamp] ${if (dirty) "#" else ""} $code%3s $textualCode%-34s | $metadataString" + } } case class CompactEntry(timeStamp: Instant, dirty: Boolean, code: Long, param: Long) { - override def toString: String = s"[$timeStamp] ${if (dirty) "#" else ""} \t $code | $param" + 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) From 76c23a7880c448f55e8e1a9fba89b2bfb85c7d16 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 15 Sep 2016 11:27:00 +0200 Subject: [PATCH 147/186] fix many bugs in InboundCompressions, #21464 * comprehensive integration test that revealed many bugs * confirmations of manifests were wrong, at two places * using wrong tables when system is restarted, including originUid in the tables with checks when receiving advertisments * close (stop scheduling) of advertisments when new incarnation, quarantine, or restart * cleanup how deadLetters ref was treated, and made it more robust * make Decoder tolerant to decompression failures, can happen in case of system restart before handshake completed * give up resending advertisment after a few attempts without confirmation, to avoid keeping outbound association open to possible dead system * don't advertise new table when no inbound messages, to avoid keeping outbound association open to possible dead system * HeaderBuilder could use manifest field from previous message, added resetMessageFields * No compression for ArteryMessage, e.g. handshake messages must go through without depending on compression tables being in sync * improve debug logging, including originUid --- .../InvertCompressionTableBenchmark.scala | 6 +- .../akka/util/LruBoundedCacheBench.scala | 2 +- .../remote/artery/MaxThroughputSpec.scala | 2 +- .../akka/remote/ArteryControlFormats.java | 294 ++++++++++++------ .../main/protobuf/ArteryControlFormats.proto | 7 +- .../akka/remote/artery/ArteryTransport.scala | 62 ++-- .../akka/remote/artery/Association.scala | 12 +- .../scala/akka/remote/artery/BufferPool.scala | 102 ++++-- .../scala/akka/remote/artery/Codecs.scala | 57 +++- .../scala/akka/remote/artery/Handshake.scala | 2 +- .../artery/compress/CompressionTable.scala | 24 +- .../artery/compress/DecompressionTable.scala | 17 +- .../artery/compress/InboundCompressions.scala | 248 ++++++++++----- .../compress/NoInboundCompressions.scala | 27 -- .../ArteryMessageSerializer.scala | 33 +- .../remote/artery/EnvelopeBufferSpec.scala | 17 +- .../ActorRefCompressionIntegrationSpec.scala | 66 ---- ...ssManifestCompressionIntegrationSpec.scala | 98 ------ .../compress/CompressionIntegrationSpec.scala | 272 +++++++++++++++- .../compress/CompressionTableSpec.scala | 8 +- .../artery/compress/CompressionTestKit.scala | 17 - ...dshakeShouldDropCompressionTableSpec.scala | 11 +- .../compress/OutboundCompressionSpec.scala | 4 +- .../ArteryMessageSerializerSpec.scala | 7 +- 24 files changed, 849 insertions(+), 546 deletions(-) delete mode 100644 akka-remote/src/main/scala/akka/remote/artery/compress/NoInboundCompressions.scala delete mode 100644 akka-remote/src/test/scala/akka/remote/artery/compress/ActorRefCompressionIntegrationSpec.scala delete mode 100644 akka-remote/src/test/scala/akka/remote/artery/compress/ClassManifestCompressionIntegrationSpec.scala delete mode 100644 akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTestKit.scala 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 index 3f3930232e..561ed83751 100644 --- 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 @@ -13,14 +13,14 @@ import org.openjdk.jmh.annotations._ class InvertCompressionTableBenchmark { /* - TODO: Possibly specialise the inversion, it's not in hot path so not doing it for now + 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(2, Map(Vector.fill[String](256)(randomName).zipWithIndex: _*)) - val compTable_1024 = CompressionTable(3, Map(Vector.fill[String](1024)(randomName).zipWithIndex: _*)) + 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/util/LruBoundedCacheBench.scala b/akka-bench-jmh/src/main/scala/akka/util/LruBoundedCacheBench.scala index cc89de28d9..f3325d4996 100644 --- a/akka-bench-jmh/src/main/scala/akka/util/LruBoundedCacheBench.scala +++ b/akka-bench-jmh/src/main/scala/akka/util/LruBoundedCacheBench.scala @@ -22,7 +22,7 @@ class LruBoundedCacheBench { @Param(Array("128", "256")) var stringSize = 0 - var lruCache: LruBoundedCache[String, String] = _ + private var lruCache: LruBoundedCache[String, String] = _ @Param(Array("90", "99")) var loadFactor: Int = _ 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 index 6e2b29184d..9966befdaa 100644 --- 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 @@ -139,7 +139,7 @@ object MaxThroughputSpec extends MultiNodeConfig { def waitingForCompression: Receive = { case ReceivedActorRefCompressionTable(_, table) ⇒ - if (table.map.contains(target)) { + if (table.dictionary.contains(target)) { sendBatch() // first some warmup target ! Start // then Start, which will echo back here context.setReceiveTimeout(Duration.Undefined) diff --git a/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java b/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java index 516c71356a..eb176e91ba 100644 --- a/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java +++ b/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java @@ -1297,19 +1297,29 @@ public final class ArteryControlFormats { */ akka.remote.ArteryControlFormats.UniqueAddressOrBuilder getFromOrBuilder(); - // required uint32 tableVersion = 2; + // required uint64 originUid = 2; /** - * required uint32 tableVersion = 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 = 2; + * required uint32 tableVersion = 3; */ int getTableVersion(); - // repeated string keys = 3; + // repeated string keys = 4; /** - * repeated string keys = 3; + * repeated string keys = 4; * *
      * actual Map is represented by separate sequences of keys and values,
@@ -1320,7 +1330,7 @@ public final class ArteryControlFormats {
     java.util.List
     getKeysList();
     /**
-     * repeated string keys = 3;
+     * repeated string keys = 4;
      *
      * 
      * actual Map is represented by separate sequences of keys and values,
@@ -1330,7 +1340,7 @@ public final class ArteryControlFormats {
      */
     int getKeysCount();
     /**
-     * repeated string keys = 3;
+     * repeated string keys = 4;
      *
      * 
      * actual Map is represented by separate sequences of keys and values,
@@ -1340,7 +1350,7 @@ public final class ArteryControlFormats {
      */
     java.lang.String getKeys(int index);
     /**
-     * repeated string keys = 3;
+     * repeated string keys = 4;
      *
      * 
      * actual Map is represented by separate sequences of keys and values,
@@ -1351,17 +1361,17 @@ public final class ArteryControlFormats {
     akka.protobuf.ByteString
         getKeysBytes(int index);
 
-    // repeated uint32 values = 4;
+    // repeated uint32 values = 5;
     /**
-     * repeated uint32 values = 4;
+     * repeated uint32 values = 5;
      */
     java.util.List getValuesList();
     /**
-     * repeated uint32 values = 4;
+     * repeated uint32 values = 5;
      */
     int getValuesCount();
     /**
-     * repeated uint32 values = 4;
+     * repeated uint32 values = 5;
      */
     int getValues(int index);
   }
@@ -1436,31 +1446,36 @@ public final class ArteryControlFormats {
             }
             case 16: {
               bitField0_ |= 0x00000002;
+              originUid_ = input.readUInt64();
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
               tableVersion_ = input.readUInt32();
               break;
             }
-            case 26: {
-              if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+            case 34: {
+              if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
                 keys_ = new akka.protobuf.LazyStringArrayList();
-                mutable_bitField0_ |= 0x00000004;
+                mutable_bitField0_ |= 0x00000008;
               }
               keys_.add(input.readBytes());
               break;
             }
-            case 32: {
-              if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
+            case 40: {
+              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
                 values_ = new java.util.ArrayList();
-                mutable_bitField0_ |= 0x00000008;
+                mutable_bitField0_ |= 0x00000010;
               }
               values_.add(input.readUInt32());
               break;
             }
-            case 34: {
+            case 42: {
               int length = input.readRawVarint32();
               int limit = input.pushLimit(length);
-              if (!((mutable_bitField0_ & 0x00000008) == 0x00000008) && input.getBytesUntilLimit() > 0) {
+              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010) && input.getBytesUntilLimit() > 0) {
                 values_ = new java.util.ArrayList();
-                mutable_bitField0_ |= 0x00000008;
+                mutable_bitField0_ |= 0x00000010;
               }
               while (input.getBytesUntilLimit() > 0) {
                 values_.add(input.readUInt32());
@@ -1476,10 +1491,10 @@ public final class ArteryControlFormats {
         throw new akka.protobuf.InvalidProtocolBufferException(
             e.getMessage()).setUnfinishedMessage(this);
       } finally {
-        if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+        if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
           keys_ = new akka.protobuf.UnmodifiableLazyStringList(keys_);
         }
-        if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
+        if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
           values_ = java.util.Collections.unmodifiableList(values_);
         }
         this.unknownFields = unknownFields.build();
@@ -1536,27 +1551,43 @@ public final class ArteryControlFormats {
       return from_;
     }
 
-    // required uint32 tableVersion = 2;
-    public static final int TABLEVERSION_FIELD_NUMBER = 2;
-    private int tableVersion_;
+    // required uint64 originUid = 2;
+    public static final int ORIGINUID_FIELD_NUMBER = 2;
+    private long originUid_;
     /**
-     * required uint32 tableVersion = 2;
+     * required uint64 originUid = 2;
      */
-    public boolean hasTableVersion() {
+    public boolean hasOriginUid() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * required uint32 tableVersion = 2;
+     * 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 = 3;
-    public static final int KEYS_FIELD_NUMBER = 3;
+    // repeated string keys = 4;
+    public static final int KEYS_FIELD_NUMBER = 4;
     private akka.protobuf.LazyStringList keys_;
     /**
-     * repeated string keys = 3;
+     * repeated string keys = 4;
      *
      * 
      * actual Map is represented by separate sequences of keys and values,
@@ -1569,7 +1600,7 @@ public final class ArteryControlFormats {
       return keys_;
     }
     /**
-     * repeated string keys = 3;
+     * repeated string keys = 4;
      *
      * 
      * actual Map is represented by separate sequences of keys and values,
@@ -1581,7 +1612,7 @@ public final class ArteryControlFormats {
       return keys_.size();
     }
     /**
-     * repeated string keys = 3;
+     * repeated string keys = 4;
      *
      * 
      * actual Map is represented by separate sequences of keys and values,
@@ -1593,7 +1624,7 @@ public final class ArteryControlFormats {
       return keys_.get(index);
     }
     /**
-     * repeated string keys = 3;
+     * repeated string keys = 4;
      *
      * 
      * actual Map is represented by separate sequences of keys and values,
@@ -1606,24 +1637,24 @@ public final class ArteryControlFormats {
       return keys_.getByteString(index);
     }
 
-    // repeated uint32 values = 4;
-    public static final int VALUES_FIELD_NUMBER = 4;
+    // repeated uint32 values = 5;
+    public static final int VALUES_FIELD_NUMBER = 5;
     private java.util.List values_;
     /**
-     * repeated uint32 values = 4;
+     * repeated uint32 values = 5;
      */
     public java.util.List
         getValuesList() {
       return values_;
     }
     /**
-     * repeated uint32 values = 4;
+     * repeated uint32 values = 5;
      */
     public int getValuesCount() {
       return values_.size();
     }
     /**
-     * repeated uint32 values = 4;
+     * repeated uint32 values = 5;
      */
     public int getValues(int index) {
       return values_.get(index);
@@ -1631,6 +1662,7 @@ public final class ArteryControlFormats {
 
     private void initFields() {
       from_ = akka.remote.ArteryControlFormats.UniqueAddress.getDefaultInstance();
+      originUid_ = 0L;
       tableVersion_ = 0;
       keys_ = akka.protobuf.LazyStringArrayList.EMPTY;
       values_ = java.util.Collections.emptyList();
@@ -1644,6 +1676,10 @@ public final class ArteryControlFormats {
         memoizedIsInitialized = 0;
         return false;
       }
+      if (!hasOriginUid()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
       if (!hasTableVersion()) {
         memoizedIsInitialized = 0;
         return false;
@@ -1663,13 +1699,16 @@ public final class ArteryControlFormats {
         output.writeMessage(1, from_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeUInt32(2, tableVersion_);
+        output.writeUInt64(2, originUid_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeUInt32(3, tableVersion_);
       }
       for (int i = 0; i < keys_.size(); i++) {
-        output.writeBytes(3, keys_.getByteString(i));
+        output.writeBytes(4, keys_.getByteString(i));
       }
       for (int i = 0; i < values_.size(); i++) {
-        output.writeUInt32(4, values_.get(i));
+        output.writeUInt32(5, values_.get(i));
       }
       getUnknownFields().writeTo(output);
     }
@@ -1686,7 +1725,11 @@ public final class ArteryControlFormats {
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += akka.protobuf.CodedOutputStream
-          .computeUInt32Size(2, tableVersion_);
+          .computeUInt64Size(2, originUid_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += akka.protobuf.CodedOutputStream
+          .computeUInt32Size(3, tableVersion_);
       }
       {
         int dataSize = 0;
@@ -1834,12 +1877,14 @@ public final class ArteryControlFormats {
           fromBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000001);
-        tableVersion_ = 0;
+        originUid_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000002);
-        keys_ = akka.protobuf.LazyStringArrayList.EMPTY;
+        tableVersion_ = 0;
         bitField0_ = (bitField0_ & ~0x00000004);
-        values_ = java.util.Collections.emptyList();
+        keys_ = akka.protobuf.LazyStringArrayList.EMPTY;
         bitField0_ = (bitField0_ & ~0x00000008);
+        values_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000010);
         return this;
       }
 
@@ -1879,16 +1924,20 @@ public final class ArteryControlFormats {
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
+        result.originUid_ = originUid_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
         result.tableVersion_ = tableVersion_;
-        if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        if (((bitField0_ & 0x00000008) == 0x00000008)) {
           keys_ = new akka.protobuf.UnmodifiableLazyStringList(
               keys_);
-          bitField0_ = (bitField0_ & ~0x00000004);
+          bitField0_ = (bitField0_ & ~0x00000008);
         }
         result.keys_ = keys_;
-        if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        if (((bitField0_ & 0x00000010) == 0x00000010)) {
           values_ = java.util.Collections.unmodifiableList(values_);
-          bitField0_ = (bitField0_ & ~0x00000008);
+          bitField0_ = (bitField0_ & ~0x00000010);
         }
         result.values_ = values_;
         result.bitField0_ = to_bitField0_;
@@ -1910,13 +1959,16 @@ public final class ArteryControlFormats {
         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_ & ~0x00000004);
+            bitField0_ = (bitField0_ & ~0x00000008);
           } else {
             ensureKeysIsMutable();
             keys_.addAll(other.keys_);
@@ -1926,7 +1978,7 @@ public final class ArteryControlFormats {
         if (!other.values_.isEmpty()) {
           if (values_.isEmpty()) {
             values_ = other.values_;
-            bitField0_ = (bitField0_ & ~0x00000008);
+            bitField0_ = (bitField0_ & ~0x00000010);
           } else {
             ensureValuesIsMutable();
             values_.addAll(other.values_);
@@ -1942,6 +1994,10 @@ public final class ArteryControlFormats {
           
           return false;
         }
+        if (!hasOriginUid()) {
+          
+          return false;
+        }
         if (!hasTableVersion()) {
           
           return false;
@@ -2089,49 +2145,82 @@ public final class ArteryControlFormats {
         return fromBuilder_;
       }
 
-      // required uint32 tableVersion = 2;
-      private int tableVersion_ ;
+      // required uint64 originUid = 2;
+      private long originUid_ ;
       /**
-       * required uint32 tableVersion = 2;
+       * required uint64 originUid = 2;
        */
-      public boolean hasTableVersion() {
+      public boolean hasOriginUid() {
         return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * required uint32 tableVersion = 2;
+       * 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 = 2;
+       * required uint32 tableVersion = 3;
        */
       public Builder setTableVersion(int value) {
-        bitField0_ |= 0x00000002;
+        bitField0_ |= 0x00000004;
         tableVersion_ = value;
         onChanged();
         return this;
       }
       /**
-       * required uint32 tableVersion = 2;
+       * required uint32 tableVersion = 3;
        */
       public Builder clearTableVersion() {
-        bitField0_ = (bitField0_ & ~0x00000002);
+        bitField0_ = (bitField0_ & ~0x00000004);
         tableVersion_ = 0;
         onChanged();
         return this;
       }
 
-      // repeated string keys = 3;
+      // repeated string keys = 4;
       private akka.protobuf.LazyStringList keys_ = akka.protobuf.LazyStringArrayList.EMPTY;
       private void ensureKeysIsMutable() {
-        if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+        if (!((bitField0_ & 0x00000008) == 0x00000008)) {
           keys_ = new akka.protobuf.LazyStringArrayList(keys_);
-          bitField0_ |= 0x00000004;
+          bitField0_ |= 0x00000008;
          }
       }
       /**
-       * repeated string keys = 3;
+       * repeated string keys = 4;
        *
        * 
        * actual Map is represented by separate sequences of keys and values,
@@ -2144,7 +2233,7 @@ public final class ArteryControlFormats {
         return java.util.Collections.unmodifiableList(keys_);
       }
       /**
-       * repeated string keys = 3;
+       * repeated string keys = 4;
        *
        * 
        * actual Map is represented by separate sequences of keys and values,
@@ -2156,7 +2245,7 @@ public final class ArteryControlFormats {
         return keys_.size();
       }
       /**
-       * repeated string keys = 3;
+       * repeated string keys = 4;
        *
        * 
        * actual Map is represented by separate sequences of keys and values,
@@ -2168,7 +2257,7 @@ public final class ArteryControlFormats {
         return keys_.get(index);
       }
       /**
-       * repeated string keys = 3;
+       * repeated string keys = 4;
        *
        * 
        * actual Map is represented by separate sequences of keys and values,
@@ -2181,7 +2270,7 @@ public final class ArteryControlFormats {
         return keys_.getByteString(index);
       }
       /**
-       * repeated string keys = 3;
+       * repeated string keys = 4;
        *
        * 
        * actual Map is represented by separate sequences of keys and values,
@@ -2200,7 +2289,7 @@ public final class ArteryControlFormats {
         return this;
       }
       /**
-       * repeated string keys = 3;
+       * repeated string keys = 4;
        *
        * 
        * actual Map is represented by separate sequences of keys and values,
@@ -2219,7 +2308,7 @@ public final class ArteryControlFormats {
         return this;
       }
       /**
-       * repeated string keys = 3;
+       * repeated string keys = 4;
        *
        * 
        * actual Map is represented by separate sequences of keys and values,
@@ -2235,7 +2324,7 @@ public final class ArteryControlFormats {
         return this;
       }
       /**
-       * repeated string keys = 3;
+       * repeated string keys = 4;
        *
        * 
        * actual Map is represented by separate sequences of keys and values,
@@ -2245,12 +2334,12 @@ public final class ArteryControlFormats {
        */
       public Builder clearKeys() {
         keys_ = akka.protobuf.LazyStringArrayList.EMPTY;
-        bitField0_ = (bitField0_ & ~0x00000004);
+        bitField0_ = (bitField0_ & ~0x00000008);
         onChanged();
         return this;
       }
       /**
-       * repeated string keys = 3;
+       * repeated string keys = 4;
        *
        * 
        * actual Map is represented by separate sequences of keys and values,
@@ -2269,35 +2358,35 @@ public final class ArteryControlFormats {
         return this;
       }
 
-      // repeated uint32 values = 4;
+      // repeated uint32 values = 5;
       private java.util.List values_ = java.util.Collections.emptyList();
       private void ensureValuesIsMutable() {
-        if (!((bitField0_ & 0x00000008) == 0x00000008)) {
+        if (!((bitField0_ & 0x00000010) == 0x00000010)) {
           values_ = new java.util.ArrayList(values_);
-          bitField0_ |= 0x00000008;
+          bitField0_ |= 0x00000010;
          }
       }
       /**
-       * repeated uint32 values = 4;
+       * repeated uint32 values = 5;
        */
       public java.util.List
           getValuesList() {
         return java.util.Collections.unmodifiableList(values_);
       }
       /**
-       * repeated uint32 values = 4;
+       * repeated uint32 values = 5;
        */
       public int getValuesCount() {
         return values_.size();
       }
       /**
-       * repeated uint32 values = 4;
+       * repeated uint32 values = 5;
        */
       public int getValues(int index) {
         return values_.get(index);
       }
       /**
-       * repeated uint32 values = 4;
+       * repeated uint32 values = 5;
        */
       public Builder setValues(
           int index, int value) {
@@ -2307,7 +2396,7 @@ public final class ArteryControlFormats {
         return this;
       }
       /**
-       * repeated uint32 values = 4;
+       * repeated uint32 values = 5;
        */
       public Builder addValues(int value) {
         ensureValuesIsMutable();
@@ -2316,7 +2405,7 @@ public final class ArteryControlFormats {
         return this;
       }
       /**
-       * repeated uint32 values = 4;
+       * repeated uint32 values = 5;
        */
       public Builder addAllValues(
           java.lang.Iterable values) {
@@ -2326,11 +2415,11 @@ public final class ArteryControlFormats {
         return this;
       }
       /**
-       * repeated uint32 values = 4;
+       * repeated uint32 values = 5;
        */
       public Builder clearValues() {
         values_ = java.util.Collections.emptyList();
-        bitField0_ = (bitField0_ & ~0x00000008);
+        bitField0_ = (bitField0_ & ~0x00000010);
         onChanged();
         return this;
       }
@@ -6055,21 +6144,22 @@ public final class ArteryControlFormats {
       "\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\"q\n\035" +
-      "CompressionTableAdvertisement\022\034\n\004from\030\001 " +
-      "\002(\0132\016.UniqueAddress\022\024\n\014tableVersion\030\002 \002(" +
-      "\r\022\014\n\004keys\030\003 \003(\t\022\016\n\006values\030\004 \003(\r\"Q\n Compr" +
-      "essionTableAdvertisementAck\022\034\n\004from\030\001 \002(" +
-      "\0132\016.UniqueAddress\022\017\n\007version\030\002 \002(\r\"\212\001\n\025S" +
-      "ystemMessageEnvelope\022\017\n\007message\030\001 \002(\014\022\024\n",
-      "\014serializerId\030\002 \002(\005\022\027\n\017messageManifest\030\003" +
-      " \001(\014\022\r\n\005seqNo\030\004 \002(\004\022\"\n\nackReplyTo\030\005 \002(\0132" +
-      "\016.UniqueAddress\"G\n\030SystemMessageDelivery" +
-      "Ack\022\r\n\005seqNo\030\001 \002(\004\022\034\n\004from\030\002 \002(\0132\016.Uniqu" +
-      "eAddress\"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\007address\030\001 \002(\0132" +
-      "\010.Address\022\013\n\003uid\030\002 \002(\004B\017\n\013akka.remoteH\001"
+      "ess\022\037\n\007address\030\001 \002(\0132\016.UniqueAddress\"\204\001\n" +
+      "\035CompressionTableAdvertisement\022\034\n\004from\030\001" +
+      " \002(\0132\016.UniqueAddress\022\021\n\toriginUid\030\002 \002(\004\022" +
+      "\024\n\014tableVersion\030\003 \002(\r\022\014\n\004keys\030\004 \003(\t\022\016\n\006v" +
+      "alues\030\005 \003(\r\"Q\n CompressionTableAdvertise" +
+      "mentAck\022\034\n\004from\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\017messageManifest\030\003 \001(\014\022\r\n\005seqNo\030\004 \002(\004\022" +
+      "\"\n\nackReplyTo\030\005 \002(\0132\016.UniqueAddress\"G\n\030S" +
+      "ystemMessageDeliveryAck\022\r\n\005seqNo\030\001 \002(\004\022\034" +
+      "\n\004from\030\002 \002(\0132\016.UniqueAddress\"K\n\007Address\022" +
+      "\020\n\010protocol\030\001 \002(\t\022\016\n\006system\030\002 \002(\t\022\020\n\010hos" +
+      "tname\030\003 \002(\t\022\014\n\004port\030\004 \002(\r\"7\n\rUniqueAddre" +
+      "ss\022\031\n\007address\030\001 \002(\0132\010.Address\022\013\n\003uid\030\002 \002" +
+      "(\004B\017\n\013akka.remoteH\001"
     };
     akka.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new akka.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -6093,7 +6183,7 @@ public final class ArteryControlFormats {
           internal_static_CompressionTableAdvertisement_fieldAccessorTable = new
             akka.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_CompressionTableAdvertisement_descriptor,
-              new java.lang.String[] { "From", "TableVersion", "Keys", "Values", });
+              new java.lang.String[] { "From", "OriginUid", "TableVersion", "Keys", "Values", });
           internal_static_CompressionTableAdvertisementAck_descriptor =
             getDescriptor().getMessageTypes().get(3);
           internal_static_CompressionTableAdvertisementAck_fieldAccessorTable = new
diff --git a/akka-remote/src/main/protobuf/ArteryControlFormats.proto b/akka-remote/src/main/protobuf/ArteryControlFormats.proto
index 5f47710670..aafff98016 100644
--- a/akka-remote/src/main/protobuf/ArteryControlFormats.proto
+++ b/akka-remote/src/main/protobuf/ArteryControlFormats.proto
@@ -24,13 +24,14 @@ message MessageWithAddress {
 // CompressionProtocol.ClassManifestCompressionAdvertisement
 message CompressionTableAdvertisement {
   required UniqueAddress from = 1;
-  required uint32 tableVersion = 2;
+  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 = 3;
-  repeated uint32 values = 4;
+  repeated string keys = 4;
+  repeated uint32 values = 5;
 }
 
 // CompressionProtocol.ActorRefCompressionAdvertisementAck
diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala
index c0d3e02ce0..799b114829 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala
@@ -295,7 +295,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R
   @volatile private[this] var aeronErrorLogTask: Cancellable = _
   @volatile private[this] var areonErrorLog: AeronErrorLog = _
 
-  @volatile private[this] var inboundCompressions: Option[InboundCompressions] = None
+  @volatile private[this] var _inboundCompressions: Option[InboundCompressions] = None
+  def inboundCompressions: Option[InboundCompressions] = _inboundCompressions
 
   def bindAddress: UniqueAddress = _bindAddress
   override def localAddress: UniqueAddress = _localAddress
@@ -421,7 +422,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R
     runInboundStreams()
     topLevelFREvents.loFreq(Transport_StartupFinished, NoMetaData)
 
-    log.info("Remoting started; listening on address: {}", defaultAddress)
+    log.info("Remoting started; listening on address: [{}] with uid [{}]", localAddress.address, localAddress.uid)
   }
 
   private lazy val shutdownHook = new Thread {
@@ -572,7 +573,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R
   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)
+    _inboundCompressions = Some(compressions)
 
     runInboundControlStream(noCompressions) // TODO should understand compressions too
     runInboundOrdinaryMessagesStream(compressions)
@@ -598,31 +599,43 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R
             import CompressionProtocol._
             m match {
               case ActorRefCompressionAdvertisement(from, table) ⇒
-                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))
+                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) ⇒
-                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))
+                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(_.confirmActorRefCompressionAdvertisement(from.uid, tableVersion))
+                inboundCompressions.foreach(_.confirmClassManifestCompressionAdvertisement(from.uid, tableVersion))
             }
 
           case Quarantined(from, to) if to == localAddress ⇒
@@ -722,6 +735,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R
         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()
@@ -759,6 +774,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R
     } yield {
       topLevelFREvents.loFreq(Transport_Stopped, NoMetaData)
 
+      _inboundCompressions.foreach(_.close())
+      _inboundCompressions = None
+
       if (aeronErrorLogTask != null) {
         aeronErrorLogTask.cancel()
         topLevelFREvents.loFreq(Transport_AeronErrorLogTaskStopped, NoMetaData)
diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala
index 3f39dd575b..15fb1d2054 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala
@@ -152,7 +152,7 @@ private[remote] class Association(
     timeoutAfter(result, changeCompressionTimeout, new ChangeOutboundCompressionFailed)
   }
 
-  private[artery] def clearCompression(): Future[Done] = {
+  private def clearOutboundCompression(): Future[Done] = {
     import transport.system.dispatcher
     val c = changeOutboundCompression
     val result =
@@ -162,6 +162,9 @@ private[remote] class Association(
     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))
@@ -227,7 +230,7 @@ private[remote] class Association(
         // 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
-        clearCompression().map { _ ⇒
+        clearOutboundCompression().map { _ ⇒
           current.uniqueRemoteAddressPromise.trySuccess(peer)
           current.uniqueRemoteAddressValue() match {
             case Some(`peer`) ⇒
@@ -240,6 +243,7 @@ private[remote] class Association(
                     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
                 }
@@ -366,8 +370,8 @@ private[remote] class Association(
                 log.warning(
                   "Association to [{}] with UID [{}] is irrecoverably failed. Quarantining address. {}",
                   remoteAddress, u, reason)
-                // clear outbound compression
-                clearCompression()
+                clearOutboundCompression()
+                clearInboundCompression(u)
                 // FIXME when we complete the switch to Long UID we must use Long here also, issue #20644
                 transport.eventPublisher.notifyListeners(QuarantinedEvent(remoteAddress, u.toInt))
                 // end delivery of system messages to that incarnation after this point
diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala
index 1786f527c9..0c21194e43 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala
@@ -89,8 +89,6 @@ private[remote] object EnvelopeBuffer {
 
   val UsAscii = Charset.forName("US-ASCII")
 
-  val DeadLettersCode = 0
-
   // accessing the internal char array of String when writing literal strings to ByteBuffer
   val StringValueFieldOffset = Unsafe.instance.objectFieldOffset(classOf[String].getDeclaredField("value"))
 }
@@ -106,6 +104,8 @@ private[remote] object HeaderBuilder {
 
   def out(): HeaderBuilder =
     new HeaderBuilderImpl(NoInboundCompressions, CompressionTable.empty[ActorRef], CompressionTable.empty[String])
+
+  final val DeadLettersCode = -1
 }
 
 /**
@@ -123,6 +123,8 @@ private[remote] sealed trait HeaderBuilder {
   def inboundActorRefCompressionTableVersion: Int
   def inboundClassManifestCompressionTableVersion: Int
 
+  def useOutboundCompression(on: Boolean): Unit
+
   def outboundActorRefCompression: CompressionTable[ActorRef]
   def setOutboundActorRefCompression(table: CompressionTable[ActorRef]): Unit
 
@@ -172,7 +174,14 @@ private[remote] sealed trait HeaderBuilder {
   def serializer: Int
 
   def setManifest(manifest: String): Unit
-  def manifest(originUid: Long): String
+  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
+
 }
 
 /**
@@ -197,27 +206,41 @@ 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 = _
-  var _flags: Byte = _
-  var _uid: Long = _
+  var _version: Byte = 0
+  var _flags: Byte = 0
+  var _uid: Long = 0
   var _inboundActorRefCompressionTableVersion: Int = 0
   var _inboundClassManifestCompressionTableVersion: Int = 0
+  var _useOutboundCompression: Boolean = true
 
   var _senderActorRef: String = null
   var _senderActorRefIdx: Int = -1
   var _recipientActorRef: String = null
   var _recipientActorRefIdx: Int = -1
 
-  var _serializer: Int = _
+  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
 
@@ -234,6 +257,9 @@ private[remote] final class HeaderBuilderImpl(
   override def inboundActorRefCompressionTableVersion: Int = _inboundActorRefCompressionTableVersion
   override def inboundClassManifestCompressionTableVersion: Int = _inboundClassManifestCompressionTableVersion
 
+  def useOutboundCompression(on: Boolean): Unit =
+    _useOutboundCompression = on
+
   def setOutboundActorRefCompression(table: CompressionTable[ActorRef]): Unit = {
     _outboundActorRefCompression = table
   }
@@ -245,39 +271,49 @@ private[remote] final class HeaderBuilderImpl(
   def outboundClassManifestCompression: CompressionTable[String] = _outboundClassManifestCompression
 
   override def setSenderActorRef(ref: ActorRef): Unit = {
-    _senderActorRefIdx = outboundActorRefCompression.compress(ref)
-    if (_senderActorRefIdx == -1) _senderActorRef = Serialization.serializedActorPath(ref) // includes local address from `currentTransportInformation`
+    // 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 = EnvelopeBuffer.DeadLettersCode
+    _senderActorRefIdx = DeadLettersCode
   }
   override def isNoSender: Boolean =
-    (_senderActorRef eq null) && _senderActorRefIdx == EnvelopeBuffer.DeadLettersCode
-  override def senderActorRef(originUid: Long): OptionVal[ActorRef] =
-    if (_senderActorRef eq null)
+    (_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 = EnvelopeBuffer.DeadLettersCode
+    _recipientActorRefIdx = DeadLettersCode
   }
   def isNoRecipient: Boolean =
-    (_recipientActorRef eq null) && _recipientActorRefIdx == EnvelopeBuffer.DeadLettersCode
+    (_recipientActorRef eq null) && _recipientActorRefIdx == DeadLettersCode
 
   def setRecipientActorRef(ref: ActorRef): Unit = {
-    _recipientActorRefIdx = outboundActorRefCompression.compress(ref)
-    if (_recipientActorRefIdx == -1) {
+    if (_useOutboundCompression) {
+      _recipientActorRefIdx = outboundActorRefCompression.compress(ref)
+      if (_recipientActorRefIdx == -1) _recipientActorRef = toSerializationFormat.getOrCompute(ref)
+    } else
       _recipientActorRef = toSerializationFormat.getOrCompute(ref)
-    }
   }
-  def recipientActorRef(originUid: Long): OptionVal[ActorRef] =
-    if (_recipientActorRef eq null)
+  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)
 
@@ -288,16 +324,18 @@ private[remote] final class HeaderBuilderImpl(
     _serializer
 
   override def setManifest(manifest: String): Unit = {
-    _manifestIdx = outboundClassManifestCompression.compress(manifest)
-    if (_manifestIdx == -1) _manifest = manifest
+    if (_useOutboundCompression) {
+      _manifestIdx = outboundClassManifestCompression.compress(manifest)
+      if (_manifestIdx == -1) _manifest = manifest
+    } else
+      _manifest = manifest
   }
-  override def manifest(originUid: Long): String = {
-    if (_manifest ne null) _manifest
+  override def manifest(originUid: Long): OptionVal[String] = {
+    if (_manifest ne null) OptionVal.Some(_manifest)
     else {
-      _manifest = inboundCompression.decompressClassManifest(
+      inboundCompression.decompressClassManifest(
         originUid,
-        inboundClassManifestCompressionTableVersion, _manifestIdx).get
-      _manifest
+        inboundClassManifestCompressionTableVersion, _manifestIdx)
     }
   }
 
@@ -434,7 +472,7 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) {
       header._senderActorRef = null
       header._senderActorRefIdx = idx
     } else {
-      header._senderActorRef = readLiteral()
+      header._senderActorRef = emptyAsNull(readLiteral())
     }
 
     // Deserialize recipient
@@ -444,7 +482,7 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) {
       header._recipientActorRef = null
       header._recipientActorRefIdx = idx
     } else {
-      header._recipientActorRef = readLiteral()
+      header._recipientActorRef = emptyAsNull(readLiteral())
     }
 
     // Deserialize class manifest
@@ -458,6 +496,10 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) {
     }
   }
 
+  private def emptyAsNull(s: String): String =
+    if (s == "") null
+    else s
+
   private def readLiteral(): String = {
     val length = byteBuffer.getShort
     if (length == 0) ""
@@ -477,7 +519,7 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) {
   }
 
   private def writeLiteral(tagOffset: Int, literal: String): Unit = {
-    val length = literal.length
+    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")
 
diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala
index 2ef13b9b3f..58474c3be3 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala
@@ -94,6 +94,11 @@ private[remote] class Encoder(
         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
@@ -288,38 +293,62 @@ private[remote] class Decoder(
       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] = headerBuilder.recipientActorRef(originUid) match {
+        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
         }
 
-        if (recipient.isEmpty && headerBuilder.recipientActorRefPath.isEmpty && !headerBuilder.isNoRecipient) {
-          log.debug("Dropping message for unknown recipient. It was probably sent from system [{}] with compression " +
+        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 sender: OptionVal[InternalActorRef] = 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
-          }
-
-          val classManifest = headerBuilder.manifest(originUid)
+          val classManifest = classManifestOpt.get
 
           if ((messageCount & heavyHitterMask) == 0) {
             // --- hit refs and manifests for heavy-hitter counting
diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala
index 63d8c05693..801547d87f 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala
@@ -230,7 +230,7 @@ private[akka] class InboundHandshake(inboundContext: InboundContext, inControlSt
               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, inboundContext.localAddress.uid, env.originUid)
+              env.message.getClass.getName, env.originUid, inboundContext.localAddress.uid)
           pull(in)
         }
       }
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
index 70bbdff2b3..106b733c16 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala
@@ -8,30 +8,30 @@ import java.util
 import java.util.Comparator
 
 /** INTERNAL API: Versioned compression table to be advertised between systems */
-private[remote] final case class CompressionTable[T](version: Int, map: Map[T, Int]) {
+private[remote] final case class CompressionTable[T](originUid: Long, version: Int, dictionary: Map[T, Int]) {
   import CompressionTable.NotCompressedId
 
   def compress(value: T): Int =
-    map.get(value) match {
+    dictionary.get(value) match {
       case Some(id) ⇒ id
       case None     ⇒ NotCompressedId
     }
 
   def invert: DecompressionTable[T] =
-    if (map.isEmpty) DecompressionTable.empty[T].copy(version = version)
+    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((map.size * (map.size + 1)) / 2) /* Dirichlet */
-      require(map.values.min == 0, "Compression table should start allocating from 0, yet lowest allocated id was " + map.values.min)
-      require(map.values.sum + map.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: " + map)
+      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)](map.size).asInstanceOf[Array[(T, Int)]]
-      val ts = Array.ofDim[Object](map.size).asInstanceOf[Array[T]]
+      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 = map.iterator
+      val mit = dictionary.iterator
       while (i < tups.length) {
         tups(i) = mit.next()
         i += 1
@@ -44,7 +44,7 @@ private[remote] final case class CompressionTable[T](version: Int, map: Map[T, I
         i += 1
       }
 
-      DecompressionTable[T](version, ts)
+      DecompressionTable[T](originUid, version, ts)
     }
 }
 /** INTERNAL API */
@@ -57,6 +57,6 @@ private[remote] object CompressionTable {
   }
   def compareBy2ndValue[T]: Comparator[Tuple2[T, Int]] = CompareBy2ndValue.asInstanceOf[Comparator[(T, Int)]]
 
-  private[this] val _empty = new CompressionTable[Any](0, Map.empty)
+  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
index d6ade2eae8..a53b5e7258 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala
@@ -5,31 +5,28 @@
 package akka.remote.artery.compress
 
 /** INTERNAL API */
-private[artery] final case class DecompressionTable[T](version: Int, table: Array[T]) {
+private[artery] final case class DecompressionTable[T](originUid: Long, version: Int, table: Array[T]) {
   // TODO version maybe better as Long? // OR implement roll-over
   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].")
+        s"Only $length ids allocated in table version [$version] for origin [$originUid].")
     table(idx)
   }
 
   def invert: CompressionTable[T] =
-    CompressionTable(version, Map(table.zipWithIndex: _*))
+    CompressionTable(originUid, version, Map(table.zipWithIndex: _*))
 
   /** Writes complete table as String (heavy operation) */
-  def toDebugString =
-    getClass.getName +
-      s"(version: $version, " +
-      (
-        if (length == 0) "[empty]"
-        else s"table: [${table.zipWithIndex.map({ case (t, i) ⇒ s"$i -> $t" }).mkString(",")}") + "])"
+  override def toString =
+    s"DecompressionTable($originUid, $version, " +
+      s"Map(${table.zipWithIndex.map({ case (t, i) ⇒ s"$i -> $t" }).mkString(",")}))"
 }
 
 /** INTERNAL API */
 private[artery] object DecompressionTable {
-  private[this] val _empty = DecompressionTable(0, Array.empty)
+  private[this] val _empty = DecompressionTable(0, 0, Array.empty)
   def empty[T] = _empty.asInstanceOf[DecompressionTable[T]]
 }
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
index 01d7928b7a..c6deac61cf 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala
@@ -15,6 +15,11 @@ import akka.util.{ OptionVal, PrettyDuration }
 import org.agrona.collections.Long2ObjectHashMap
 
 import scala.annotation.tailrec
+import scala.concurrent.Future
+import akka.Done
+import akka.actor.Cancellable
+import java.util.concurrent.atomic.AtomicBoolean
+import java.util.concurrent.atomic.AtomicInteger
 
 /**
  * INTERNAL API
@@ -30,6 +35,17 @@ private[remote] trait InboundCompressions {
   def hitClassManifest(originUid: Long, remote: Address, manifest: String, n: Int): Unit
   def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String]
   def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit
+
+  /**
+   * Cancel advertisement scheduling
+   */
+  def close(): Unit
+
+  /**
+   * Remove compression and cancel advertisement scheduling for a specific origin
+   */
+  def close(originUid: Long): Unit
+
 }
 
 /**
@@ -42,14 +58,13 @@ private[remote] final class InboundCompressionsImpl(
   inboundContext: InboundContext,
   settings:       ArterySettings.Compression) extends InboundCompressions {
 
-  // TODO we also must remove the ones that won't be used anymore - when quarantine triggers?
-  //      Why is that important? Won't be naturally be removed in new advertisements since they
-  //      are not used any more?
+  private val stopped = new AtomicBoolean
+
   private[this] val _actorRefsIns = new Long2ObjectHashMap[InboundActorRefCompression]()
   private val createInboundActorRefsForOrigin = new LongFunction[InboundActorRefCompression] {
     override def apply(originUid: Long): InboundActorRefCompression = {
       val actorRefHitters = new TopHeavyHitters[ActorRef](settings.ActorRefs.Max)
-      new InboundActorRefCompression(system, settings, originUid, inboundContext, actorRefHitters)
+      new InboundActorRefCompression(system, settings, originUid, inboundContext, actorRefHitters, stopped)
     }
   }
   private def actorRefsIn(originUid: Long): InboundActorRefCompression =
@@ -59,7 +74,7 @@ private[remote] final class InboundCompressionsImpl(
   private val createInboundManifestsForOrigin = new LongFunction[InboundManifestCompression] {
     override def apply(originUid: Long): InboundManifestCompression = {
       val manifestHitters = new TopHeavyHitters[String](settings.Manifests.Max)
-      new InboundManifestCompression(system, settings, originUid, inboundContext, manifestHitters)
+      new InboundManifestCompression(system, settings, originUid, inboundContext, manifestHitters, stopped)
     }
   }
   private def classManifestsIn(originUid: Long): InboundManifestCompression =
@@ -75,7 +90,10 @@ private[remote] final class InboundCompressionsImpl(
   }
 
   override def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = {
-    actorRefsIn(originUid).confirmAdvertisement(tableVersion)
+    _actorRefsIns.get(originUid) match {
+      case null ⇒ // ignore, it was closed
+      case a    ⇒ a.confirmAdvertisement(tableVersion)
+    }
   }
 
   // class manifest compression ---
@@ -86,8 +104,23 @@ private[remote] final class InboundCompressionsImpl(
     if (ArterySettings.Compression.Debug) println(s"[compress] hitClassManifest($originUid, $address, $manifest, $n)")
     classManifestsIn(originUid).increment(address, manifest, n)
   }
-  override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit =
-    actorRefsIn(originUid).confirmAdvertisement(tableVersion)
+  override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = {
+    _classManifestsIns.get(originUid) match {
+      case null ⇒ // ignore, it was closed
+      case a    ⇒ a.confirmAdvertisement(tableVersion)
+    }
+  }
+
+  override def close(): Unit = stopped.set(true)
+
+  override def close(originUid: Long): Unit = {
+    actorRefsIn(originUid).close()
+    classManifestsIn(originUid).close()
+    // FIXME This is not safe, it can be created again (concurrently), at least in theory.
+    //       However, we should make the inbound compressions owned by the Decoder and it doesn't have to be thread-safe
+    _actorRefsIns.remove(originUid)
+    _classManifestsIns.remove(originUid)
+  }
 
   // testing utilities ---
 
@@ -117,24 +150,18 @@ private[remote] final class InboundActorRefCompression(
   settings:       ArterySettings.Compression,
   originUid:      Long,
   inboundContext: InboundContext,
-  heavyHitters:   TopHeavyHitters[ActorRef]) extends InboundCompression[ActorRef](system, settings, originUid, inboundContext, heavyHitters) {
-
-  preAllocate(system.deadLetters)
-
-  /* Since the table is empty here, anything we increment here becomes a heavy hitter immediately. */
-  def preAllocate(allocations: ActorRef*): Unit = {
-    allocations foreach { ref ⇒ increment(null, ref, 100000) }
-  }
+  heavyHitters:   TopHeavyHitters[ActorRef],
+  stopped:        AtomicBoolean)
+  extends InboundCompression[ActorRef](system, settings, originUid, inboundContext, heavyHitters, stopped) {
 
   override def decompress(tableVersion: Int, idx: Int): OptionVal[ActorRef] =
-    if (idx == 0) OptionVal.Some(system.deadLetters)
-    else super.decompressInternal(tableVersion, idx, 0)
+    super.decompressInternal(tableVersion, idx, 0)
 
-  scheduleNextTableAdvertisement()
   override protected def tableAdvertisementInterval = settings.ActorRefs.AdvertisementInterval
 
   override def advertiseCompressionTable(outboundContext: OutboundContext, table: CompressionTable[ActorRef]): Unit = {
-    log.debug(s"Advertise ActorRef compression [$table], from [${inboundContext.localAddress}] to [${outboundContext.remoteAddress}]")
+    log.debug(s"Advertise {} compression [{}] to [{}#{}]", Logging.simpleName(getClass), table, outboundContext.remoteAddress,
+      originUid)
     outboundContext.sendControl(CompressionProtocol.ActorRefCompressionAdvertisement(inboundContext.localAddress, table))
   }
 }
@@ -144,13 +171,15 @@ final class InboundManifestCompression(
   settings:       ArterySettings.Compression,
   originUid:      Long,
   inboundContext: InboundContext,
-  heavyHitters:   TopHeavyHitters[String]) extends InboundCompression[String](system, settings, originUid, inboundContext, heavyHitters) {
+  heavyHitters:   TopHeavyHitters[String],
+  stopped:        AtomicBoolean)
+  extends InboundCompression[String](system, settings, originUid, inboundContext, heavyHitters, stopped) {
 
-  scheduleNextTableAdvertisement()
   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)
+    log.debug(s"Advertise {} compression [{}] to [{}#{}]", Logging.simpleName(getClass), table, outboundContext.remoteAddress,
+      originUid)
     outboundContext.sendControl(CompressionProtocol.ClassManifestCompressionAdvertisement(inboundContext.localAddress, table))
   }
 
@@ -198,14 +227,34 @@ private[remote] abstract class InboundCompression[T >: Null](
   val settings:     ArterySettings.Compression,
   originUid:        Long,
   inboundContext:   InboundContext,
-  val heavyHitters: TopHeavyHitters[T]) {
+  val heavyHitters: TopHeavyHitters[T],
+  stopped:          AtomicBoolean) {
 
-  lazy val log = Logging(system, getClass.getSimpleName)
+  val log = Logging(system, getClass)
 
-  private[this] val state: AtomicReference[InboundCompression.State[T]] = new AtomicReference(InboundCompression.State.empty)
+  // 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 */
@@ -237,46 +286,53 @@ private[remote] abstract class InboundCompression[T >: Null](
       if (value != null) OptionVal.Some[T](value)
       else throw new UnknownCompressedIdException(idx)
     } else if (incomingTableVersion < activeVersion) {
-      log.debug("Received value compressed with old table: [{}], current table version is: [{}]", incomingTableVersion, activeVersion)
-      OptionVal.None
-    } else if (incomingTableVersion == current.nextTable.version) {
       log.debug(
-        "Received first value compressed using the next prepared compression table, flipping to it (version: {})",
-        current.nextTable.version)
+        "Received value from originUid [{}] compressed with old table: [{}], current table version is: [{}]",
+        originUid, incomingTableVersion, activeVersion)
+      OptionVal.None
+    } 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, activeTable will not be able to handle this
+      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 is using compression table version higher than the highest allocated table on this node. " +
+        "Inbound message from originUid [{}] is using unknown compression table version. " +
           "It was probably sent with compression table built for previous incarnation of this system. " +
-          "State: activeTable: {}, nextTable: {}, incoming tableVersion: {}",
-        activeVersion, current.nextTable.version, incomingTableVersion)
+          "Versions activeTable: {}, nextTable: {}, incomingTable: {}",
+        originUid, activeVersion, current.nextTable.version, incomingTableVersion)
       OptionVal.None
     }
   }
 
-  def confirmAdvertisement(tableVersion: Int): Unit = {
+  @tailrec final def confirmAdvertisement(tableVersion: Int): 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 {}", tableVersion)
+          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 {} but in progress {}", 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` occurance for the given key and call `heavyHittedDetected` if element has become a heavy hitter.
+   * 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 */
@@ -296,27 +352,6 @@ private[remote] abstract class InboundCompression[T >: Null](
   private[remote] def triggerNextTableAdvertisement(): Unit = // TODO use this in tests for triggering
     runNextTableAdvertisement()
 
-  def scheduleNextTableAdvertisement(): Unit =
-    tableAdvertisementInterval match {
-      case d: FiniteDuration ⇒
-        try {
-          system.scheduler.scheduleOnce(d, ScheduledTableAdvertisementRunnable)(system.dispatcher)
-          log.debug("Scheduled {} advertisement in [{}] from now...", getClass.getSimpleName, PrettyDuration.format(tableAdvertisementInterval, includeNanos = false, 1))
-        } catch {
-          case ex: IllegalStateException ⇒
-            // this is usually harmless
-            log.debug("Unable to schedule {} advertisement, " +
-              "likely system is shutting down. Reason: {}", getClass.getName, ex.getMessage)
-        }
-      case _ ⇒ // ignore...
-    }
-
-  private val ScheduledTableAdvertisementRunnable = new Runnable {
-    override def run(): Unit =
-      try runNextTableAdvertisement()
-      finally scheduleNextTableAdvertisement()
-  }
-
   /**
    * Entry point to advertising a new compression table.
    *
@@ -328,32 +363,52 @@ private[remote] abstract class InboundCompression[T >: Null](
    * Triggers compression table advertisement. May be triggered by schedule or manually, i.e. for testing.
    */
   private[remote] def runNextTableAdvertisement() = {
-    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) ⇒
-            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))
-              advertiseCompressionTable(association, table)
+    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.warning("No Association for originUid [{}] yet, unable to advertise compression table.", 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) ⇒
-        // The ActorRefCompressionAdvertisement message is resent because it can be lost
-        log.debug("Advertisment in progress for version {}, resending", inProgress.version)
-        inboundContext.association(originUid) match {
-          case OptionVal.Some(association) ⇒
-            advertiseCompressionTable(association, inProgress) // resend
-          case OptionVal.None ⇒
-        }
+        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)
+          }
+      }
     }
   }
 
@@ -365,7 +420,7 @@ private[remote] abstract class InboundCompression[T >: Null](
 
   private def prepareCompressionAdvertisement(nextTableVersion: Int): CompressionTable[T] = {
     // TODO surely we can do better than that, optimise
-    CompressionTable(nextTableVersion, Map(heavyHitters.snapshot.filterNot(_ == null).zipWithIndex: _*))
+    CompressionTable(originUid, nextTableVersion, Map(heavyHitters.snapshot.filterNot(_ == null).zipWithIndex: _*))
   }
 
   override def toString =
@@ -379,3 +434,26 @@ final class UnknownCompressedIdException(id: Long)
       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!
+ */
+case object NoInboundCompressions extends InboundCompressions {
+  override def hitActorRef(originUid: Long, remote: Address, ref: ActorRef, n: Int): Unit = ()
+  override def decompressActorRef(originUid: Long, tableVersion: Int, 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: Int): Unit = ()
+
+  override def hitClassManifest(originUid: Long, remote: Address, manifest: String, n: Int): Unit = ()
+  override def decompressClassManifest(originUid: Long, tableVersion: Int, 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: Int): Unit = ()
+
+  override def close(): Unit = ()
+
+  override def close(originUid: Long): Unit = ()
+}
diff --git a/akka-remote/src/main/scala/akka/remote/artery/compress/NoInboundCompressions.scala b/akka-remote/src/main/scala/akka/remote/artery/compress/NoInboundCompressions.scala
deleted file mode 100644
index 7219cb0cfa..0000000000
--- a/akka-remote/src/main/scala/akka/remote/artery/compress/NoInboundCompressions.scala
+++ /dev/null
@@ -1,27 +0,0 @@
-/**
- * Copyright (C) 2016 Lightbend Inc. 
- */
-package akka.remote.artery.compress
-
-import akka.actor.{ ActorRef, Address }
-import akka.util.OptionVal
-
-/**
- * INTERNAL API
- *
- * Literarily, no compression!
- */
-case object NoInboundCompressions extends InboundCompressions {
-  override def hitActorRef(originUid: Long, remote: Address, ref: ActorRef, n: Int): Unit = ()
-  override def decompressActorRef(originUid: Long, tableVersion: Int, 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: Int): Unit = ()
-
-  override def hitClassManifest(originUid: Long, remote: Address, manifest: String, n: Int): Unit = ()
-  override def decompressClassManifest(originUid: Long, tableVersion: Int, 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: Int): Unit = ()
-}
-
diff --git a/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala
index 98fe74a0ad..30499f86a4 100644
--- a/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala
+++ b/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala
@@ -26,6 +26,8 @@ private[akka] object ArteryMessageSerializer {
   private val SystemMessageEnvelopeManifest = "j"
   private val SystemMessageDeliveryAckManifest = "k"
   private val SystemMessageDeliveryNackManifest = "l"
+
+  private final val DeadLettersRepresentation = ""
 }
 
 /** INTERNAL API */
@@ -93,42 +95,45 @@ private[akka] final class ArteryMessageSerializer(val system: ExtendedActorSyste
   def deserializeQuarantined(quarantined: ArteryControlFormats.Quarantined): Quarantined =
     Quarantined(deserializeUniqueAddress(quarantined.getFrom), deserializeUniqueAddress(quarantined.getTo))
 
-  def serializeActorRef(ref: ActorRef): String = Serialization.serializedActorPath(ref)
-  def deserializeActorRef(str: String): ActorRef = system.provider.resolveActorRef(str)
+  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 =
-    // FIXME: is it guaranteed that idx 0 is reserved for DeadLetters? In the best case, this knowledge should be managed in only one place.
-    serializeCompressionAdvertisement(adv)(serializeActorRef, _ != 0 /* 0 is reserved for DeadLetters and doesn't need to be serialized explicitly */ )
+    serializeCompressionAdvertisement(adv)(serializeActorRef)
 
   def deserializeActorRefCompressionAdvertisement(bytes: Array[Byte]): ActorRefCompressionAdvertisement =
-    deserializeCompressionAdvertisement(bytes, deserializeActorRef, ActorRefCompressionAdvertisement, Seq(system.deadLetters → 0) /* add DeadLetters explicitly */ )
+    deserializeCompressionAdvertisement(bytes, deserializeActorRef, ActorRefCompressionAdvertisement)
 
-  def serializeCompressionAdvertisement[T](adv: CompressionAdvertisement[T])(keySerializer: T ⇒ String, valueFilter: Int ⇒ Boolean = _ ⇒ true): ArteryControlFormats.CompressionTableAdvertisement = {
+  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.map.foreach {
-      case (key, value) if valueFilter(value) ⇒
+    adv.table.dictionary.foreach {
+      case (key, value) ⇒
         builder
           .addKeys(keySerializer(key))
           .addValues(value)
-      case _ ⇒
     }
 
     builder.build
   }
 
-  def deserializeCompressionAdvertisement[T, U](bytes: Array[Byte], keyDeserializer: String ⇒ T, create: (UniqueAddress, CompressionTable[T]) ⇒ U, extraValues: Seq[(T, Int)] = Nil): U = {
+  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 */ ) ++
-        extraValues
+        protoAdv.getValuesList.asScala.asInstanceOf[Iterable[Int]] /* to avoid having to call toInt explicitly */ )
 
-    val table = CompressionTable(protoAdv.getTableVersion, kvs.toMap)
+    val table = CompressionTable(protoAdv.getOriginUid, protoAdv.getTableVersion, kvs.toMap)
     create(deserializeUniqueAddress(protoAdv.getFrom), table)
   }
 
@@ -210,4 +215,4 @@ private[akka] final class ArteryMessageSerializer(val system: ExtendedActorSyste
 
   def deserializeAddress(address: ArteryControlFormats.Address): Address =
     Address(address.getProtocol, address.getSystem, address.getHostname, address.getPort)
-}
\ No newline at end of file
+}
diff --git a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala
index c6f85a2e98..ecd049b445 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala
@@ -33,10 +33,10 @@ class EnvelopeBufferSpec extends AkkaSpec {
     val idxToManifest = manifestToIdx.map(_.swap)
 
     val outboundActorRefTable: CompressionTable[ActorRef] =
-      CompressionTable(version = 0xCAFE, refToIdx)
+      CompressionTable(17L, version = 0xCAFE, refToIdx)
 
     val outboundClassManifestTable: CompressionTable[String] =
-      CompressionTable(version = 0xBABE, manifestToIdx)
+      CompressionTable(17L, version = 0xBABE, manifestToIdx)
 
     override def hitActorRef(originUid: Long, remote: Address, ref: ActorRef, n: Int): Unit = ()
     override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = OptionVal(idxToRef(idx))
@@ -45,6 +45,9 @@ class EnvelopeBufferSpec extends AkkaSpec {
     override def hitClassManifest(originUid: Long, remote: Address, manifest: String, n: Int): Unit = ()
     override def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] = OptionVal(idxToManifest(idx))
     override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = ()
+
+    override def close(): Unit = ()
+    override def close(originUid: Long): Unit = ()
   }
 
   "EnvelopeBuffer" must {
@@ -83,7 +86,7 @@ class EnvelopeBufferSpec extends AkkaSpec {
       headerOut.senderActorRefPath should ===(OptionVal.None)
       headerOut.recipientActorRef(originUid).get.path.toSerializationFormat should ===("akka://EnvelopeBufferSpec/compressable1")
       headerOut.recipientActorRefPath should ===(OptionVal.None)
-      headerOut.manifest(originUid) should ===("manifest1")
+      headerOut.manifest(originUid).get should ===("manifest1")
     }
 
     "be able to encode and decode headers with uncompressed literals" in {
@@ -116,7 +119,7 @@ class EnvelopeBufferSpec extends AkkaSpec {
       headerOut.senderActorRef(originUid) should ===(OptionVal.None)
       headerOut.recipientActorRefPath should ===(OptionVal.Some("akka://EnvelopeBufferSpec/uncompressable11"))
       headerOut.recipientActorRef(originUid) should ===(OptionVal.None)
-      headerOut.manifest(originUid) should ===("uncompressable3333")
+      headerOut.manifest(originUid).get should ===("uncompressable3333")
     }
 
     "be able to encode and decode headers with mixed literals" in {
@@ -144,7 +147,7 @@ class EnvelopeBufferSpec extends AkkaSpec {
       headerOut.senderActorRefPath should ===(OptionVal.None)
       headerOut.recipientActorRefPath should ===(OptionVal.Some("akka://EnvelopeBufferSpec/uncompressable1"))
       headerOut.recipientActorRef(originUid) should ===(OptionVal.None)
-      headerOut.manifest(originUid) should ===("manifest1")
+      headerOut.manifest(originUid).get should ===("manifest1")
 
       val senderRef = minimalRef("uncompressable0")
 
@@ -171,7 +174,7 @@ class EnvelopeBufferSpec extends AkkaSpec {
       headerOut.senderActorRef(originUid) should ===(OptionVal.None)
       headerOut.recipientActorRef(originUid).get.path.toSerializationFormat should ===("akka://EnvelopeBufferSpec/reallylongcompressablestring")
       headerOut.recipientActorRefPath should ===(OptionVal.None)
-      headerOut.manifest(originUid) should ===("longlonglongliteralmanifest")
+      headerOut.manifest(originUid).get should ===("longlonglongliteralmanifest")
     }
 
     "be able to encode and decode headers with mixed literals and payload" in {
@@ -197,7 +200,7 @@ class EnvelopeBufferSpec extends AkkaSpec {
       headerOut.senderActorRefPath should ===(OptionVal.None)
       headerOut.recipientActorRefPath should ===(OptionVal.Some("akka://EnvelopeBufferSpec/uncompressable1"))
       headerOut.recipientActorRef(originUid) should ===(OptionVal.None)
-      headerOut.manifest(originUid) should ===("manifest1")
+      headerOut.manifest(originUid).get should ===("manifest1")
 
       ByteString.fromByteBuffer(envelope.byteBuffer) should ===(payload)
     }
diff --git a/akka-remote/src/test/scala/akka/remote/artery/compress/ActorRefCompressionIntegrationSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/ActorRefCompressionIntegrationSpec.scala
deleted file mode 100644
index f59005268a..0000000000
--- a/akka-remote/src/test/scala/akka/remote/artery/compress/ActorRefCompressionIntegrationSpec.scala
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Copyright (C) 2016 Lightbend Inc. 
- */
-
-package akka.remote.artery.compress
-
-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 org.scalatest.concurrent.{ Eventually, PatienceConfiguration }
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
-
-class ActorRefCompressionIntegrationSpec extends AkkaSpec(CompressionIntegrationSpec.commonConfig)
-  with ImplicitSender with BeforeAndAfter with Eventually {
-  import CompressionIntegrationSpec._
-
-  implicit val t = Timeout(3.seconds)
-  var systemB = ActorSystem("systemB", configB)
-
-  "Outgoing ActorRef compression table" must {
-    "compress chatty actor" in {
-      val messagesToExchange = 10
-
-      // listen for compression table events
-      val aProbe = TestProbe()
-      val b1Probe = TestProbe()
-      system.eventStream.subscribe(aProbe.ref, classOf[CompressionProtocol.Events.ReceivedActorRefCompressionTable])
-      systemB.eventStream.subscribe(b1Probe.ref, classOf[CompressionProtocol.Events.ReceivedActorRefCompressionTable])
-
-      def voidSel = system.actorSelection(s"akka://systemB@localhost:$portB/user/void")
-      systemB.actorOf(TestActors.blackholeProps, "void")
-
-      // cause testActor-1 to become a heavy hitter
-      (1 to messagesToExchange).foreach { i ⇒ voidSel ! "hello" } // does not reply, but a hot receiver should be advertised
-
-      val a1 = aProbe.expectMsgType[Events.ReceivedActorRefCompressionTable](10.seconds)
-      info("System [A] received: " + a1)
-      assertCompression[ActorRef](a1.table, 0, _ should ===(system.deadLetters))
-      assertCompression[ActorRef](a1.table, 1, _ should ===(testActor))
-    }
-  }
-
-  def assertCompression[T](table: CompressionTable[T], id: Int, assertion: T ⇒ Unit): Unit = {
-    table.map.find(_._2 == id)
-      .orElse { throw new AssertionError(s"No key was compressed to the id [$id]! Table was: $table") }
-      .foreach(i ⇒ assertion(i._1))
-  }
-
-  def identify(_system: String, port: Int, name: String): ActorRef = {
-    val selection = system.actorSelection(s"akka://${_system}@localhost:$port/user/$name").resolveOne(3.seconds)
-    Await.result(selection, 4.seconds)
-  }
-
-  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/ClassManifestCompressionIntegrationSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/ClassManifestCompressionIntegrationSpec.scala
deleted file mode 100644
index fbd5568d52..0000000000
--- a/akka-remote/src/test/scala/akka/remote/artery/compress/ClassManifestCompressionIntegrationSpec.scala
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Copyright (C) 2016 Lightbend Inc. 
- */
-
-package akka.remote.artery.compress
-
-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 org.scalatest.concurrent.{ Eventually, PatienceConfiguration }
-
-import scala.concurrent.Await
-import scala.concurrent.duration._
-
-class ClassManifestCompressionIntegrationSpec extends AkkaSpec(CompressionIntegrationSpec.commonConfig)
-  with ImplicitSender with BeforeAndAfter with Eventually {
-  import CompressionIntegrationSpec._
-
-  implicit val t = Timeout(3.seconds)
-  var systemB = ActorSystem("systemB", configB)
-
-  "Outgoing Manifest compression table" must {
-    "compress chatty manifest" in {
-      val messagesToExchange = 10
-
-      // listen for compression table events
-      val aProbe = TestProbe()
-      val b1Probe = TestProbe()
-      system.eventStream.subscribe(aProbe.ref, classOf[CompressionProtocol.Events.ReceivedClassManifestCompressionTable])
-      systemB.eventStream.subscribe(b1Probe.ref, classOf[CompressionProtocol.Events.ReceivedClassManifestCompressionTable])
-      systemB.actorOf(TestActors.blackholeProps, "void-2")
-
-      Thread.sleep(1000)
-      val voidRef = Await.result(system.actorSelection(s"akka://systemB@localhost:$portB/user/void-2").resolveOne(3.second), 3.seconds)
-
-      // cause testActor-1 to become a heavy hitter
-      (1 to messagesToExchange).foreach { i ⇒ voidRef ! TestMessage("hello") } // does not reply, but a hot receiver should be advertised
-
-      eventually(PatienceConfiguration.Timeout(20.seconds)) {
-        val a1 = aProbe.expectMsgType[Events.ReceivedClassManifestCompressionTable](10.seconds)
-        info("System [A] received: " + a1)
-        assertCompression[String](a1.table, 0, _ should ===("TestMessageManifest"))
-      }
-    }
-  }
-
-  def assertCompression[T](table: CompressionTable[T], id: Int, assertion: T ⇒ Unit): Unit = {
-    table.map.find(_._2 == id)
-      .orElse { throw new AssertionError(s"No key was compressed to the id [$id]! Table was: $table") }
-      .foreach(i ⇒ assertion(i._1))
-  }
-
-  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
-  }
-
-  override def afterTermination(): Unit =
-    shutdownAllActorSystems()
-
-  private def shutdownAllActorSystems(): Unit = {
-    if (systemB != null) shutdown(systemB)
-  }
-}
-
-import akka.actor.ExtendedActorSystem
-import akka.serialization.SerializerWithStringManifest
-
-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/CompressionIntegrationSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala
index 47f7523603..49337d6bb1 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala
@@ -4,20 +4,28 @@
 
 package akka.remote.artery.compress
 
-import akka.testkit.SocketUtil
 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 {
-  // need the port before systems are started
-  val portB = SocketUtil.temporaryServerAddress("localhost", udp = true).getPort
 
   val commonConfig = ConfigFactory.parseString(s"""
      akka {
        loglevel = INFO
 
        actor {
-         provider = "akka.remote.RemoteActorRefProvider"
-
          serializers {
            test-message = "akka.remote.artery.compress.TestMessageSerializer"
          }
@@ -25,19 +33,255 @@ object CompressionIntegrationSpec {
            "akka.remote.artery.compress.TestMessage" = test-message
          }
        }
-       remote.artery.enabled = on
-       remote.artery.canonical.hostname = localhost
-       remote.artery.canonical.port = 0
-       remote.artery.advanced.handshake-timeout = 10s
 
        remote.artery.advanced.compression {
-         actor-refs.advertisement-interval = 3 seconds
-         manifests.advertisement-interval = 3 seconds
+         actor-refs.advertisement-interval = 2 seconds
+         manifests.advertisement-interval = 2 seconds
        }
-
      }
   """)
 
-  val configB = ConfigFactory.parseString(s"akka.remote.artery.canonical.port = $portB")
-    .withFallback(commonConfig)
+}
+
+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 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 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 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 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 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 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 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 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 should be >= (1)
+        a2.table.version 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 should be >= (1)
+        a2.table.version 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 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 should be >= (1)
+        b2.table.dictionary.keySet should contain(echoRefB2)
+      }
+    }
+
+  }
+
+}
+
+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
index 27b61c03d2..ae8c9f17ea 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTableSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTableSpec.scala
@@ -10,12 +10,14 @@ class CompressionTableSpec extends AkkaSpec {
 
   "CompressionTable" must {
     "should invert" in {
-      val decomp = CompressionTable(1, Map("0" → 0, "1" → 1, "2" → 2, "3" → 3)).invert
+      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(1, Map("1" → 1, "3" → 3)) // missing 0 is a gap too
+      val compressionTable = CompressionTable(17L, 1, Map("1" → 1, "3" → 3)) // missing 0 is a gap too
 
       val ex = intercept[IllegalArgumentException] {
         compressionTable.invert
@@ -24,7 +26,7 @@ class CompressionTableSpec extends AkkaSpec {
     }
 
     "should not allow having gaps in compression ids (inversion would fail)" in {
-      val compressionTable = CompressionTable(1, Map("0" → 0, "1" → 1, "3" → 3)) // missing 0 is a gap too
+      val compressionTable = CompressionTable(17L, 1, Map("0" → 0, "1" → 1, "3" → 3)) // missing 0 is a gap too
 
       val ex = intercept[IllegalArgumentException] {
         compressionTable.invert
diff --git a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTestKit.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTestKit.scala
deleted file mode 100644
index 7bdd8973e8..0000000000
--- a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionTestKit.scala
+++ /dev/null
@@ -1,17 +0,0 @@
-/*
- * Copyright (C) 2016 Lightbend Inc. 
- */
-
-package akka.remote.artery.compress
-
-/* INTERNAL API */
-private[akka] trait CompressionTestKit {
-  def assertCompression[T](table: CompressionTable[T], id: Int, assertion: T ⇒ Unit): Unit = {
-    table.map.find(_._2 == id)
-      .orElse { throw new AssertionError(s"No key was compressed to the id [$id]! Table was: $table") }
-      .foreach(i ⇒ assertion(i._1))
-  }
-}
-
-/* INTERNAL API */
-private[akka] object CompressionTestKit extends CompressionTestKit
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
index 4cf2c69646..869d8c0ff1 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala
@@ -47,8 +47,7 @@ object HandshakeShouldDropCompressionTableSpec {
 }
 
 class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDropCompressionTableSpec.commonConfig)
-  with ImplicitSender with BeforeAndAfter
-  with CompressionTestKit {
+  with ImplicitSender with BeforeAndAfter {
   import HandshakeShouldDropCompressionTableSpec._
 
   implicit val t = Timeout(3.seconds)
@@ -81,7 +80,7 @@ class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDr
 
       val a0 = aProbe.expectMsgType[ReceivedActorRefCompressionTable](10.seconds)
       info("System [A] received: " + a0)
-      a0.table.map.keySet should contain(testActor)
+      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) }
@@ -90,7 +89,7 @@ class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDr
 
       val a1 = aProbe.expectMsgType[ReceivedActorRefCompressionTable](10.seconds)
       info("System [A] received: " + a1)
-      a1.table.map.keySet should contain(a1Probe.ref)
+      a1.table.dictionary.keySet should contain(a1Probe.ref)
 
       log.warning("SHUTTING DOWN system {}...", systemB)
       shutdown(systemB)
@@ -112,7 +111,7 @@ class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDr
 
       val a2 = aNewProbe.expectMsgType[ReceivedActorRefCompressionTable](10.seconds)
       info("System [A] received: " + a2)
-      a2.table.map.keySet should contain(testActor)
+      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
@@ -121,7 +120,7 @@ class HandshakeShouldDropCompressionTableSpec extends AkkaSpec(HandshakeShouldDr
 
       val a3 = aNewProbe.expectMsgType[ReceivedActorRefCompressionTable](10.seconds)
       info("Received second compression: " + a3)
-      a3.table.map.keySet should contain(aNew2Probe.ref)
+      a3.table.dictionary.keySet should contain(aNew2Probe.ref)
     }
   }
 
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
index ac04610e9b..3c18d06f58 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/compress/OutboundCompressionSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/compress/OutboundCompressionSpec.scala
@@ -20,11 +20,11 @@ class OutboundCompressionSpec extends AkkaSpec {
     }
 
     "compress previously registered actor ref" in {
-      val table = CompressionTable(1, Map(system.deadLetters → 0, alice → 1))
+      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, map = table.map.updated(bob, 2))
+      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
index 34bc13b1de..2d247067b9 100644
--- a/akka-remote/src/test/scala/akka/remote/serialization/ArteryMessageSerializerSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/serialization/ArteryMessageSerializerSpec.scala
@@ -24,14 +24,13 @@ class ArteryMessageSerializerSpec extends AkkaSpec {
       "ActorSystemTerminatingAck" → ActorSystemTerminatingAck(uniqueAddress()),
       "HandshakeReq" → HandshakeReq(uniqueAddress()),
       "HandshakeRsp" → HandshakeRsp(uniqueAddress()),
-      "ActorRefCompressionAdvertisement" → ActorRefCompressionAdvertisement(uniqueAddress(), CompressionTable(123, Map(actorA → 123, actorB → 456, system.deadLetters → 0))),
+      "ActorRefCompressionAdvertisement" → ActorRefCompressionAdvertisement(uniqueAddress(), CompressionTable(17L, 123, Map(actorA → 123, actorB → 456, system.deadLetters → 0))),
       "ActorRefCompressionAdvertisementAck" → ActorRefCompressionAdvertisementAck(uniqueAddress(), 23),
-      "ClassManifestCompressionAdvertisement" → ClassManifestCompressionAdvertisement(uniqueAddress(), CompressionTable(42, Map("a" → 535, "b" → 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())
-    ).foreach {
+      "SystemMessageDelivery.Nack" → SystemMessageDelivery.Nack(98765432109876L, uniqueAddress())).foreach {
         case (scenario, item) ⇒
           s"resolve serializer for $scenario" in {
             val serializer = SerializationExtension(system)

From 8ecd7419acd4b90e893210040c9c484caad76079 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Endre=20S=C3=A1ndor=20Varga?= 
Date: Mon, 19 Sep 2016 12:48:07 +0200
Subject: [PATCH 148/186] #21419: Reenable ClusterDeathWatchSpec

---
 .../scala/akka/cluster/ClusterDeathWatchSpec.scala         | 7 ++-----
 1 file changed, 2 insertions(+), 5 deletions(-)

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 9c7903f9bc..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,11 +59,8 @@ abstract class ClusterDeathWatchSpec
   }
 
   "An actor watching a remote actor in the cluster" must {
-    "work with artery" in {
-      pending
-    }
 
-    /*    "receive Terminated when watched node becomes Down/Removed" in within(20 seconds) {
+    "receive Terminated when watched node becomes Down/Removed" in within(20 seconds) {
       awaitClusterUp(first, second, third, fourth)
       enterBarrier("cluster-up")
 
@@ -266,7 +263,7 @@ abstract class ClusterDeathWatchSpec
         enterBarrier("after-4")
       }
 
-    }*/
+    }
 
   }
 }

From a939e30b49c918b58eb5ed32c056e08178f10476 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Johan=20Andr=C3=A9n?= 
Date: Mon, 19 Sep 2016 13:22:54 +0200
Subject: [PATCH 149/186] Fix artery test file leak #21484

* Include actor system name in artery dir path to ease debugging leaks
* Base class name changed to make actor system autonaming work
* Add shutdown hook directly in transport start
* Wait for completion in shutdown hook (actual leak fix)
---
 .../testkit/FlightRecordingSupport.scala      |  2 +-
 .../remote/AttemptSysMsgRedeliverySpec.scala  |  4 +-
 .../akka/remote/LookupRemoteActorSpec.scala   |  4 +-
 .../akka/remote/NewRemoteActorSpec.scala      |  4 +-
 .../PiercingShouldKeepQuarantineSpec.scala    |  4 +-
 .../akka/remote/RemoteDeliverySpec.scala      |  4 +-
 .../RemoteDeploymentDeathWatchSpec.scala      |  4 +-
 .../akka/remote/RemoteGatePiercingSpec.scala  |  2 +-
 .../remote/RemoteNodeDeathWatchSpec.scala     |  6 +--
 .../RemoteNodeRestartDeathWatchSpec.scala     |  2 +-
 .../remote/RemoteNodeRestartGateSpec.scala    |  2 +-
 .../RemoteNodeShutdownAndComesBackSpec.scala  |  2 +-
 .../remote/RemoteQuarantinePiercingSpec.scala |  4 +-
 .../akka/remote/RemoteReDeploymentSpec.scala  |  4 +-
 .../RemoteRestartedQuarantinedSpec.scala      |  2 +-
 ...Spec.scala => RemotingMultiNodeSpec.scala} |  4 +-
 .../scala/akka/remote/Ticket15109Spec.scala   |  2 +-
 .../akka/remote/artery/LatencySpec.scala      |  6 +--
 .../remote/artery/MaxThroughputSpec.scala     |  6 +--
 .../RemoteRestartedQuarantinedSpec.scala      |  6 +--
 .../artery/SurviveNetworkPartitionSpec.scala  |  6 +--
 .../remote/routing/RemoteRandomSpec.scala     |  6 +--
 .../remote/routing/RemoteRoundRobinSpec.scala | 37 ++++++-------------
 .../routing/RemoteScatterGatherSpec.scala     |  6 +--
 .../testconductor/TestConductorSpec.scala     |  6 +--
 .../remote/testkit/MultiNodeSpecSpec.scala    |  6 +--
 .../akka/remote/artery/ArteryTransport.scala  | 23 +++++++-----
 .../remote/artery/FlightRecorderReader.scala  |  2 +-
 28 files changed, 78 insertions(+), 88 deletions(-)
 rename akka-remote-tests/src/multi-jvm/scala/akka/remote/{MultiNodeRemotingSpec.scala => RemotingMultiNodeSpec.scala} (93%)

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
index c5aa24275b..a19b067f46 100644
--- 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
@@ -46,7 +46,7 @@ trait FlightRecordingSupport { self: MultiNodeSpec ⇒
   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("Flight recorder dump:")
+      println(s"Flight recorder dump from '$flightRecorderFile':")
       FlightRecorderReader.dumpToStdout(flightRecorderFile)
     }
   }
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 fd1d2d079b..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
@@ -24,7 +24,7 @@ class AttemptSysMsgRedeliveryMultiJvmSpec(artery: Boolean) extends MultiNodeConf
   commonConfig(debugConfig(on = false).withFallback(
     ConfigFactory.parseString(s"""
       akka.remote.artery.enabled = $artery
-      """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf))
+      """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf))
 
   testTransport(on = true)
 
@@ -53,7 +53,7 @@ object AttemptSysMsgRedeliverySpec {
 }
 
 abstract class AttemptSysMsgRedeliverySpec(multiNodeConfig: AttemptSysMsgRedeliveryMultiJvmSpec)
-  extends MultiNodeRemotingSpec(multiNodeConfig) {
+  extends RemotingMultiNodeSpec(multiNodeConfig) {
   import multiNodeConfig._
   import AttemptSysMsgRedeliverySpec._
 
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 47fe16b294..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
@@ -18,7 +18,7 @@ class LookupRemoteActorMultiJvmSpec(artery: Boolean) extends MultiNodeConfig {
   commonConfig(debugConfig(on = false).withFallback(
     ConfigFactory.parseString(s"""
       akka.remote.artery.enabled = $artery
-      """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf))
+      """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf))
 
   val master = role("master")
   val slave = role("slave")
@@ -40,7 +40,7 @@ object LookupRemoteActorSpec {
 }
 
 abstract class LookupRemoteActorSpec(multiNodeConfig: LookupRemoteActorMultiJvmSpec)
-  extends MultiNodeRemotingSpec(multiNodeConfig) {
+  extends RemotingMultiNodeSpec(multiNodeConfig) {
   import multiNodeConfig._
   import LookupRemoteActorSpec._
 
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 66025899da..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
@@ -20,7 +20,7 @@ class NewRemoteActorMultiJvmSpec(artery: Boolean) extends MultiNodeConfig {
     ConfigFactory.parseString(s"""
       akka.remote.log-remote-lifecycle-events = off
       akka.remote.artery.enabled = $artery
-      """).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf)))
+      """).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf)))
 
   val master = role("master")
   val slave = role("slave")
@@ -55,7 +55,7 @@ object NewRemoteActorSpec {
 }
 
 abstract class NewRemoteActorSpec(multiNodeConfig: NewRemoteActorMultiJvmSpec)
-  extends MultiNodeRemotingSpec(multiNodeConfig) {
+  extends RemotingMultiNodeSpec(multiNodeConfig) {
   import multiNodeConfig._
   import NewRemoteActorSpec._
 
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 6c3b9df226..12efbba067 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
@@ -15,7 +15,7 @@ class PiercingShouldKeepQuarantineConfig(artery: Boolean) extends MultiNodeConfi
     ConfigFactory.parseString(s"""
       akka.remote.retry-gate-closed-for = 0.5s
       akka.remote.artery.enabled = $artery
-      """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf))
+      """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf))
 
 }
 
@@ -38,7 +38,7 @@ object PiercingShouldKeepQuarantineSpec {
 }
 
 abstract class PiercingShouldKeepQuarantineSpec(multiNodeConfig: PiercingShouldKeepQuarantineConfig)
-  extends MultiNodeRemotingSpec(multiNodeConfig) {
+  extends RemotingMultiNodeSpec(multiNodeConfig) {
   import multiNodeConfig._
   import PiercingShouldKeepQuarantineSpec._
 
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 d54c90e4f8..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
@@ -26,7 +26,7 @@ class RemoteDeliveryConfig(artery: Boolean) extends MultiNodeConfig {
   commonConfig(debugConfig(on = false).withFallback(
     ConfigFactory.parseString(s"""
       akka.remote.artery.enabled = $artery
-      """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf))
+      """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf))
 }
 
 class RemoteDeliveryMultiJvmNode1 extends RemoteDeliverySpec(new RemoteDeliveryConfig(artery = false))
@@ -48,7 +48,7 @@ object RemoteDeliverySpec {
 }
 
 abstract class RemoteDeliverySpec(multiNodeConfig: RemoteDeliveryConfig)
-  extends MultiNodeRemotingSpec(multiNodeConfig) {
+  extends RemotingMultiNodeSpec(multiNodeConfig) {
   import multiNodeConfig._
   import RemoteDeliverySpec._
 
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 a469a5baf1..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
@@ -27,7 +27,7 @@ class RemoteDeploymentDeathWatchMultiJvmSpec(artery: Boolean) extends MultiNodeC
       akka.loglevel = INFO
       akka.remote.log-remote-lifecycle-events = off
       akka.remote.artery.enabled = $artery
-      """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf))
+      """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf))
 
   deployOn(second, """/hello.remote = "@third@" """)
 
@@ -69,7 +69,7 @@ object RemoteDeploymentDeathWatchSpec {
 }
 
 abstract class RemoteDeploymentDeathWatchSpec(multiNodeConfig: RemoteDeploymentDeathWatchMultiJvmSpec)
-  extends MultiNodeRemotingSpec(multiNodeConfig) {
+  extends RemotingMultiNodeSpec(multiNodeConfig) {
   import multiNodeConfig._
   import RemoteDeploymentDeathWatchSpec._
 
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 08f6269945..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
@@ -51,7 +51,7 @@ class RemoteGatePiercingSpecMultiJvmNode1 extends RemoteGatePiercingSpec
 class RemoteGatePiercingSpecMultiJvmNode2 extends RemoteGatePiercingSpec
 
 abstract class RemoteGatePiercingSpec
-  extends MultiNodeRemotingSpec(RemoteGatePiercingSpec) {
+  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 4fe5da25ce..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
@@ -31,7 +31,7 @@ class RemoteNodeDeathWatchConfig(artery: Boolean) extends MultiNodeConfig {
       ## 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(MultiNodeRemotingSpec.arteryFlightRecordingConf))
+      """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf))
 
 }
 
@@ -92,7 +92,7 @@ object RemoteNodeDeathWatchSpec {
 }
 
 abstract class RemoteNodeDeathWatchSpec(multiNodeConfig: RemoteNodeDeathWatchConfig)
-  extends MultiNodeRemotingSpec(multiNodeConfig) {
+  extends RemotingMultiNodeSpec(multiNodeConfig) {
   import multiNodeConfig._
   import RemoteNodeDeathWatchSpec._
   import RemoteWatcher._
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 63dc11987a..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
@@ -63,7 +63,7 @@ object RemoteNodeRestartDeathWatchSpec {
 }
 
 abstract class RemoteNodeRestartDeathWatchSpec(multiNodeConfig: RemoteNodeRestartDeathWatchConfig)
-  extends MultiNodeRemotingSpec(multiNodeConfig) {
+  extends RemotingMultiNodeSpec(multiNodeConfig) {
   import multiNodeConfig._
   import RemoteNodeRestartDeathWatchSpec._
 
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 ee7015602d..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,7 +46,7 @@ class RemoteNodeRestartGateSpecMultiJvmNode1 extends RemoteNodeRestartGateSpec
 class RemoteNodeRestartGateSpecMultiJvmNode2 extends RemoteNodeRestartGateSpec
 
 abstract class RemoteNodeRestartGateSpec
-  extends MultiNodeRemotingSpec(RemoteNodeRestartGateSpec) {
+  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 b396c6e402..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
@@ -47,7 +47,7 @@ class RemoteNodeShutdownAndComesBackMultiJvmNode1 extends RemoteNodeShutdownAndC
 class RemoteNodeShutdownAndComesBackMultiJvmNode2 extends RemoteNodeShutdownAndComesBackSpec
 
 abstract class RemoteNodeShutdownAndComesBackSpec
-  extends MultiNodeRemotingSpec(RemoteNodeShutdownAndComesBackSpec) {
+  extends RemotingMultiNodeSpec(RemoteNodeShutdownAndComesBackSpec) {
 
   import RemoteNodeShutdownAndComesBackSpec._
 
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 7790ea489e..401a3bf104 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
@@ -27,7 +27,7 @@ class RemoteQuarantinePiercingConfig(artery: Boolean) extends MultiNodeConfig {
       akka.loglevel = INFO
       akka.remote.log-remote-lifecycle-events = INFO
       akka.remote.artery.enabled = $artery
-      """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf))
+      """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf))
 
 }
 
@@ -51,7 +51,7 @@ object RemoteQuarantinePiercingSpec {
 }
 
 abstract class RemoteQuarantinePiercingSpec(multiNodeConfig: RemoteQuarantinePiercingConfig)
-  extends MultiNodeRemotingSpec(multiNodeConfig) {
+  extends RemotingMultiNodeSpec(multiNodeConfig) {
   import multiNodeConfig._
   import RemoteQuarantinePiercingSpec._
 
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 5e7253286f..27bfe09e22 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
@@ -35,7 +35,7 @@ class RemoteReDeploymentConfig(artery: Boolean) extends MultiNodeConfig {
          acceptable-heartbeat-pause=2.5s
        }
        akka.remote.artery.enabled = $artery
-       """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf))
+       """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf))
 
   testTransport(on = true)
 
@@ -106,7 +106,7 @@ object RemoteReDeploymentMultiJvmSpec {
 }
 
 abstract class RemoteReDeploymentMultiJvmSpec(multiNodeConfig: RemoteReDeploymentConfig)
-  extends MultiNodeRemotingSpec(multiNodeConfig) {
+  extends RemotingMultiNodeSpec(multiNodeConfig) {
 
   def sleepAfterKill: FiniteDuration
   def expectQuarantine: Boolean
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 cd53abd011..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,7 +56,7 @@ class RemoteRestartedQuarantinedSpecMultiJvmNode1 extends RemoteRestartedQuarant
 class RemoteRestartedQuarantinedSpecMultiJvmNode2 extends RemoteRestartedQuarantinedSpec
 
 abstract class RemoteRestartedQuarantinedSpec
-  extends MultiNodeRemotingSpec(RemoteRestartedQuarantinedSpec) {
+  extends RemotingMultiNodeSpec(RemoteRestartedQuarantinedSpec) {
 
   import RemoteRestartedQuarantinedSpec._
 
diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/MultiNodeRemotingSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemotingMultiNodeSpec.scala
similarity index 93%
rename from akka-remote-tests/src/multi-jvm/scala/akka/remote/MultiNodeRemotingSpec.scala
rename to akka-remote-tests/src/multi-jvm/scala/akka/remote/RemotingMultiNodeSpec.scala
index 65e92bc68a..fddc369110 100644
--- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/MultiNodeRemotingSpec.scala
+++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/RemotingMultiNodeSpec.scala
@@ -10,7 +10,7 @@ import akka.testkit.{ DefaultTimeout, ImplicitSender }
 import com.typesafe.config.ConfigFactory
 import org.scalatest.{ Outcome, Suite }
 
-object MultiNodeRemotingSpec {
+object RemotingMultiNodeSpec {
 
   def arteryFlightRecordingConf =
     ConfigFactory.parseString(
@@ -23,7 +23,7 @@ object MultiNodeRemotingSpec {
 
 }
 
-abstract class MultiNodeRemotingSpec(config: MultiNodeConfig) extends MultiNodeSpec(config)
+abstract class RemotingMultiNodeSpec(config: MultiNodeConfig) extends MultiNodeSpec(config)
   with Suite
   with STMultiNodeSpec
   with FlightRecordingSupport
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 b43610fea1..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,7 +47,7 @@ object Ticket15109Spec extends MultiNodeConfig {
 class Ticket15109SpecMultiJvmNode1 extends Ticket15109Spec
 class Ticket15109SpecMultiJvmNode2 extends Ticket15109Spec
 
-abstract class Ticket15109Spec extends MultiNodeRemotingSpec(Ticket15109Spec) {
+abstract class Ticket15109Spec extends RemotingMultiNodeSpec(Ticket15109Spec) {
 
   import Ticket15109Spec._
 
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
index d7a5763be7..884a1fbbed 100644
--- 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
@@ -9,7 +9,7 @@ import java.util.concurrent.locks.LockSupport
 
 import scala.concurrent.duration._
 import akka.actor._
-import akka.remote.MultiNodeRemotingSpec
+import akka.remote.RemotingMultiNodeSpec
 import akka.remote.testconductor.RoleName
 import akka.remote.testkit.MultiNodeConfig
 import akka.remote.testkit.MultiNodeSpec
@@ -53,7 +53,7 @@ object LatencySpec extends MultiNodeConfig {
            }
          }
        }
-       """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf))
+       """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf))
 
   final case object Reset
 
@@ -161,7 +161,7 @@ class LatencySpecMultiJvmNode1 extends LatencySpec
 class LatencySpecMultiJvmNode2 extends LatencySpec
 
 abstract class LatencySpec
-  extends MultiNodeRemotingSpec(LatencySpec) {
+  extends RemotingMultiNodeSpec(LatencySpec) {
 
   import LatencySpec._
 
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
index 9966befdaa..ca19ed0339 100644
--- 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
@@ -9,7 +9,7 @@ import java.util.concurrent.TimeUnit.NANOSECONDS
 
 import scala.concurrent.duration._
 import akka.actor._
-import akka.remote.{ MultiNodeRemotingSpec, RARP, RemoteActorRefProvider }
+import akka.remote.{ RemotingMultiNodeSpec, RARP, RemoteActorRefProvider }
 import akka.remote.testconductor.RoleName
 import akka.remote.testkit.MultiNodeConfig
 import akka.remote.testkit.MultiNodeSpec
@@ -67,7 +67,7 @@ object MaxThroughputSpec extends MultiNodeConfig {
            }
          }
        }
-       """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf))
+       """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf))
 
   case object Run
   sealed trait Echo extends DeadLetterSuppression with JavaSerializable
@@ -277,7 +277,7 @@ object MaxThroughputSpec extends MultiNodeConfig {
 class MaxThroughputSpecMultiJvmNode1 extends MaxThroughputSpec
 class MaxThroughputSpecMultiJvmNode2 extends MaxThroughputSpec
 
-abstract class MaxThroughputSpec extends MultiNodeRemotingSpec(MaxThroughputSpec) with PerfFlamesSupport {
+abstract class MaxThroughputSpec extends RemotingMultiNodeSpec(MaxThroughputSpec) with PerfFlamesSupport {
 
   import MaxThroughputSpec._
 
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
index 333de1a87e..feb28d84a4 100644
--- 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
@@ -20,7 +20,7 @@ import akka.remote.testconductor.RoleName
 import akka.actor.Identify
 
 import scala.concurrent.Await
-import akka.remote.{ AddressUidExtension, MultiNodeRemotingSpec, RARP, ThisActorSystemQuarantinedEvent }
+import akka.remote.{ AddressUidExtension, RemotingMultiNodeSpec, RARP, ThisActorSystemQuarantinedEvent }
 
 object RemoteRestartedQuarantinedSpec extends MultiNodeConfig {
   val first = role("first")
@@ -31,7 +31,7 @@ object RemoteRestartedQuarantinedSpec extends MultiNodeConfig {
       akka.loglevel = WARNING
       akka.remote.log-remote-lifecycle-events = WARNING
       akka.remote.artery.enabled = on
-      """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf))
+      """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf))
 
   class Subject extends Actor {
     def receive = {
@@ -45,7 +45,7 @@ object RemoteRestartedQuarantinedSpec extends MultiNodeConfig {
 class RemoteRestartedQuarantinedSpecMultiJvmNode1 extends RemoteRestartedQuarantinedSpec
 class RemoteRestartedQuarantinedSpecMultiJvmNode2 extends RemoteRestartedQuarantinedSpec
 
-abstract class RemoteRestartedQuarantinedSpec extends MultiNodeRemotingSpec(RemoteRestartedQuarantinedSpec) {
+abstract class RemoteRestartedQuarantinedSpec extends RemotingMultiNodeSpec(RemoteRestartedQuarantinedSpec) {
 
   import RemoteRestartedQuarantinedSpec._
 
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
index 3356111b5d..72d02c5952 100644
--- 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
@@ -7,7 +7,7 @@ import scala.concurrent.duration._
 import akka.actor._
 import akka.actor.ActorIdentity
 import akka.actor.Identify
-import akka.remote.{ MultiNodeRemotingSpec, QuarantinedEvent, RARP }
+import akka.remote.{ RemotingMultiNodeSpec, QuarantinedEvent, RARP }
 import akka.remote.testconductor.RoleName
 import akka.remote.testkit.MultiNodeConfig
 import akka.remote.testkit.MultiNodeSpec
@@ -25,7 +25,7 @@ object SurviveNetworkPartitionSpec extends MultiNodeConfig {
       akka.loglevel = INFO
       akka.remote.artery.enabled = on
       akka.remote.artery.advanced.give-up-system-message-after = 4s
-      """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf))
+      """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf))
 
   testTransport(on = true)
 }
@@ -33,7 +33,7 @@ object SurviveNetworkPartitionSpec extends MultiNodeConfig {
 class SurviveNetworkPartitionSpecMultiJvmNode1 extends SurviveNetworkPartitionSpec
 class SurviveNetworkPartitionSpecMultiJvmNode2 extends SurviveNetworkPartitionSpec
 
-abstract class SurviveNetworkPartitionSpec extends MultiNodeRemotingSpec(SurviveNetworkPartitionSpec) {
+abstract class SurviveNetworkPartitionSpec extends RemotingMultiNodeSpec(SurviveNetworkPartitionSpec) {
 
   import SurviveNetworkPartitionSpec._
 
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 27cce605b0..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,7 +9,7 @@ import akka.actor.ActorRef
 import akka.actor.Address
 import akka.actor.PoisonPill
 import akka.actor.Props
-import akka.remote.MultiNodeRemotingSpec
+import akka.remote.RemotingMultiNodeSpec
 import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
 import akka.routing.Broadcast
 import akka.routing.RandomPool
@@ -27,7 +27,7 @@ class RemoteRandomConfig(artery: Boolean) extends MultiNodeConfig {
   commonConfig(debugConfig(on = false).withFallback(
     ConfigFactory.parseString(s"""
       akka.remote.artery.enabled = $artery
-      """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf))
+      """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf))
 
   deployOnAll("""
       /service-hello {
@@ -56,7 +56,7 @@ object RemoteRandomSpec {
   }
 }
 
-class RemoteRandomSpec(multiNodeConfig: RemoteRandomConfig) extends MultiNodeRemotingSpec(multiNodeConfig)
+class RemoteRandomSpec(multiNodeConfig: RemoteRandomConfig) extends RemotingMultiNodeSpec(multiNodeConfig)
   with DefaultTimeout {
   import multiNodeConfig._
   import RemoteRandomSpec._
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 8c21c7c644..e87cb3004d 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,32 +3,19 @@
  */
 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.MultiNodeRemotingSpec
-import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
-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 scala.concurrent.duration._
 import com.typesafe.config.ConfigFactory
 
+import scala.collection.immutable
+import scala.concurrent.Await
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
 class RemoteRoundRobinConfig(artery: Boolean) extends MultiNodeConfig {
 
   val first = role("first")
@@ -39,7 +26,7 @@ class RemoteRoundRobinConfig(artery: Boolean) extends MultiNodeConfig {
   commonConfig(debugConfig(on = false).withFallback(
     ConfigFactory.parseString(s"""
       akka.remote.artery.enabled = $artery
-      """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf))
+      """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf))
 
   deployOnAll("""
       /service-hello {
@@ -86,10 +73,10 @@ object RemoteRoundRobinSpec {
   }
 }
 
-class RemoteRoundRobinSpec(multiNodeConfig: RemoteRoundRobinConfig) extends MultiNodeRemotingSpec(multiNodeConfig)
+class RemoteRoundRobinSpec(multiNodeConfig: RemoteRoundRobinConfig) extends RemotingMultiNodeSpec(multiNodeConfig)
   with DefaultTimeout {
-  import multiNodeConfig._
   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 d671d5779b..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,7 +9,7 @@ import akka.actor.ActorRef
 import akka.actor.Address
 import akka.actor.PoisonPill
 import akka.actor.Props
-import akka.remote.MultiNodeRemotingSpec
+import akka.remote.RemotingMultiNodeSpec
 import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
 import akka.routing.Broadcast
 import akka.routing.ScatterGatherFirstCompletedPool
@@ -28,7 +28,7 @@ class RemoteScatterGatherConfig(artery: Boolean) extends MultiNodeConfig {
   commonConfig(debugConfig(on = false).withFallback(
     ConfigFactory.parseString(s"""
       akka.remote.artery.enabled = $artery
-      """)).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf))
+      """)).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf))
 
   deployOnAll("""
       /service-hello {
@@ -57,7 +57,7 @@ object RemoteScatterGatherSpec {
   }
 }
 
-class RemoteScatterGatherSpec(multiNodeConfig: RemoteScatterGatherConfig) extends MultiNodeRemotingSpec(multiNodeConfig)
+class RemoteScatterGatherSpec(multiNodeConfig: RemoteScatterGatherConfig) extends RemotingMultiNodeSpec(multiNodeConfig)
   with DefaultTimeout {
   import multiNodeConfig._
   import RemoteScatterGatherSpec._
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 c5cb873d65..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
@@ -15,12 +15,12 @@ import akka.testkit.LongRunningTest
 import java.net.InetSocketAddress
 import java.net.InetAddress
 
-import akka.remote.MultiNodeRemotingSpec
+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).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf))
+  commonConfig(debugConfig(on = false).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf))
 
   val master = role("master")
   val slave = role("slave")
@@ -31,7 +31,7 @@ object TestConductorMultiJvmSpec extends MultiNodeConfig {
 class TestConductorMultiJvmNode1 extends TestConductorSpec
 class TestConductorMultiJvmNode2 extends TestConductorSpec
 
-class TestConductorSpec extends MultiNodeRemotingSpec(TestConductorMultiJvmSpec) {
+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 99fc3b6add..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,11 +3,11 @@
  */
 package akka.remote.testkit
 
-import akka.remote.MultiNodeRemotingSpec
+import akka.remote.RemotingMultiNodeSpec
 import akka.testkit.LongRunningTest
 
 object MultiNodeSpecMultiJvmSpec extends MultiNodeConfig {
-  commonConfig(debugConfig(on = false).withFallback(MultiNodeRemotingSpec.arteryFlightRecordingConf))
+  commonConfig(debugConfig(on = false).withFallback(RemotingMultiNodeSpec.arteryFlightRecordingConf))
 
   val node1 = role("node1")
   val node2 = role("node2")
@@ -20,7 +20,7 @@ class MultiNodeSpecSpecMultiJvmNode2 extends MultiNodeSpecSpec
 class MultiNodeSpecSpecMultiJvmNode3 extends MultiNodeSpecSpec
 class MultiNodeSpecSpecMultiJvmNode4 extends MultiNodeSpecSpec
 
-class MultiNodeSpecSpec extends MultiNodeRemotingSpec(MultiNodeSpecMultiJvmSpec) {
+class MultiNodeSpecSpec extends RemotingMultiNodeSpec(MultiNodeSpecMultiJvmSpec) {
 
   import MultiNodeSpecMultiJvmSpec._
 
diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala
index 799b114829..027ea6098c 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala
@@ -7,12 +7,13 @@ 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.{ Await, ExecutionContext, Future, Promise }
 import scala.concurrent.duration._
 import scala.util.Failure
 import scala.util.Success
@@ -47,6 +48,7 @@ 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.ActorAttributes.Dispatcher
 import akka.stream.ActorMaterializer
 import akka.stream.KillSwitches
 import akka.stream.Materializer
@@ -58,11 +60,7 @@ import akka.stream.scaladsl.Sink
 import akka.stream.scaladsl.Source
 import akka.util.OptionVal
 import akka.util.WildcardIndex
-import io.aeron.Aeron
-import io.aeron.AvailableImageHandler
-import io.aeron.CncFileDescriptor
-import io.aeron.Image
-import io.aeron.UnavailableImageHandler
+import io.aeron._
 import io.aeron.driver.MediaDriver
 import io.aeron.driver.ThreadingMode
 import io.aeron.exceptions.ConductorServiceTimeoutException
@@ -381,6 +379,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R
   override def settings = provider.remoteSettings.Artery
 
   override def start(): Unit = {
+    Runtime.getRuntime.addShutdownHook(shutdownHook)
     startMediaDriver()
     startAeron()
     topLevelFREvents.loFreq(Transport_AeronStarted, NoMetaData)
@@ -428,8 +427,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R
   private lazy val shutdownHook = new Thread {
     override def run(): Unit = {
       if (!_shutdown) {
-        internalShutdown()
-
+        Await.result(internalShutdown(), 20.seconds)
       }
     }
   }
@@ -437,8 +435,14 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R
   private def startMediaDriver(): Unit = {
     if (settings.Advanced.EmbeddedMediaDriver) {
       val driverContext = new MediaDriver.Context
-      if (settings.Advanced.AeronDirectoryName.nonEmpty)
+      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.ImageLivenessTimeoutNs.toNanos)
       driverContext.driverTimeoutMs(settings.Advanced.DriverTimeout.toMillis)
@@ -468,7 +472,6 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R
       val driver = MediaDriver.launchEmbedded(driverContext)
       log.info("Started embedded media driver in directory [{}]", driver.aeronDirectoryName)
       topLevelFREvents.loFreq(Transport_MediaDriverStarted, driver.aeronDirectoryName().getBytes("US-ASCII"))
-      Runtime.getRuntime.addShutdownHook(shutdownHook)
       if (!mediaDriver.compareAndSet(None, Some(driver))) {
         throw new IllegalStateException("media driver started more than once")
       }
diff --git a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderReader.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderReader.scala
index 762accc4c7..43173bc8c6 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderReader.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderReader.scala
@@ -199,7 +199,7 @@ private[akka] final class FlightRecorderReader(fileChannel: FileChannel) {
   case class RichEntry(timeStamp: Instant, dirty: Boolean, code: Long, metadata: Array[Byte]) {
     override def toString: String = {
       val textualCode = FlightRecorderEvents.eventDictionary.getOrElse(code, "").take(34)
-      val metadataString = new String(metadata, "UTF-8")
+      val metadataString = new String(metadata, "US-ASCII")
       f"[$timeStamp] ${if (dirty) "#" else ""} $code%3s $textualCode%-34s | $metadataString"
     }
   }

From 4fca0447dd8bd0395052825a19fef543d321e60f Mon Sep 17 00:00:00 2001
From: Patrik Nordwall 
Date: Mon, 19 Sep 2016 14:51:36 +0200
Subject: [PATCH 150/186] improve shutdown of TaskRunner, #21430

* important that no tasks are run after shutdown
* could be a reason for segfault
---
 .../scala/akka/remote/artery/TaskRunner.scala | 22 ++++++++++++++-----
 1 file changed, 17 insertions(+), 5 deletions(-)

diff --git a/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala b/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala
index e3f093e0f6..1f6ca158d4 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/TaskRunner.scala
@@ -63,6 +63,14 @@ private[akka] object TaskRunner {
       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`.
@@ -138,12 +146,14 @@ private[akka] class TaskRunner(system: ExtendedActorSystem, val idleCpuLevel: In
       running = true
       while (running) {
         processCommand(cmdQueue.poll())
-        executeTasks()
-        if (reset) {
-          reset = false
-          idleStrategy.reset()
+        if (running) {
+          executeTasks()
+          if (reset) {
+            reset = false
+            idleStrategy.reset()
+          }
+          idleStrategy.idle()
         }
-        idleStrategy.idle()
       }
     } catch {
       case NonFatal(e) ⇒
@@ -178,6 +188,8 @@ private[akka] class TaskRunner(system: ExtendedActorSystem, val idleCpuLevel: In
       case Remove(task) ⇒ tasks.remove(task)
       case Shutdown ⇒
         running = false
+        tasks.removeAll() // gc friendly
+        while (cmdQueue.poll() != null) () // gc friendly
         shutdown.trySuccess(Done)
     }
   }

From 82598938f1de8136acabf761f796dc596354fdcc Mon Sep 17 00:00:00 2001
From: Patrik Nordwall 
Date: Tue, 20 Sep 2016 11:37:31 +0200
Subject: [PATCH 151/186] close inbound compressions properly

* after quarantine we should never start new inbound compressions for
  that uid
* I noticed that the hit counting was creating new instances also for
  quarantined, and thereby scheduling advertisments
---
 .../artery/compress/InboundCompressions.scala | 89 +++++++++++++------
 1 file changed, 63 insertions(+), 26 deletions(-)

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
index c6deac61cf..75dd11bfe9 100644
--- a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala
+++ b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala
@@ -60,66 +60,97 @@ private[remote] final class InboundCompressionsImpl(
 
   private val stopped = new AtomicBoolean
 
-  private[this] val _actorRefsIns = new Long2ObjectHashMap[InboundActorRefCompression]()
-  private val createInboundActorRefsForOrigin = new LongFunction[InboundActorRefCompression] {
-    override def apply(originUid: Long): InboundActorRefCompression = {
+  // 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)
-      new InboundActorRefCompression(system, settings, originUid, inboundContext, actorRefHitters, stopped)
+      Some(new InboundActorRefCompression(system, settings, originUid, inboundContext, actorRefHitters, stopped))
     }
   }
-  private def actorRefsIn(originUid: Long): InboundActorRefCompression =
+  private def actorRefsIn(originUid: Long): Option[InboundActorRefCompression] =
     _actorRefsIns.computeIfAbsent(originUid, createInboundActorRefsForOrigin)
 
-  private[this] val _classManifestsIns = new Long2ObjectHashMap[InboundManifestCompression]()
-  private val createInboundManifestsForOrigin = new LongFunction[InboundManifestCompression] {
-    override def apply(originUid: Long): InboundManifestCompression = {
+  // 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)
-      new InboundManifestCompression(system, settings, originUid, inboundContext, manifestHitters, stopped)
+      Some(new InboundManifestCompression(system, settings, originUid, inboundContext, manifestHitters, stopped))
     }
   }
-  private def classManifestsIn(originUid: Long): InboundManifestCompression =
+  private def classManifestsIn(originUid: Long): Option[InboundManifestCompression] =
     _classManifestsIns.computeIfAbsent(originUid, createInboundManifestsForOrigin)
 
   // actor ref compression ---
 
   override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] =
-    actorRefsIn(originUid).decompress(tableVersion, idx)
+    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).increment(address, ref, n)
+    actorRefsIn(originUid) match {
+      case Some(a) ⇒ a.increment(address, ref, n)
+      case None    ⇒ // closed
+    }
   }
 
   override def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = {
     _actorRefsIns.get(originUid) match {
-      case null ⇒ // ignore, it was closed
-      case a    ⇒ a.confirmAdvertisement(tableVersion)
+      case null    ⇒ // ignore
+      case Some(a) ⇒ a.confirmAdvertisement(tableVersion)
+      case None    ⇒ // closed
     }
   }
 
   // class manifest compression ---
 
   override def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] =
-    classManifestsIn(originUid).decompress(tableVersion, idx)
+    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).increment(address, manifest, n)
+    classManifestsIn(originUid) match {
+      case Some(a) ⇒ a.increment(address, manifest, n)
+      case None    ⇒ // closed
+    }
   }
   override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = {
     _classManifestsIns.get(originUid) match {
-      case null ⇒ // ignore, it was closed
-      case a    ⇒ a.confirmAdvertisement(tableVersion)
+      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 = {
-    actorRefsIn(originUid).close()
-    classManifestsIn(originUid).close()
-    // FIXME This is not safe, it can be created again (concurrently), at least in theory.
-    //       However, we should make the inbound compressions owned by the Decoder and it doesn't have to be thread-safe
-    _actorRefsIns.remove(originUid)
-    _classManifestsIns.remove(originUid)
+    _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 ---
@@ -127,13 +158,19 @@ private[remote] final class InboundCompressionsImpl(
   /** INTERNAL API: for testing only */
   private[remote] def runNextActorRefAdvertisement() = {
     import scala.collection.JavaConverters._
-    _actorRefsIns.values().asScala.foreach { inbound ⇒ inbound.runNextTableAdvertisement() }
+    _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 { inbound ⇒ inbound.runNextTableAdvertisement() }
+    _classManifestsIns.values().asScala.foreach {
+      case Some(inbound) ⇒ inbound.runNextTableAdvertisement()
+      case None          ⇒ // closed
+    }
   }
 }
 

From 9f7389448adaeac5ca5b5816aeb9e6ee8544495f Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Endre=20S=C3=A1ndor=20Varga?= 
Date: Tue, 20 Sep 2016 12:38:58 +0200
Subject: [PATCH 152/186] Fix AFR file deletion on Windows

---
 .../multi-jvm/scala/akka/remote/RemoteReDeploymentSpec.scala | 3 +++
 .../scala/akka/remote/artery/TaskRunnerMetrics.scala         | 2 ++
 .../scala/akka/remote/routing/RemoteRoundRobinSpec.scala     | 2 +-
 .../test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala  | 5 +----
 .../test/scala/akka/remote/artery/FlightRecorderSpec.scala   | 4 ++--
 5 files changed, 9 insertions(+), 7 deletions(-)

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 27bfe09e22..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
@@ -176,6 +176,9 @@ abstract class RemoteReDeploymentMultiJvmSpec(multiNodeConfig: RemoteReDeploymen
 
       expectNoMsg(1.second)
 
+      runOn(second) {
+        Await.result(sys.terminate(), 10.seconds)
+      }
     }
 
   }
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
index f939669d81..c26e5b88b8 100644
--- 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
@@ -36,6 +36,8 @@ class TaskRunnerMetrics(system: ActorSystem) {
               }
             }
           })
+
+          reader.close()
           entryOffset = c
 
           if (aeronSourceHistogram.getTotalCount > 0) {
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 e87cb3004d..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,7 +3,7 @@
  */
 package akka.remote.routing
 
-import akka.actor.{Actor, ActorRef, Address, PoisonPill, Props}
+import akka.actor.{ Actor, ActorRef, Address, PoisonPill, Props }
 import akka.pattern.ask
 import akka.remote.RemotingMultiNodeSpec
 import akka.remote.testkit.MultiNodeConfig
diff --git a/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala
index 4e27bcc44d..32afc155a7 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/ArteryMultiNodeSpec.scala
@@ -81,13 +81,10 @@ abstract class ArteryMultiNodeSpec(config: Config) extends AkkaSpec(config.withF
     out
   }
 
-  override protected def beforeTermination(): Unit = {
-    handleFlightRecorderFile()
-  }
-
   override def afterTermination(): Unit = {
     remoteSystems.foreach(sys ⇒ shutdown(sys))
     remoteSystems = Vector.empty
+    handleFlightRecorderFile()
   }
 
   private def handleFlightRecorderFile(): Unit = {
diff --git a/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala
index 4a04660b9b..9f082877d5 100644
--- a/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala
+++ b/akka-remote/src/test/scala/akka/remote/artery/FlightRecorderSpec.scala
@@ -12,7 +12,7 @@ import java.util.Arrays
 import java.util.concurrent.{ CountDownLatch, TimeUnit }
 
 import akka.testkit.AkkaSpec
-import com.google.common.jimfs.Jimfs
+import com.google.common.jimfs.{ Configuration, Jimfs }
 
 class FlightRecorderSpec extends AkkaSpec {
   import FlightRecorderReader._
@@ -380,7 +380,7 @@ class FlightRecorderSpec extends AkkaSpec {
         Files.isWritable(path) should ===(true)
         Files.isReadable(path) should ===(true)
       }
-      val fs = Jimfs.newFileSystem()
+      val fs = Jimfs.newFileSystem(Configuration.unix())
 
       try {
         val tmpPath = FlightRecorder.createFlightRecorderFile("", fs)

From 08670ca1552a36d9fda148847b67d8dacd906e62 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Martynas=20Mickevi=C4=8Dius?= 
Date: Tue, 20 Sep 2016 14:23:50 +0300
Subject: [PATCH 153/186] Add non-local recipient check when handling
 HandshaleReq (#21497)

---
 .../akka/remote/ArteryControlFormats.java     | 790 +++++++++++++++++-
 .../main/protobuf/ArteryControlFormats.proto  |   6 +-
 .../scala/akka/remote/artery/Codecs.scala     |   9 +-
 .../scala/akka/remote/artery/Handshake.scala  |  27 +-
 .../akka/remote/artery/InboundEnvelope.scala  |  30 +-
 .../remote/artery/MessageDispatcher.scala     |   9 +-
 .../remote/artery/SystemMessageDelivery.scala |   2 -
 .../ArteryMessageSerializer.scala             |  15 +-
 .../remote/artery/HandshakeDenySpec.scala     |  48 ++
 .../remote/artery/HandshakeRetrySpec.scala    |   3 +-
 .../artery/InboundControlJunctionSpec.scala   |   7 +-
 .../remote/artery/InboundHandshakeSpec.scala  |  11 +-
 .../remote/artery/OutboundHandshakeSpec.scala |  18 +-
 .../artery/SystemMessageAckerSpec.scala       |   2 +-
 .../artery/SystemMessageDeliverySpec.scala    |   6 +-
 .../akka/remote/artery/TestContext.scala      |   4 +-
 .../ArteryMessageSerializerSpec.scala         |   2 +-
 17 files changed, 892 insertions(+), 97 deletions(-)
 create mode 100644 akka-remote/src/test/scala/akka/remote/artery/HandshakeDenySpec.scala

diff --git a/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java b/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java
index eb176e91ba..22deb5c3cd 100644
--- a/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java
+++ b/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java
@@ -764,7 +764,6 @@ public final class ArteryControlFormats {
    * address field. A message that needs to changed later can be cloned from this one and then adapted.
    * ActorSystemTerminating
    * ActorSystemTerminating.Ack
-   * OutboundHandshake.HandshakeReq
    * OutboundHandshake.HandshakeRsp
    * 
*/ @@ -1016,7 +1015,6 @@ public final class ArteryControlFormats { * address field. A message that needs to changed later can be cloned from this one and then adapted. * ActorSystemTerminating * ActorSystemTerminating.Ack - * OutboundHandshake.HandshakeReq * OutboundHandshake.HandshakeRsp *
*/ @@ -1280,6 +1278,737 @@ public final class ArteryControlFormats { // @@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 { @@ -6102,6 +6831,11 @@ public final class ArteryControlFormats { 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 @@ -6144,22 +6878,24 @@ public final class ArteryControlFormats { "\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\"\204\001\n" + - "\035CompressionTableAdvertisement\022\034\n\004from\030\001" + - " \002(\0132\016.UniqueAddress\022\021\n\toriginUid\030\002 \002(\004\022" + - "\024\n\014tableVersion\030\003 \002(\r\022\014\n\004keys\030\004 \003(\t\022\016\n\006v" + - "alues\030\005 \003(\r\"Q\n CompressionTableAdvertise" + - "mentAck\022\034\n\004from\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\017messageManifest\030\003 \001(\014\022\r\n\005seqNo\030\004 \002(\004\022" + - "\"\n\nackReplyTo\030\005 \002(\0132\016.UniqueAddress\"G\n\030S" + - "ystemMessageDeliveryAck\022\r\n\005seqNo\030\001 \002(\004\022\034" + - "\n\004from\030\002 \002(\0132\016.UniqueAddress\"K\n\007Address\022" + - "\020\n\010protocol\030\001 \002(\t\022\016\n\006system\030\002 \002(\t\022\020\n\010hos" + - "tname\030\003 \002(\t\022\014\n\004port\030\004 \002(\r\"7\n\rUniqueAddre" + - "ss\022\031\n\007address\030\001 \002(\0132\010.Address\022\013\n\003uid\030\002 \002" + - "(\004B\017\n\013akka.remoteH\001" + "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(\004B\017\n\013akka.r", + "emoteH\001" }; akka.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new akka.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -6178,38 +6914,44 @@ public final class ArteryControlFormats { akka.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_MessageWithAddress_descriptor, new java.lang.String[] { "Address", }); - internal_static_CompressionTableAdvertisement_descriptor = + 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(3); + 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(4); + 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(5); + 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(6); + 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(7); + getDescriptor().getMessageTypes().get(8); internal_static_UniqueAddress_fieldAccessorTable = new akka.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_UniqueAddress_descriptor, diff --git a/akka-remote/src/main/protobuf/ArteryControlFormats.proto b/akka-remote/src/main/protobuf/ArteryControlFormats.proto index aafff98016..d71d66fe63 100644 --- a/akka-remote/src/main/protobuf/ArteryControlFormats.proto +++ b/akka-remote/src/main/protobuf/ArteryControlFormats.proto @@ -14,12 +14,16 @@ message Quarantined { // address field. A message that needs to changed later can be cloned from this one and then adapted. // ActorSystemTerminating // ActorSystemTerminating.Ack -// OutboundHandshake.HandshakeReq // OutboundHandshake.HandshakeRsp message MessageWithAddress { required UniqueAddress address = 1; } +message HandshakeReq { + required UniqueAddress from = 1; + required Address to = 2; +} + // CompressionProtocol.ActorRefCompressionAdvertisement // CompressionProtocol.ClassManifestCompressionAdvertisement message CompressionTableAdvertisement { diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 58474c3be3..a470e5ae89 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -11,7 +11,7 @@ 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, PrettyByteString } +import akka.util.{ ByteString, OptionVal } import akka.actor.EmptyLocalActorRef import akka.remote.artery.compress.InboundCompressions import akka.stream.stage.TimerGraphStageLogic @@ -24,8 +24,6 @@ import akka.stream.stage.GraphStageWithMaterializedValue import scala.concurrent.Promise -import scala.annotation.switch - /** * INTERNAL API */ @@ -378,7 +376,6 @@ private[remote] class Decoder( val decoded = inEnvelopePool.acquire().init( recipient, - localAddress, // FIXME: this is used for the "non-local recipient" check in MessageDispatcher. Is this needed anymore? sender, originUid, headerBuilder.serializer, @@ -405,8 +402,9 @@ private[remote] class Decoder( scheduleOnce(RetryResolveRemoteDeployedRecipient( retryResolveRemoteDeployedRecipientAttempts, recipientActorRefPath, decoded), retryResolveRemoteDeployedRecipientInterval) - } else + } else { push(out, decoded) + } } } @@ -537,4 +535,3 @@ private[remote] class Deserializer( setHandlers(in, out, this) } } - diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala index 801547d87f..3448c26e78 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -20,6 +20,7 @@ import akka.stream.stage.TimerGraphStageLogic import akka.util.OptionVal import akka.Done import scala.concurrent.Future +import akka.actor.Address /** * INTERNAL API @@ -32,7 +33,7 @@ private[akka] object OutboundHandshake { */ class HandshakeTimeoutException(msg: String) extends RuntimeException(msg) with NoStackTrace - final case class HandshakeReq(from: UniqueAddress) extends ControlMessage + final case class HandshakeReq(from: UniqueAddress, to: Address) extends ControlMessage final case class HandshakeRsp(from: UniqueAddress) extends Reply private sealed trait HandshakeState @@ -130,7 +131,7 @@ private[akka] class OutboundHandshake( injectHandshakeTickScheduled = true scheduleOnce(InjectHandshakeTick, injectHandshakeInterval) val env: OutboundEnvelope = outboundEnvelopePool.acquire().init( - recipient = OptionVal.None, message = HandshakeReq(outboundContext.localAddress), sender = OptionVal.None) + recipient = OptionVal.None, message = HandshakeReq(outboundContext.localAddress, outboundContext.remoteAddress), sender = OptionVal.None) push(out, env) } @@ -176,7 +177,7 @@ private[akka] class InboundHandshake(inboundContext: InboundContext, inControlSt override def onPush(): Unit = { val env = grab(in) env.message match { - case HandshakeReq(from) ⇒ onHandshakeReq(from) + case HandshakeReq(from, to) ⇒ onHandshakeReq(from, to) case HandshakeRsp(from) ⇒ after(inboundContext.completeHandshake(from)) { pull(in) @@ -191,16 +192,28 @@ private[akka] class InboundHandshake(inboundContext: InboundContext, inControlSt override def onPush(): Unit = { val env = grab(in) env.message match { - case HandshakeReq(from) ⇒ onHandshakeReq(from) + case HandshakeReq(from, to) ⇒ onHandshakeReq(from, to) case _ ⇒ onMessage(env) } } }) - private def onHandshakeReq(from: UniqueAddress): Unit = { - after(inboundContext.completeHandshake(from)) { - inboundContext.sendControl(from.address, HandshakeRsp(inboundContext.localAddress)) + 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) } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala b/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala index c017d73107..b9d17362ae 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala @@ -16,14 +16,13 @@ private[remote] object InboundEnvelope { * Only used in tests */ def apply( - recipient: OptionVal[InternalActorRef], - recipientAddress: Address, - message: AnyRef, - sender: OptionVal[ActorRef], - originUid: Long, - association: OptionVal[OutboundContext]): InboundEnvelope = { + recipient: OptionVal[InternalActorRef], + message: AnyRef, + sender: OptionVal[ActorRef], + originUid: Long, + association: OptionVal[OutboundContext]): InboundEnvelope = { val env = new ReusableInboundEnvelope - env.init(recipient, recipientAddress, sender, originUid, -1, "", 0, null, association) + env.init(recipient, sender, originUid, -1, "", 0, null, association) .withMessage(message) } @@ -116,15 +115,14 @@ private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { } def init( - recipient: OptionVal[InternalActorRef], - recipientAddress: Address, - sender: OptionVal[ActorRef], - originUid: Long, - serializer: Int, - classManifest: String, - flags: Byte, - envelopeBuffer: EnvelopeBuffer, - association: OptionVal[OutboundContext]): InboundEnvelope = { + recipient: OptionVal[InternalActorRef], + sender: OptionVal[ActorRef], + originUid: Long, + serializer: Int, + classManifest: String, + flags: Byte, + envelopeBuffer: EnvelopeBuffer, + association: OptionVal[OutboundContext]): InboundEnvelope = { _recipient = recipient _recipientAddress = recipientAddress _sender = sender diff --git a/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala b/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala index 7525473479..b5dc011a6e 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala @@ -71,13 +71,8 @@ private[akka] class MessageDispatcher( case r @ (_: RemoteRef | _: RepointableRef) if !r.isLocal && !UntrustedMode ⇒ if (LogReceive) log.debug("received remote-destined message {}", msgLog) - if (provider.transport.addresses(recipientAddress)) - // if it was originally addressed to us but is in fact remote from our point of view (i.e. remote-deployed) - r.!(message)(sender) - else - log.error( - "dropping message [{}] for non-local recipient [{}] arriving at [{}] inbound addresses are [{}]", - message.getClass, r, recipientAddress, provider.transport.addresses.mkString(", ")) + // 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 [{}] arriving at [{}] inbound addresses are [{}]", diff --git a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala index 3e093fa5f8..1a6eed90d7 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -24,8 +24,6 @@ import akka.stream.stage.OutHandler import akka.stream.stage.TimerGraphStageLogic import akka.remote.artery.OutboundHandshake.HandshakeReq import akka.actor.ActorRef -import akka.remote.PriorityMessage -import akka.actor.ActorSelectionMessage import akka.dispatch.sysmsg.SystemMessage import scala.util.control.NoStackTrace diff --git a/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala index 30499f86a4..ee2694a3ac 100644 --- a/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala @@ -56,7 +56,7 @@ private[akka] final class ArteryMessageSerializer(val system: ExtendedActorSyste override def toBinary(o: AnyRef): Array[Byte] = (o match { // most frequent ones first case env: SystemMessageDelivery.SystemMessageEnvelope ⇒ serializeSystemMessageEnvelope(env) case SystemMessageDelivery.Ack(seqNo, from) ⇒ serializeSystemMessageDeliveryAck(seqNo, from) - case HandshakeReq(from) ⇒ serializeWithAddress(from) + case HandshakeReq(from, to) ⇒ serializeHandshakeReq(from, to) case HandshakeRsp(from) ⇒ serializeWithAddress(from) case SystemMessageDelivery.Nack(seqNo, from) ⇒ serializeSystemMessageDeliveryAck(seqNo, from) case q: Quarantined ⇒ serializeQuarantined(q) @@ -71,7 +71,7 @@ private[akka] final class ArteryMessageSerializer(val system: ExtendedActorSyste 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 ⇒ deserializeWithFromAddress(bytes, HandshakeReq) + case HandshakeReqManifest ⇒ deserializeHandshakeReq(bytes, HandshakeReq) case HandshakeRspManifest ⇒ deserializeWithFromAddress(bytes, HandshakeRsp) case SystemMessageDeliveryNackManifest ⇒ deserializeSystemMessageDeliveryAck(bytes, SystemMessageDelivery.Nack) case QuarantinedManifest ⇒ deserializeQuarantined(ArteryControlFormats.Quarantined.parseFrom(bytes)) @@ -192,6 +192,17 @@ private[akka] final class ArteryMessageSerializer(val system: ExtendedActorSyste 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)) 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..864c0bbe49 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeDenySpec.scala @@ -0,0 +1,48 @@ +/** + * 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 + } + """) + +} + +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/HandshakeRetrySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala index 3e8feadf59..2ce40262e0 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala @@ -22,8 +22,7 @@ object HandshakeRetrySpec { remote.artery.enabled = on remote.artery.canonical.hostname = localhost remote.artery.canonical.port = 0 - remote.handshake-timeout = 10s - + remote.artery.advanced.handshake-timeout = 10s } """) diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala index 44e3a26692..a6030bdffc 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundControlJunctionSpec.scala @@ -3,12 +3,9 @@ */ package akka.remote.artery -import scala.concurrent.duration._ import akka.actor.Address -import akka.actor.InternalActorRef import akka.remote.UniqueAddress import akka.remote.artery.InboundControlJunction.ControlMessageObserver -import akka.remote.artery.SystemMessageDelivery._ import akka.stream.ActorMaterializer import akka.stream.ActorMaterializerSettings import akka.stream.scaladsl.Keep @@ -49,14 +46,14 @@ class InboundControlJunctionSpec val recipient = OptionVal.None // not used val ((upstream, controlSubject), downstream) = TestSource.probe[AnyRef] - .map(msg ⇒ InboundEnvelope(recipient, addressB.address, msg, OptionVal.None, addressA.uid, OptionVal.None)) + .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) { + override def notify(env: InboundEnvelope) = { observerProbe.ref ! env.message } }) diff --git a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala index befdc927e0..96f1b7d648 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/InboundHandshakeSpec.scala @@ -6,11 +6,9 @@ package akka.remote.artery import scala.concurrent.Await import scala.concurrent.duration._ import akka.actor.Address -import akka.actor.InternalActorRef import akka.remote.UniqueAddress import akka.remote.artery.OutboundHandshake.HandshakeReq import akka.remote.artery.OutboundHandshake.HandshakeRsp -import akka.remote.artery.SystemMessageDelivery._ import akka.stream.ActorMaterializer import akka.stream.ActorMaterializerSettings import akka.stream.scaladsl.Keep @@ -30,7 +28,6 @@ object InboundHandshakeSpec { } class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { - import InboundHandshakeSpec._ val matSettings = ActorMaterializerSettings(system).withFuzzing(true) implicit val mat = ActorMaterializer(matSettings)(system) @@ -41,7 +38,7 @@ class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { 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, addressB.address, msg, OptionVal.None, addressA.uid, + .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 } @@ -57,7 +54,7 @@ class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { val (upstream, downstream) = setupStream(inboundContext) downstream.request(10) - upstream.sendNext(HandshakeReq(addressA)) + upstream.sendNext(HandshakeReq(addressA, addressB.address)) upstream.sendNext("msg1") replyProbe.expectMsg(HandshakeRsp(addressB)) downstream.expectNext("msg1") @@ -69,7 +66,7 @@ class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { val (upstream, downstream) = setupStream(inboundContext) downstream.request(10) - upstream.sendNext(HandshakeReq(addressA)) + upstream.sendNext(HandshakeReq(addressA, addressB.address)) upstream.sendNext("msg1") downstream.expectNext("msg1") val uniqueRemoteAddress = Await.result( @@ -89,7 +86,7 @@ class InboundHandshakeSpec extends AkkaSpec with ImplicitSender { downstream.expectNoMsg(200.millis) // messages from unknown are dropped // and accept messages after handshake - upstream.sendNext(HandshakeReq(addressA)) + upstream.sendNext(HandshakeReq(addressA, addressB.address)) upstream.sendNext("msg18") replyProbe.expectMsg(HandshakeRsp(addressB)) downstream.expectNext("msg18") diff --git a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala index 3b044e654c..b5a87f0940 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/OutboundHandshakeSpec.scala @@ -53,7 +53,7 @@ class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { val (upstream, downstream) = setupStream(outboundContext) downstream.request(10) - downstream.expectNext(HandshakeReq(addressA)) + downstream.expectNext(HandshakeReq(addressA, addressB.address)) downstream.cancel() } @@ -65,7 +65,7 @@ class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { upstream.sendNext("msg1") downstream.request(10) - downstream.expectNext(HandshakeReq(addressA)) + downstream.expectNext(HandshakeReq(addressA, addressB.address)) downstream.expectNext("msg1") downstream.cancel() } @@ -76,7 +76,7 @@ class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { val (upstream, downstream) = setupStream(outboundContext, timeout = 200.millis) downstream.request(1) - downstream.expectNext(HandshakeReq(addressA)) + downstream.expectNext(HandshakeReq(addressA, addressB.address)) downstream.expectError().getClass should be(classOf[HandshakeTimeoutException]) } @@ -86,9 +86,9 @@ class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { val (upstream, downstream) = setupStream(outboundContext, retryInterval = 100.millis) downstream.request(10) - downstream.expectNext(HandshakeReq(addressA)) - downstream.expectNext(HandshakeReq(addressA)) - downstream.expectNext(HandshakeReq(addressA)) + downstream.expectNext(HandshakeReq(addressA, addressB.address)) + downstream.expectNext(HandshakeReq(addressA, addressB.address)) + downstream.expectNext(HandshakeReq(addressA, addressB.address)) downstream.cancel() } @@ -98,7 +98,7 @@ class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { val (upstream, downstream) = setupStream(outboundContext) downstream.request(10) - downstream.expectNext(HandshakeReq(addressA)) + downstream.expectNext(HandshakeReq(addressA, addressB.address)) upstream.sendNext("msg1") downstream.expectNoMsg(200.millis) // InboundHandshake stage will complete the handshake when receiving HandshakeRsp @@ -116,7 +116,7 @@ class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { downstream.request(10) upstream.sendNext("msg1") - downstream.expectNext(HandshakeReq(addressA)) + downstream.expectNext(HandshakeReq(addressA, addressB.address)) inboundContext.completeHandshake(addressB) downstream.expectNext("msg1") @@ -124,7 +124,7 @@ class OutboundHandshakeSpec extends AkkaSpec with ImplicitSender { upstream.sendNext("msg2") upstream.sendNext("msg3") upstream.sendNext("msg4") - downstream.expectNext(HandshakeReq(addressA)) + downstream.expectNext(HandshakeReq(addressA, addressB.address)) downstream.expectNext("msg2") downstream.expectNext("msg3") downstream.expectNext("msg4") diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageAckerSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageAckerSpec.scala index 5ba36ba0ef..fd0d44fdf1 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageAckerSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageAckerSpec.scala @@ -34,7 +34,7 @@ class SystemMessageAckerSpec extends AkkaSpec with ImplicitSender { TestSource.probe[AnyRef] .map { case sysMsg @ SystemMessageEnvelope(_, _, ackReplyTo) ⇒ - InboundEnvelope(recipient, addressA.address, sysMsg, OptionVal.None, ackReplyTo.uid, + InboundEnvelope(recipient, sysMsg, OptionVal.None, ackReplyTo.uid, inboundContext.association(ackReplyTo.uid)) } .via(new SystemMessageAcker(inboundContext)) diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index 9264c7d980..cc7ed61608 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -10,12 +10,10 @@ import scala.concurrent.duration._ import akka.NotUsed import akka.actor.ActorIdentity import akka.actor.ActorSystem -import akka.actor.ExtendedActorSystem import akka.actor.Identify -import akka.actor.InternalActorRef import akka.actor.PoisonPill import akka.actor.RootActorPath -import akka.remote.{ AddressUidExtension, RARP, RemoteActorRef, UniqueAddress } +import akka.remote.{ AddressUidExtension, RARP, UniqueAddress } import akka.remote.artery.SystemMessageDelivery._ import akka.stream.ActorMaterializer import akka.stream.ActorMaterializerSettings @@ -79,7 +77,7 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi Flow[OutboundEnvelope] .map(outboundEnvelope ⇒ outboundEnvelope.message match { case sysEnv: SystemMessageEnvelope ⇒ - InboundEnvelope(recipient, addressB.address, sysEnv, OptionVal.None, addressA.uid, + InboundEnvelope(recipient, sysEnv, OptionVal.None, addressA.uid, inboundContext.association(addressA.uid)) }) .async diff --git a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala index db5965589d..422211634b 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/TestContext.scala @@ -12,12 +12,10 @@ import scala.util.Success import akka.Done import akka.actor.ActorRef import akka.actor.Address -import akka.remote.RemoteActorRef import akka.remote.UniqueAddress import akka.remote.artery.InboundControlJunction.ControlMessageObserver import akka.remote.artery.InboundControlJunction.ControlMessageSubject import akka.util.OptionVal -import akka.actor.InternalActorRef import akka.dispatch.ExecutionContexts import com.typesafe.config.ConfigFactory @@ -94,7 +92,7 @@ private[remote] class TestOutboundContext( override def sendControl(message: ControlMessage) = { controlProbe.foreach(_ ! message) - controlSubject.sendControl(InboundEnvelope(OptionVal.None, remoteAddress, message, OptionVal.None, localAddress.uid, + controlSubject.sendControl(InboundEnvelope(OptionVal.None, message, OptionVal.None, localAddress.uid, OptionVal.None)) } diff --git a/akka-remote/src/test/scala/akka/remote/serialization/ArteryMessageSerializerSpec.scala b/akka-remote/src/test/scala/akka/remote/serialization/ArteryMessageSerializerSpec.scala index 2d247067b9..bdf5333632 100644 --- a/akka-remote/src/test/scala/akka/remote/serialization/ArteryMessageSerializerSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/serialization/ArteryMessageSerializerSpec.scala @@ -22,7 +22,7 @@ class ArteryMessageSerializerSpec extends AkkaSpec { "Quarantined" → Quarantined(uniqueAddress(), uniqueAddress()), "ActorSystemTerminating" → ActorSystemTerminating(uniqueAddress()), "ActorSystemTerminatingAck" → ActorSystemTerminatingAck(uniqueAddress()), - "HandshakeReq" → HandshakeReq(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), From 642a145ca1f5a180de343ca4b9f32b2dfef04b1c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Dubiel?= Date: Fri, 22 Jul 2016 21:40:14 +0200 Subject: [PATCH 154/186] =art Heap structure in heavy hitters --- .../compress/HeavyHittersBenchmark.scala | 4 +- .../artery/compress/TopHeavyHitters.scala | 258 +++++++++++++----- .../artery/compress/HeavyHittersSpec.scala | 20 +- 3 files changed, 203 insertions(+), 79 deletions(-) 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 index b6e14f9465..6db11656b7 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/compress/HeavyHittersBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/compress/HeavyHittersBenchmark.scala @@ -81,9 +81,9 @@ class HeavyHittersBenchmark { @Benchmark @OperationsPerInvocation(8192) def updateExistingHitter(blackhole: Blackhole): Unit = { - var i = 0 + var i: Int = 0 while (i < 8192) { - blackhole.consume(topN.update("HEAVY_HITTER", Long.MaxValue)) // definitely a heavy hitter + blackhole.consume(topN.update(preallocatedStrings(i % 16), Long.MaxValue)) // definitely a heavy hitter i += 1 } } 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 index 4f6a9e0a15..715b081b71 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/TopHeavyHitters.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/TopHeavyHitters.scala @@ -3,13 +3,10 @@ */ package akka.remote.artery.compress -import java.util import java.util.Objects -import akka.japi.Util - import scala.annotation.{ switch, tailrec } -import scala.collection.immutable +import scala.reflect.ClassTag /** * INTERNAL API @@ -22,31 +19,48 @@ import scala.collection.immutable * 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](val max: Int) { - import TopHeavyHitters._ - private[this] var _lowestHitterIdx: Int = 0 +private[remote] final class TopHeavyHitters[T >: Null](val max: Int)(implicit classTag: ClassTag[T]) { - private[this] val hashes: Array[Int] = Array.ofDim(max) - private[this] val items: Array[T] = Array.ofDim[Object](max).asInstanceOf[Array[T]] - private[this] val weights: Array[Long] = Array.ofDim(max) + 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 copy(!) of items which are currently considered to be heavy hitters. */ + /** Returns the current heavy hitters, order is not of significance */ def snapshot: Array[T] = { val snap = Array.ofDim(max).asInstanceOf[Array[T]] - System.arraycopy(items, 0, snap, 0, items.length) + 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: $lowestHitterIdx (weight: $lowestHitterWeight) - | - | hashes: ${hashes.toList.mkString("[", ", ", "]")} - | weights: ${weights.toList.mkString("[", ", ", "]")} - | items: ${items.toList.mkString("[", ", ", "]")} - |)""".stripMargin + | 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, @@ -56,30 +70,48 @@ private[remote] final class TopHeavyHitters[T](val max: Int) { */ // 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 + 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 - (findHashIdx(0, hashCode): @switch) match { // worst case O(n), can't really bin search here since indexes are kept in synch with other arrays hmm... + 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(1) + rarely O(n) if needs to update lowest 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. + // 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) + O(n), we simply replace the current lowest heavy hitter - else replaceExistingHeavyHitter(actualIdx, hashCode, item, count) // usually O(1), worst case O(n) if we need to scan due to hash conflicts + 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 - @tailrec private def findItemIdx(searchFromIndex: Int, hashCode: HashCodeVal, o: T): Int = + 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 findItemIdx(findHashIdx(searchFromIndex + 1, hashCode), hashCode, o) + else loop((searchFromIndex + 1) & mask, searchFromIndex, hashCode, o) + } /** * Replace existing heavy hitter – give it a new `count` value. @@ -91,72 +123,162 @@ private[remote] final class TopHeavyHitters[T](val max: Int) { @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)) { - putCount(foundHashIndex, count) // we don't need to change `hashCode` or `item`, those remain the same - if (foundHashIndex == lowestHitterIdx) updateLowestHitterIdx() // need to update the lowestHitter since we just bumped its count - false // not a "new" heavy hitter, since we only replaced it (so it was signaled as new once before) + 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): Boolean = { - put(_lowestHitterIdx, hashCode, item, count) - updateLowestHitterIdx() - true + 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 putCount(idx: Int, count: Long): Unit = + private def updateCount(idx: Int, count: Long): Unit = weights(idx) = count - private def put(idx: Int, hashCode: HashCodeVal, item: T, count: Long): Unit = { - hashes(idx) = hashCode.get - items(idx) = item - weights(idx) = count - } - - /** Perform a scan for the lowest hitter (by weight). */ - private def updateLowestHitterIdx(): Int = { - _lowestHitterIdx = findIndexOfMinimum(weights) - _lowestHitterIdx + /** + * 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 = - weights(_lowestHitterIdx) - - // do not expose we're array based - private def lowestHitterIdx: Int = - _lowestHitterIdx - - private def findEqIndex(hashes: Array[Int], searchFromIndex: Int, hashCode: Int): Int = { - var i: Int = searchFromIndex - while (i < hashes.length) { - if (hashes(i) == hashCode) return i - i += 1 + def lowestHitterWeight: Long = { + val index: Int = lowestHitterIndex + if (index > 0) { + weights(index) + } else { + 0 } - -1 + } - private def findIndexOfMinimum(weights: Array[Long]): Int = { - var _lowestHitterIdx: Int = -1 - var min: Long = Long.MaxValue + 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 < weights.length) { - if (weights(i) < min) { - min = weights(i) - _lowestHitterIdx = i + while (i < hashes.length) { + val index = (i + searchFromIndex) & mask + if (hashes(index) == hashCode) { + return index } i += 1 } - _lowestHitterIdx + -1 } override def toString = @@ -164,8 +286,10 @@ private[remote] final class TopHeavyHitters[T](val max: Int) { } 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/test/scala/akka/remote/artery/compress/HeavyHittersSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/compress/HeavyHittersSpec.scala index 647d81f318..ba18e46449 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/HeavyHittersSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/HeavyHittersSpec.scala @@ -10,7 +10,7 @@ class HeavyHittersSpec extends WordSpecLike with Matchers { "TopHeavyHitters" must { "should work" in { - val hitters = new TopHeavyHitters[String](3) + val hitters = new TopHeavyHitters[String](4) hitters.update("A", 10) shouldBe true hitters.snapshot.filter(_ ne null).toSet should ===(Set("A")) @@ -21,20 +21,20 @@ class HeavyHittersSpec extends WordSpecLike with Matchers { 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")) + 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("B", "D", "E")) + 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("BB", "D", "E")) + 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("BB", "D", "E")) + hitters.snapshot.filter(_ ne null).toSet should ===(Set("B", "BB", "D", "E")) } "correctly replace a hitter" in { - val hitters = new TopHeavyHitters[String](3) + val hitters = new TopHeavyHitters[String](4) hitters.update("A", 10) shouldBe true hitters.snapshot.filter(_ ne null).toSet should ===(Set("A")) @@ -44,7 +44,7 @@ class HeavyHittersSpec extends WordSpecLike with Matchers { } "correctly drop least heavy hitter when more than N are inserted" in { - val hitters = new TopHeavyHitters[String](3) + val hitters = new TopHeavyHitters[String](4) hitters.update("A", 1) shouldBe true hitters.snapshot.filter(_ ne null).toSet should ===(Set("A")) @@ -54,15 +54,15 @@ class HeavyHittersSpec extends WordSpecLike with Matchers { hitters.update("C", 33) shouldBe true hitters.snapshot.filter(_ ne null).toSet should ===(Set("A", "B", "C")) - hitters.lowestHitterWeight should ===(1) + 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("B", "C", "D")) + 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", "C", "D")) + hitters.snapshot.filter(_ ne null).toSet should ===(Set("X", "B", "C", "D")) } "replace the right item even when hashCodes collide" in { From 0370acc1211d1ce3d725ca35f10a246440488fb6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Wed, 21 Sep 2016 13:24:35 +0200 Subject: [PATCH 155/186] Fix artery segfaults on termination (#21501) --- .../akka/remote/artery/ArteryTransport.scala | 204 ++++++++++-------- .../akka/remote/artery/Association.scala | 94 ++++---- 2 files changed, 173 insertions(+), 125 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 027ea6098c..14f4d9ba27 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -36,6 +36,7 @@ import akka.remote.RemoteTransport import akka.remote.RemotingLifecycleEvent 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 @@ -307,7 +308,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val codec: AkkaPduCodec = AkkaPduProtobufCodec private val killSwitch: SharedKillSwitch = KillSwitches.shared("transportKillSwitch") private[this] val streamCompletions = new AtomicReference(Map.empty[String, Future[Done]]) - @volatile private[this] var _shutdown = false + private[this] val hasBeenShutdown = new AtomicBoolean(false) private val testState = new SharedTestState @@ -426,7 +427,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private lazy val shutdownHook = new Thread { override def run(): Unit = { - if (!_shutdown) { + if (hasBeenShutdown.compareAndSet(false, true)) { Await.result(internalShutdown(), 20.seconds) } } @@ -586,6 +587,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundControlStream(compression: InboundCompressions): Unit = { + if (isShutdown) throw ShuttingDown val (ctrl, completed) = aeronSource(controlStreamId, envelopeBufferPool) .via(inboundFlow(compression)) @@ -596,66 +598,69 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R controlSubject.attach(new ControlMessageObserver { override def notify(inboundEnvelope: InboundEnvelope): Unit = { - - 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)) + 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 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)) - } + } 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) - publishLifecycleEvent(lifecycleEvent) + 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) + publishLifecycleEvent(lifecycleEvent) - case _: ActorSystemTerminating ⇒ - inboundEnvelope.sender match { - case OptionVal.Some(snd) ⇒ snd.tell(ActorSystemTerminatingAck(localAddress), ActorRef.noSender) - case OptionVal.None ⇒ log.error("Expected sender for ActorSystemTerminating message") - } + case _: ActorSystemTerminating ⇒ + inboundEnvelope.sender match { + case OptionVal.Some(snd) ⇒ snd.tell(ActorSystemTerminatingAck(localAddress), ActorRef.noSender) + case OptionVal.None ⇒ log.error("Expected sender for ActorSystemTerminating message") + } - case _ ⇒ // not interesting + case _ ⇒ // not interesting + } + } catch { + case ShuttingDown ⇒ // silence it } } }) @@ -664,6 +669,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundOrdinaryMessagesStream(compression: InboundCompressions): Unit = { + if (isShutdown) throw ShuttingDown val completed = if (inboundLanes == 1) { aeronSource(ordinaryStreamId, envelopeBufferPool) @@ -717,6 +723,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } private def runInboundLargeMessagesStream(): Unit = { + if (isShutdown) throw ShuttingDown val disableCompression = NoInboundCompressions // no compression on large message stream for now val completed = aeronSource(largeStreamId, largeEnvelopeBufferPool) @@ -752,18 +759,21 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } override def shutdown(): Future[Done] = { - _shutdown = true - 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).withDispatcher(settings.Dispatcher), "remoteFlushOnShutdown") - flushingPromise.future - } - implicit val ec = remoteDispatcher - flushing.recover { case _ ⇒ Done }.flatMap(_ ⇒ internalShutdown()) + if (hasBeenShutdown.compareAndSet(false, true)) { + 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).withDispatcher(settings.Dispatcher), "remoteFlushOnShutdown") + flushingPromise.future + } + implicit val ec = remoteDispatcher + flushing.recover { case _ ⇒ Done }.flatMap(_ ⇒ internalShutdown()) + } else { + Future.successful(Done) + } } private def internalShutdown(): Future[Done] = { @@ -820,7 +830,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } yield Done } - private[remote] def isShutdown: Boolean = _shutdown + private[remote] def isShutdown: Boolean = hasBeenShutdown.get() override def managementCommand(cmd: Any): Future[Boolean] = { cmd match { @@ -834,24 +844,33 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // InboundContext override def sendControl(to: Address, message: ControlMessage) = - association(to).sendControl(message) + try { + association(to).sendControl(message) + } catch { + case ShuttingDown ⇒ // silence it + } - override def send(message: Any, sender: OptionVal[ActorRef], recipient: RemoteActorRef): Unit = { - val cached = recipient.cachedAssociation + 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 - } + 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)) - } + a.send(message, sender, OptionVal.Some(recipient)) + } catch { + case ShuttingDown ⇒ // silence it + } override def association(remoteAddress: Address): Association = { - require(remoteAddress != localAddress.address, "Attemted association with self address!") + 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) } @@ -859,16 +878,24 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R associationRegistry.association(uid) override def completeHandshake(peer: UniqueAddress): Future[Done] = { - val a = associationRegistry.setUID(peer) - a.completeHandshake(peer) + try { + val a = associationRegistry.setUID(peer) + a.completeHandshake(peer) + } catch { + case ShuttingDown ⇒ Future.successful(Done) // silence it + } } private def publishLifecycleEvent(event: RemotingLifecycleEvent): Unit = eventPublisher.notifyListeners(event) override def quarantine(remoteAddress: Address, uid: Option[Int], reason: String): Unit = { - // FIXME use Long uid - association(remoteAddress).quarantine(reason, uid.map(_.toLong)) + try { + // FIXME use Long uid + association(remoteAddress).quarantine(reason, uid.map(_.toLong)) + } catch { + case ShuttingDown ⇒ // silence it + } } def outboundLarge(outboundContext: OutboundContext): Sink[OutboundEnvelope, Future[Done]] = @@ -1023,6 +1050,9 @@ private[remote] object ArteryTransport { 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 + def autoSelectPort(hostname: String): Int = { val socket = DatagramChannel.open().socket() socket.bind(new InetSocketAddress(hostname, 0)) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 15fb1d2054..bdbabfc6f0 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -14,7 +14,6 @@ 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 @@ -26,7 +25,7 @@ import akka.remote._ import akka.remote.DaemonMsgCreate import akka.remote.QuarantinedEvent import akka.remote.artery.AeronSink.GaveUpMessageException -import akka.remote.artery.ArteryTransport.AeronTerminated +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 @@ -257,8 +256,12 @@ private[remote] class Association( // OutboundContext override def sendControl(message: ControlMessage): Unit = { - if (!transport.isShutdown) - outboundControlIngress.sendControlMessage(message) + try { + if (!transport.isShutdown) + outboundControlIngress.sendControlMessage(message) + } catch { + case ShuttingDown => // silence it + } } def send(message: Any, sender: OptionVal[ActorRef], recipient: OptionVal[RemoteActorRef]): Unit = { @@ -279,40 +282,43 @@ private[remote] class Association( // allow ActorSelectionMessage to pass through quarantine, to be able to establish interaction with new system if (message.isInstanceOf[ActorSelectionMessage] || !associationState.isQuarantined() || message == ClearSystemMessageDelivery) { - 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) - + 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( @@ -401,6 +407,8 @@ private[remote] class Association( * 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]) @@ -409,6 +417,7 @@ private[remote] class Association( } 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() @@ -419,6 +428,7 @@ private[remote] class Association( } private def runOutboundControlStream(): Unit = { + if (transport.isShutdown) throw ShuttingDown // stage in the control stream may access the outboundControlIngress before returned here // using CountDownLatch to make sure that materialization is completed before accessing outboundControlIngress materializing = new CountDownLatch(1) @@ -453,6 +463,7 @@ private[remote] class Association( } private def runOutboundOrdinaryMessagesStream(): Unit = { + if (transport.isShutdown) throw ShuttingDown if (outboundLanes == 1) { val queueIndex = OrdinaryQueueIndex val wrapper = getOrCreateQueueWrapper(queueIndex, queueSize) @@ -530,6 +541,7 @@ private[remote] class Association( } private def runOutboundLargeMessagesStream(): Unit = { + if (transport.isShutdown) throw ShuttingDown 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 @@ -621,6 +633,9 @@ private[remote] class AssociationRegistry(createAssociation: Address ⇒ Associa 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 { @@ -639,6 +654,9 @@ private[remote] class AssociationRegistry(createAssociation: Address ⇒ Associa 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) From 1926560e41f55618d7cbcdf4f29c38b8dc6052ab Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 19 Sep 2016 11:17:41 +0200 Subject: [PATCH 156/186] stop outbound streams when quarantined, #21407 * they can't be stopped immediately because we want to send some final message and we reply to inbound messages with `Quarantined` * and improve logging --- .../src/main/scala/akka/event/Logging.scala | 11 ++ .../scala/akka/cluster/ClusterHeartbeat.scala | 6 +- .../akka/cluster/ClusterRemoteWatcher.scala | 8 +- akka-remote/src/main/resources/reference.conf | 24 ++-- .../scala/akka/remote/RemoteWatcher.scala | 4 +- .../src/main/scala/akka/remote/Remoting.scala | 5 + .../akka/remote/artery/ArterySettings.scala | 6 +- .../akka/remote/artery/ArteryTransport.scala | 12 +- .../akka/remote/artery/Association.scala | 122 +++++++++++++----- .../scala/akka/remote/artery/Codecs.scala | 14 +- .../scala/akka/remote/artery/Control.scala | 3 +- .../artery/InboundQuarantineCheck.scala | 25 +++- .../remote/artery/MessageDispatcher.scala | 6 +- .../scala/akka/remote/artery/TestStage.scala | 5 +- 14 files changed, 171 insertions(+), 80 deletions(-) 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-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 454b9ba81a..397f3a6da3 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,6 +52,7 @@ private[cluster] class ClusterRemoteWatcher( unreachableReaperInterval, heartbeatExpectedResponseAfter) { + private val arteryEnabled = RARP(context.system).provider.remoteSettings.Artery.Enabled val cluster = Cluster(context.system) import cluster.selfAddress @@ -89,8 +91,10 @@ 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), "Cluster member removed") + // TODO We should probably always quarantine when member is removed, + // but keeping old behavior for old remoting for now + if (arteryEnabled || previousStatus == MemberStatus.Down) { + quarantine(m.address, Some(m.uniqueAddress.uid), s"Cluster member removed, previous status [$previousStatus]") } publishAddressTerminated(m.address) } diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 3d6a596822..f79a1be708 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -147,14 +147,6 @@ akka { # but must be resolved to ActorRefs first. large-message-destinations = [] - # Sets the log granularity level at which Akka logs artery events. This setting - # can take the values OFF, ERROR, WARNING, INFO or DEBUG. Please note that the effective - # logging level is still determined by the global logging level of the actor system: - # for example debug level artery events will be only logged if the system - # is running with debug level logging. - # Failures to deserialize received messages also fall under this flag. - log-lifecycle-events = DEBUG - # If set to a nonempty string artery will use the given dispatcher for # its internal actors otherwise the default dispatcher is used. use-dispatcher = "akka.remote.default-remote-dispatcher" @@ -252,23 +244,29 @@ akka { # 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 - # Timeout after which the inbound stream is going to be restarted. + # See 'inbound-max-restarts' inbound-restart-timeout = 5 seconds - # Max number of restarts for the inbound stream. + # 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 - # Timeout after which outbout stream is going to be restarted for every association. + # See 'outbound-max-restarts' outbound-restart-timeout = 5 seconds - # Max number of restars of the outbound stream for every association. + # 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. diff --git a/akka-remote/src/main/scala/akka/remote/RemoteWatcher.scala b/akka-remote/src/main/scala/akka/remote/RemoteWatcher.scala index 7681eaf8a3..33c8730f40 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteWatcher.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteWatcher.scala @@ -31,8 +31,8 @@ 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 // sent to self only case object HeartbeatTick diff --git a/akka-remote/src/main/scala/akka/remote/Remoting.scala b/akka-remote/src/main/scala/akka/remote/Remoting.scala index 21597bf459..415925a42f 100644 --- a/akka-remote/src/main/scala/akka/remote/Remoting.scala +++ b/akka-remote/src/main/scala/akka/remote/Remoting.scala @@ -59,6 +59,11 @@ private[akka] object RARP extends ExtensionId[RARP] with ExtensionIdProvider { */ private[akka] trait PriorityMessage +/** + * Failure detector heartbeat messages are marked with this trait. + */ +private[akka] trait HeartbeatMessage extends PriorityMessage + /** * INTERNAL API */ diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala index eb14e7b506..630eb9585f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala @@ -53,10 +53,6 @@ private[akka] final class ArterySettings private (config: Config) { val segments = entry.split('/').tail tree.insert(segments, NotUsed) } - val LifecycleEventsLogLevel: LogLevel = Logging.levelFor(toRootLowerCase(getString("log-lifecycle-events"))) match { - case Some(level) ⇒ level - case None ⇒ throw new ConfigurationException("Logging level must be one of (off, debug, info, warning, error)") - } val Dispatcher = getString("use-dispatcher") object Advanced { @@ -103,6 +99,8 @@ private[akka] final class ArterySettings private (config: Config) { 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 ImageLivenessTimeoutNs = config.getMillisDuration("image-liveness-timeout").requiring(interval ⇒ diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 14f4d9ba27..2e17a80c19 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -29,7 +29,6 @@ import akka.actor.Props import akka.event.Logging import akka.event.LoggingAdapter import akka.remote.AddressUidExtension -import akka.remote.EventPublisher import akka.remote.RemoteActorRef import akka.remote.RemoteActorRefProvider import akka.remote.RemoteTransport @@ -303,7 +302,6 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R override def addresses: Set[Address] = _addresses override def localAddressForRemote(remote: Address): Address = defaultAddress override val log: LoggingAdapter = Logging(system, getClass.getName) - val eventPublisher = new EventPublisher(system, log, settings.LifecycleEventsLogLevel) private val codec: AkkaPduCodec = AkkaPduProtobufCodec private val killSwitch: SharedKillSwitch = KillSwitches.shared("transportKillSwitch") @@ -621,7 +619,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R "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)) + _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) @@ -649,7 +647,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R // Instead, the downing strategy should act on ThisActorSystemQuarantinedEvent, e.g. // use it as a STONITH signal. val lifecycleEvent = ThisActorSystemQuarantinedEvent(localAddress.address, from.address) - publishLifecycleEvent(lifecycleEvent) + system.eventStream.publish(lifecycleEvent) case _: ActorSystemTerminating ⇒ inboundEnvelope.sender match { @@ -760,6 +758,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R 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) @@ -886,9 +885,6 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } } - private def publishLifecycleEvent(event: RemotingLifecycleEvent): Unit = - eventPublisher.notifyListeners(event) - override def quarantine(remoteAddress: Address, uid: Option[Int], reason: String): Unit = { try { // FIXME use Long uid @@ -1026,7 +1022,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R /** INTERNAL API: for testing only. */ private[remote] def triggerCompressionAdvertisements(actorRef: Boolean, manifest: Boolean) = { - inboundCompressions.foreach { + _inboundCompressions.foreach { case c: InboundCompressionsImpl if actorRef || manifest ⇒ log.info("Triggering compression table advertisement for {}", c) if (actorRef) c.runNextActorRefAdvertisement() diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index bdbabfc6f0..106d7ddc9d 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -43,6 +43,9 @@ 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 @@ -73,6 +76,8 @@ private[remote] object Association { final val ControlQueueIndex = 0 final val LargeQueueIndex = 1 final val OrdinaryQueueIndex = 2 + + private object OutboundStreamStopSignal extends RuntimeException with NoStackTrace } /** @@ -139,7 +144,8 @@ private[remote] class Association( else Future.sequence(c.map(_.changeActorRefCompression(table))).map(_ ⇒ Done) timeoutAfter(result, changeCompressionTimeout, new ChangeOutboundCompressionFailed) } - private[this] val streamCompletions = new AtomicReference(Map.empty[String, Future[Done]]) + private[this] val streamCompletions = new AtomicReference(Map.empty[String, (SharedKillSwitch, Future[Done])]) + private[this] val idle = new AtomicReference[Option[Cancellable]](None) private[artery] def changeClassManifestCompression(table: CompressionTable[String]): Future[Done] = { import transport.system.dispatcher @@ -180,7 +186,8 @@ private[remote] class Association( case w: LazyQueueWrapper ⇒ w.runMaterialize() case _ ⇒ } - // materialization not completed yet + // 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 @@ -239,6 +246,7 @@ private[remote] class Association( 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) @@ -258,9 +266,14 @@ private[remote] class Association( override def sendControl(message: ControlMessage): Unit = { try { if (!transport.isShutdown) - outboundControlIngress.sendControlMessage(message) + if (associationState.isQuarantined()) { + log.debug("Send control message [{}] to quarantined [{}]", Logging.messageClassName(message), + remoteAddress) + startIdleTimer() + } + outboundControlIngress.sendControlMessage(message) } catch { - case ShuttingDown => // silence it + case ShuttingDown ⇒ // silence it } } @@ -275,13 +288,19 @@ private[remote] class Association( def dropped(queueIndex: Int, qSize: Int, env: OutboundEnvelope): Unit = { log.debug( "Dropping message [{}] from [{}] to [{}] due to overflow of send queue, size [{}]", - message.getClass, sender.getOrElse(deadletters), recipient.getOrElse(recipient), qSize) + 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] || !associationState.isQuarantined() || message == ClearSystemMessageDelivery) { + 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 ⇒ @@ -318,12 +337,12 @@ private[remote] class Association( dropped(queueIndex, queueSize, outboundEnvelope) } } catch { - case ShuttingDown => // silence it + case ShuttingDown ⇒ // silence it } } else if (log.isDebugEnabled) log.debug( "Dropping message [{}] from [{}] to [{}] due to quarantined system [{}]", - message.getClass, sender.getOrElse(deadletters), recipient.getOrElse(recipient), remoteAddress) + Logging.messageClassName(message), sender.getOrElse(deadletters), recipient.getOrElse(recipient), remoteAddress) } private def selectQueue(recipient: OptionVal[RemoteActorRef]): Int = { @@ -374,16 +393,19 @@ private[remote] class Association( if (swapState(current, newState)) { // quarantine state change was performed log.warning( - "Association to [{}] with UID [{}] is irrecoverably failed. Quarantining address. {}", + "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) + // FIXME when we complete the switch to Long UID we must use Long here also, issue #20644 + transport.system.eventStream.publish(QuarantinedEvent(remoteAddress, u.toInt)) clearOutboundCompression() clearInboundCompression(u) - // FIXME when we complete the switch to Long UID we must use Long here also, issue #20644 - transport.eventPublisher.notifyListeners(QuarantinedEvent(remoteAddress, u.toInt)) // 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 } @@ -402,6 +424,22 @@ private[remote] class Association( } + 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()) + streamCompletions.get.valuesIterator.foreach { + case (killSwitch, _) ⇒ killSwitch.abort(OutboundStreamStopSignal) + } + }(transport.system.dispatcher))) + } + /** * Called once after construction when the `Association` instance * wins the CAS in the `AssociationRegistry`. It will materialize @@ -429,16 +467,17 @@ private[remote] class Association( private def runOutboundControlStream(): Unit = { if (transport.isShutdown) throw ShuttingDown - // stage in the control stream may access the outboundControlIngress before returned here - // using CountDownLatch to make sure that materialization is completed before accessing outboundControlIngress - materializing = new CountDownLatch(1) + 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) @@ -449,7 +488,7 @@ private[remote] class Association( _outboundControlIngress = OptionVal.Some(control) materializing.countDown() attachStreamRestart("Outbound control stream", ControlQueueIndex, controlQueueSize, - completed, () ⇒ runOutboundControlStream()) + streamKillSwitch, completed, () ⇒ runOutboundControlStream()) } private def getOrCreateQueueWrapper(queueIndex: Int, capacity: Int): QueueWrapper = { @@ -465,13 +504,17 @@ private[remote] class Association( 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) @@ -483,9 +526,10 @@ private[remote] class Association( changeOutboundCompression = Vector(changeCompression) attachStreamRestart("Outbound message stream", OrdinaryQueueIndex, queueSize, - completed, () ⇒ runOutboundOrdinaryMessagesStream()) + streamKillSwitch, 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 @@ -493,10 +537,10 @@ private[remote] class Association( wrapper }.toVector - val laneKillSwitch = KillSwitches.shared("outboundLaneKillSwitch") + val streamKillSwitch = KillSwitches.shared("outboundMessagesKillSwitch") val lane = Source.fromGraph(new SendQueue[OutboundEnvelope]) - .via(laneKillSwitch.flow) + .via(streamKillSwitch.flow) .via(transport.outboundTestFlow(this)) .viaMat(transport.outboundLane(this))(Keep.both) .watchTermination()(Keep.both) @@ -507,7 +551,7 @@ private[remote] class Association( } val (mergeHub, aeronSinkCompleted) = MergeHub.source[EnvelopeBuffer] - .via(laneKillSwitch.flow) + .via(streamKillSwitch.flow) .toMat(transport.aeronSink(this))(Keep.both).run()(materializer) val values: Vector[(SendQueue.QueueValue[OutboundEnvelope], Encoder.ChangeOutboundCompression, Future[Done])] = @@ -522,9 +566,9 @@ private[remote] class Association( // tear down all parts if one part fails or completes completed.onFailure { - case reason: Throwable ⇒ laneKillSwitch.abort(reason) + case reason: Throwable ⇒ streamKillSwitch.abort(reason) } - (laneCompletedValues :+ aeronSinkCompleted).foreach(_.onSuccess { case _ ⇒ laneKillSwitch.shutdown() }) + (laneCompletedValues :+ aeronSinkCompleted).foreach(_.onSuccess { case _ ⇒ streamKillSwitch.shutdown() }) queueValues.zip(wrappers).zipWithIndex.foreach { case ((q, w), i) ⇒ @@ -536,17 +580,21 @@ private[remote] class Association( changeOutboundCompression = changeCompressionValues attachStreamRestart("Outbound message stream", OrdinaryQueueIndex, queueSize, - completed, () ⇒ runOutboundOrdinaryMessagesStream()) + streamKillSwitch, 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) @@ -556,32 +604,38 @@ private[remote] class Association( queues(LargeQueueIndex) = queueValue queuesVisibility = true // volatile write for visibility of the queues array attachStreamRestart("Outbound large message stream", LargeQueueIndex, largeQueueSize, - completed, () ⇒ runOutboundLargeMessagesStream()) + streamKillSwitch, completed, () ⇒ runOutboundLargeMessagesStream()) } private def attachStreamRestart(streamName: String, queueIndex: Int, queueCapacity: Int, - streamCompleted: Future[Done], restart: () ⇒ Unit): Unit = { + streamKillSwitch: SharedKillSwitch, streamCompleted: Future[Done], restart: () ⇒ Unit): Unit = { def lazyRestart(): Unit = { changeOutboundCompression = Vector.empty - if (queueIndex == ControlQueueIndex) + 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 - updateStreamCompletion(streamName, streamCompleted.recover { case _ ⇒ Done }) + updateStreamCompletion(streamName, (streamKillSwitch, streamCompleted.recover { case _ ⇒ Done })) streamCompleted.onFailure { case ArteryTransport.ShutdownSignal ⇒ // shutdown as expected 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) + log.error(cause, s"{} to [{}] failed after shutdown. {}", streamName, remoteAddress, cause.getMessage) 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) + log.debug("{} to [{}] failed. Restarting it. {}", streamName, remoteAddress, cause.getMessage) // restart unconditionally, without counting restarts lazyRestart() case cause ⇒ @@ -593,10 +647,10 @@ private[remote] class Association( } if (restartCounter.restart()) { - log.error(cause, "{} to {} failed. Restarting it. {}", streamName, remoteAddress, cause.getMessage) + 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}", + 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() } @@ -605,10 +659,10 @@ private[remote] class Association( // set the future that completes when the current stream for a given name completes @tailrec - private def updateStreamCompletion(streamName: String, streamCompleted: Future[Done]): Unit = { + private def updateStreamCompletion(streamName: String, streamCompletion: (SharedKillSwitch, Future[Done])): Unit = { val prev = streamCompletions.get() - if (!streamCompletions.compareAndSet(prev, prev + (streamName → streamCompleted))) { - updateStreamCompletion(streamName, streamCompleted) + if (!streamCompletions.compareAndSet(prev, prev + (streamName → streamCompletion))) { + updateStreamCompletion(streamName, streamCompletion) } } @@ -618,7 +672,7 @@ private[remote] class Association( */ def streamsCompleted: Future[Done] = { implicit val ec = materializer.executionContext - Future.sequence(streamCompletions.get().values).map(_ ⇒ Done) + Future.sequence(streamCompletions.get().values.map(_._2)).map(_ ⇒ Done) } override def toString: String = diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index a470e5ae89..3e6ae64836 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -23,6 +23,7 @@ import akka.Done import akka.stream.stage.GraphStageWithMaterializedValue import scala.concurrent.Promise +import akka.event.Logging /** * INTERNAL API @@ -126,15 +127,18 @@ private[remote] class Encoder( bufferPool.release(envelope) outboundEnvelope.message match { case _: SystemMessageEnvelope ⇒ - log.error(e, "Failed to serialize system message [{}].", outboundEnvelope.message.getClass.getName) + 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(s"Discarding oversized payload sent to ${outboundEnvelope.recipient}: " + - s"max allowed size ${envelope.byteBuffer.limit()} bytes. Message type [${outboundEnvelope.message.getClass.getName}].") - log.error(reason, "Failed to serialize oversized message [{}].", outboundEnvelope.message.getClass.getName) + 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 [{}].", outboundEnvelope.message.getClass.getName) + log.error(e, "Failed to serialize message [{}].", Logging.messageClassName(outboundEnvelope.message)) pull(in) } } finally { diff --git a/akka-remote/src/main/scala/akka/remote/artery/Control.scala b/akka-remote/src/main/scala/akka/remote/artery/Control.scala index d28df42a05..0e89ca46c5 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Control.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Control.scala @@ -18,6 +18,7 @@ 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[akka] trait ArteryMessage extends Serializable @@ -206,7 +207,7 @@ private[akka] class OutboundControlJunction( buffer.offer(wrap(message)) else { // it's alright to drop control messages - log.debug("Dropping control message [{}] due to full buffer.", message.getClass.getName) + log.debug("Dropping control message [{}] due to full buffer.", Logging.messageClassName(message)) } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala b/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala index a2b58991f5..8ac426bdca 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala @@ -13,6 +13,9 @@ 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 @@ -23,7 +26,9 @@ private[akka] class InboundQuarantineCheck(inboundContext: InboundContext) exten override val shape: FlowShape[InboundEnvelope, InboundEnvelope] = FlowShape(in, out) override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = - new GraphStageLogic(shape) with InHandler with OutHandler { + new GraphStageLogic(shape) with InHandler with OutHandler with StageLogging { + + override protected def logSource = classOf[InboundQuarantineCheck] // InHandler override def onPush(): Unit = { @@ -34,15 +39,27 @@ private[akka] class InboundQuarantineCheck(inboundContext: InboundContext) exten push(out, env) case OptionVal.Some(association) ⇒ if (association.associationState.isQuarantined(env.originUid)) { - inboundContext.sendControl( - association.remoteAddress, - Quarantined(inboundContext.localAddress, UniqueAddress(association.remoteAddress, 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) diff --git a/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala b/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala index b5dc011a6e..4a61c4b191 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala @@ -64,7 +64,9 @@ private[akka] class MessageDispatcher( // 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 ⇒ - log.debug("operating in UntrustedMode, dropping inbound PossiblyHarmful message of type [{}]", msg.getClass.getName) + log.debug( + "operating in UntrustedMode, dropping inbound PossiblyHarmful message of type [{}]", + Logging.messageClassName(msg)) case msg: SystemMessage ⇒ l.sendSystemMessage(msg) case msg ⇒ l.!(msg)(sender) } @@ -76,7 +78,7 @@ private[akka] class MessageDispatcher( case r ⇒ log.error( "dropping message [{}] for unknown recipient [{}] arriving at [{}] inbound addresses are [{}]", - message.getClass, r, recipientAddress, provider.transport.addresses.mkString(", ")) + Logging.messageClassName(message), r, recipientAddress, provider.transport.addresses.mkString(", ")) } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala b/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala index a71bb341c0..2b7ca5e27c 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/TestStage.scala @@ -20,6 +20,7 @@ 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 @@ -98,7 +99,7 @@ private[remote] class OutboundTestStage(outboundContext: OutboundContext, state: if (state.isBlackhole(outboundContext.localAddress.address, outboundContext.remoteAddress)) { log.debug( "dropping outbound message [{}] to [{}] because of blackhole", - env.message.getClass.getName, outboundContext.remoteAddress) + Logging.messageClassName(env.message), outboundContext.remoteAddress) pull(in) // drop message } else push(out, env) @@ -144,7 +145,7 @@ private[remote] class InboundTestStage(inboundContext: InboundContext, state: Sh if (state.isBlackhole(inboundContext.localAddress.address, association.remoteAddress)) { log.debug( "dropping inbound message [{}] from [{}] with UID [{}] because of blackhole", - env.message.getClass.getName, association.remoteAddress, env.originUid) + Logging.messageClassName(env.message), association.remoteAddress, env.originUid) pull(in) // drop message } else push(out, env) From f1590a59b4c9d6e1c826de587dcd2ca13798275b Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 21 Sep 2016 17:27:34 +0200 Subject: [PATCH 157/186] revert quarantine removed (leaving) cluster member, #21509 --- .../src/main/scala/akka/cluster/ClusterRemoteWatcher.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala index 397f3a6da3..5dd1ed3da8 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala @@ -52,7 +52,6 @@ private[cluster] class ClusterRemoteWatcher( unreachableReaperInterval, heartbeatExpectedResponseAfter) { - private val arteryEnabled = RARP(context.system).provider.remoteSettings.Artery.Enabled val cluster = Cluster(context.system) import cluster.selfAddress @@ -91,9 +90,9 @@ private[cluster] class ClusterRemoteWatcher( def memberRemoved(m: Member, previousStatus: MemberStatus): Unit = if (m.address != selfAddress) { clusterNodes -= m.address - // TODO We should probably always quarantine when member is removed, - // but keeping old behavior for old remoting for now - if (arteryEnabled || previousStatus == MemberStatus.Down) { + // The reason we don't quarantine gracefully removed members (leaving) is that + // Cluster Singleton need to exchange TakeOver/HandOver messages. + if (previousStatus == MemberStatus.Down) { quarantine(m.address, Some(m.uniqueAddress.uid), s"Cluster member removed, previous status [$previousStatus]") } publishAddressTerminated(m.address) From 9f175f56deb69025137f11436ecd2f491b44b162 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 21 Sep 2016 20:27:04 +0200 Subject: [PATCH 158/186] fix problem with quick restart, #21512 * image-liveness-timeout must be less than the handshake-timeout, otherwise the publication for the handshake will give up too early when previous image is still considered alive --- .../scala/akka/cluster/SharedMediaDriverSupport.scala | 2 +- akka-remote/src/main/resources/reference.conf | 6 ++++-- .../src/main/scala/akka/remote/artery/ArterySettings.scala | 3 ++- .../src/main/scala/akka/remote/artery/ArteryTransport.scala | 2 +- .../test/scala/akka/remote/artery/HandshakeDenySpec.scala | 1 + .../scala/akka/remote/artery/HandshakeFailureSpec.scala | 1 + .../test/scala/akka/remote/artery/HandshakeRetrySpec.scala | 1 + .../src/test/scala/akka/remote/artery/LateConnectSpec.scala | 1 + .../compress/HandshakeShouldDropCompressionTableSpec.scala | 1 + 9 files changed, 13 insertions(+), 5 deletions(-) diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/SharedMediaDriverSupport.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/SharedMediaDriverSupport.scala index 925bfbc4d5..f45a54b245 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/SharedMediaDriverSupport.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/SharedMediaDriverSupport.scala @@ -35,7 +35,7 @@ object SharedMediaDriverSupport { val driverContext = new MediaDriver.Context driverContext.aeronDirectoryName(aeronDir) driverContext.clientLivenessTimeoutNs(arterySettings.Advanced.ClientLivenessTimeout.toNanos) - driverContext.imageLivenessTimeoutNs(arterySettings.Advanced.ImageLivenessTimeoutNs.toNanos) + driverContext.imageLivenessTimeoutNs(arterySettings.Advanced.ImageLivenessTimeout.toNanos) driverContext.driverTimeoutMs(arterySettings.Advanced.DriverTimeout.toMillis) val idleCpuLevel = arterySettings.Advanced.IdleCpuLevel diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index f79a1be708..ef2116885f 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -227,7 +227,8 @@ akka { system-message-resend-interval = 1 second # The timeout for outbound associations to perform the handshake. - handshake-timeout = 15 s + # 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 @@ -274,7 +275,8 @@ akka { # Timeout for each the INACTIVE and LINGER stages an aeron image # will be retained for when it is no longer referenced. - image-liveness-timeout = 20 seconds + # 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. diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala index 630eb9585f..dcb54fa34d 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala @@ -103,8 +103,9 @@ private[akka] final class ArterySettings private (config: Config) { 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 ImageLivenessTimeoutNs = config.getMillisDuration("image-liveness-timeout").requiring(interval ⇒ + 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") diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 2e17a80c19..5dbbcf7513 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -443,7 +443,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R driverContext.aeronDirectoryName(randomName) } driverContext.clientLivenessTimeoutNs(settings.Advanced.ClientLivenessTimeout.toNanos) - driverContext.imageLivenessTimeoutNs(settings.Advanced.ImageLivenessTimeoutNs.toNanos) + driverContext.imageLivenessTimeoutNs(settings.Advanced.ImageLivenessTimeout.toNanos) driverContext.driverTimeoutMs(settings.Advanced.DriverTimeout.toMillis) val idleCpuLevel = settings.Advanced.IdleCpuLevel diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeDenySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeDenySpec.scala index 864c0bbe49..fe6e9993ca 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeDenySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeDenySpec.scala @@ -20,6 +20,7 @@ object HandshakeDenySpec { remote.artery.canonical.hostname = localhost remote.artery.canonical.port = 0 remote.artery.advanced.handshake-timeout = 2s + remote.artery.advanced.image-liveness-timeout = 1.9s } """) diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala index 802f5a58f8..ec72c0bd4a 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeFailureSpec.scala @@ -24,6 +24,7 @@ object HandshakeFailureSpec { remote.artery.canonical.hostname = localhost remote.artery.canonical.port = 0 remote.artery.advanced.handshake-timeout = 2s + remote.artery.advanced.image-liveness-timeout = 1.9s } """) diff --git a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala index 2ce40262e0..8573ad34f1 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/HandshakeRetrySpec.scala @@ -23,6 +23,7 @@ object HandshakeRetrySpec { remote.artery.canonical.hostname = localhost remote.artery.canonical.port = 0 remote.artery.advanced.handshake-timeout = 10s + remote.artery.advanced.image-liveness-timeout = 7s } """) diff --git a/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala index 36f1690dd2..e50b71d91a 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/LateConnectSpec.scala @@ -24,6 +24,7 @@ object LateConnectSpec { remote.artery.canonical.hostname = localhost remote.artery.canonical.port = 0 remote.artery.advanced.handshake-timeout = 3s + remote.artery.advanced.image-liveness-timeout = 2.9s } """) 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 index 869d8c0ff1..8cc12fc4d8 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/HandshakeShouldDropCompressionTableSpec.scala @@ -31,6 +31,7 @@ object HandshakeShouldDropCompressionTableSpec { 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 { From 455d6a45cc9ed55d046fea68395f003fc244efe6 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 22 Sep 2016 11:07:17 +0200 Subject: [PATCH 159/186] fix shutdown race in sendControl, #21514 (#21517) * fix shutdown race in sendControl, #21514 * the stack trace showed IllegalStateException: outboundControlIngress not initialized yet via the call to sendControl * that could happen if there is a shutdown at the same time, which is exactly what the test does * it was actually caused by a merge mistake, but now it got even better * countDown latch on shutdown --- .../akka/remote/testkit/MultiNodeSpec.scala | 4 ++++ .../akka/remote/artery/ArteryTransport.scala | 1 + .../scala/akka/remote/artery/Association.scala | 18 +++++++++++++----- 3 files changed, 18 insertions(+), 5 deletions(-) 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 079de2f5f1..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 @@ -59,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 diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 2e17a80c19..7a0407471c 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -426,6 +426,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R 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) } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 106d7ddc9d..8226b6c1f1 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -191,7 +191,9 @@ private[remote] class Association( materializing.await(10, TimeUnit.SECONDS) _outboundControlIngress match { case OptionVal.Some(o) ⇒ o - case OptionVal.None ⇒ throw new IllegalStateException("outboundControlIngress not initialized yet") + case OptionVal.None ⇒ + if (transport.isShutdown) throw ShuttingDown + else throw new IllegalStateException("outboundControlIngress not initialized yet") } } } @@ -265,13 +267,14 @@ private[remote] class Association( // OutboundContext override def sendControl(message: ControlMessage): Unit = { try { - if (!transport.isShutdown) + if (!transport.isShutdown) { if (associationState.isQuarantined()) { log.debug("Send control message [{}] to quarantined [{}]", Logging.messageClassName(message), remoteAddress) startIdleTimer() } - outboundControlIngress.sendControlMessage(message) + outboundControlIngress.sendControlMessage(message) + } } catch { case ShuttingDown ⇒ // silence it } @@ -624,11 +627,16 @@ private[remote] class Association( implicit val ec = materializer.executionContext updateStreamCompletion(streamName, (streamKillSwitch, streamCompleted.recover { case _ ⇒ Done })) streamCompleted.onFailure { - case ArteryTransport.ShutdownSignal ⇒ // shutdown as expected - case _: AeronTerminated ⇒ // shutdown already in progress + 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 From 1a6661f552345fd1ecaec4824742c198e6ff691e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Endre=20S=C3=A1ndor=20Varga?= Date: Thu, 22 Sep 2016 13:49:56 +0200 Subject: [PATCH 160/186] 21400: Flush ordinary and control message streams --- .../akka/remote/artery/ArteryTransport.scala | 46 ++++++++++--- .../akka/remote/artery/Association.scala | 36 +++++++++- .../scala/akka/remote/artery/SendQueue.scala | 4 ++ .../remote/artery/FlushOnShutdownSpec.scala | 69 +++++++++++++++++++ 4 files changed, 143 insertions(+), 12 deletions(-) create mode 100644 akka-remote/src/test/scala/akka/remote/artery/FlushOnShutdownSpec.scala diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 7a0407471c..13aa0fe9e5 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -240,15 +240,19 @@ private[remote] object FlushOnShutdown { private[remote] class FlushOnShutdown(done: Promise[Done], timeout: FiniteDuration, inboundContext: InboundContext, associations: Set[Association]) extends Actor { - var remaining = associations.flatMap(_.associationState.uniqueRemoteAddressValue) + var remaining = Map.empty[UniqueAddress, Int] val timeoutTask = context.system.scheduler.scheduleOnce(timeout, self, FlushOnShutdown.Timeout)(context.dispatcher) override def preStart(): Unit = { - // FIXME shall we also try to flush the ordinary message stream, not only control stream? - val msg = ActorSystemTerminating(inboundContext.localAddress) try { - associations.foreach { a ⇒ a.send(msg, OptionVal.Some(self), OptionVal.None) } + 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 + } + } } catch { case NonFatal(e) ⇒ // send may throw @@ -264,7 +268,14 @@ private[remote] class FlushOnShutdown(done: Promise[Done], timeout: FiniteDurati def receive = { case ActorSystemTerminatingAck(from) ⇒ - remaining -= 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 ⇒ @@ -650,12 +661,6 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R val lifecycleEvent = ThisActorSystemQuarantinedEvent(localAddress.address, from.address) system.eventStream.publish(lifecycleEvent) - case _: ActorSystemTerminating ⇒ - inboundEnvelope.sender match { - case OptionVal.Some(snd) ⇒ snd.tell(ActorSystemTerminatingAck(localAddress), ActorRef.noSender) - case OptionVal.None ⇒ log.error("Expected sender for ActorSystemTerminating message") - } - case _ ⇒ // not interesting } } catch { @@ -976,10 +981,28 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R 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) @@ -1000,6 +1023,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R 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) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 8226b6c1f1..eaaa875f3f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -57,6 +57,18 @@ private[remote] object Association { 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 { @@ -71,6 +83,8 @@ private[remote] object Association { runMaterialize() queue.offer(message) } + + override def isEnabled: Boolean = true } final val ControlQueueIndex = 0 @@ -120,7 +134,12 @@ private[remote] class Association( private[this] val queues: Array[SendQueue.ProducerApi[OutboundEnvelope]] = Array.ofDim(2 + outboundLanes) queues(ControlQueueIndex) = QueueWrapperImpl(createQueue(controlQueueSize)) // control stream - queues(LargeQueueIndex) = QueueWrapperImpl(createQueue(largeQueueSize)) // large messages 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 } @@ -379,6 +398,21 @@ private[remote] class Association( } } + def sendTerminationHint(replyTo: ActorRef): Int = { + if (!associationState.isQuarantined()) { + val msg = ActorSystemTerminating(localAddress) + var sent = 0 + queues.iterator.filter(_.isEnabled).foreach { queue ⇒ + val envelope = outboundEnvelopePool.acquire() + .init(OptionVal.None, msg, OptionVal.Some(replyTo)) + + queue.offer(envelope) + sent += 1 + } + sent + } else 0 + } + // OutboundContext override def quarantine(reason: String): Unit = { val uid = associationState.uniqueRemoteAddressValue().map(_.uid) diff --git a/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala b/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala index 73bcd0609b..075a6064ab 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SendQueue.scala @@ -27,6 +27,8 @@ import scala.util.Failure private[remote] object SendQueue { trait ProducerApi[T] { def offer(message: T): Boolean + + def isEnabled: Boolean } trait QueueValue[T] extends ProducerApi[T] { @@ -126,6 +128,8 @@ private[remote] final class SendQueue[T] extends GraphStageWithMaterializedValue } result } + + override def isEnabled: Boolean = true } (logic, queueValue) 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) + } + + } + +} From 30603d194a65b7cd37c2f7d5bb1f39da7e709bea Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 23 Sep 2016 12:30:54 +0200 Subject: [PATCH 161/186] log-received-messages and log-sent-messages, #21506 (#21508) * log-received-messages and log-sent-messages, #21506 * also duplicate the trusted settings to artery section --- .../akka/remote/artery/CodecBenchmark.scala | 4 +- akka-remote/src/main/resources/reference.conf | 21 +++++++- .../akka/remote/artery/ArterySettings.scala | 7 +++ .../akka/remote/artery/ArteryTransport.scala | 8 ++- .../scala/akka/remote/artery/Codecs.scala | 16 +++++- .../akka/remote/artery/InboundEnvelope.scala | 7 +-- .../remote/artery/MessageDispatcher.scala | 50 +++++++++++-------- .../akka/remote/artery/UntrustedSpec.scala | 4 +- 8 files changed, 83 insertions(+), 34 deletions(-) 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 index 0e6927c705..f0903d4925 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -140,7 +140,7 @@ class CodecBenchmark { val N = 100000 val encoder: Flow[OutboundEnvelope, EnvelopeBuffer, Encoder.ChangeOutboundCompression] = - Flow.fromGraph(new Encoder(uniqueLocalAddress, system.asInstanceOf[ExtendedActorSystem], outboundEnvelopePool, envelopePool)) + 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))) @@ -197,7 +197,7 @@ class CodecBenchmark { val N = 100000 val encoder: Flow[OutboundEnvelope, EnvelopeBuffer, Encoder.ChangeOutboundCompression] = - Flow.fromGraph(new Encoder(uniqueLocalAddress, system.asInstanceOf[ExtendedActorSystem], outboundEnvelopePool, envelopePool)) + Flow.fromGraph(new Encoder(uniqueLocalAddress, system.asInstanceOf[ExtendedActorSystem], outboundEnvelopePool, envelopePool, false)) val localRecipient = resolvedRef.path.toSerializationFormatWithAddress(uniqueLocalAddress.address) val provider = RARP(system).provider diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index ef2116885f..447efb3c8b 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -89,7 +89,7 @@ akka { artery { - ### FIXME: Temporary switch for the PoC + # Enable the new remoting with this flag enabled = off # Canonical address is the address other clients should connect to. @@ -150,6 +150,25 @@ akka { # If set to a nonempty string artery will use the given dispatcher for # its internal actors otherwise the default dispatcher is used. use-dispatcher = "akka.remote.default-remote-dispatcher" + + # 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 { # For enabling testing features, such as blackhole in akka-remote-testkit. diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala index dcb54fa34d..3e68421f0b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala @@ -3,6 +3,7 @@ */ package akka.remote.artery +import akka.japi.Util.immutableSeq import akka.ConfigurationException import akka.event.Logging import akka.event.Logging.LogLevel @@ -55,6 +56,12 @@ private[akka] final class ArterySettings private (config: Config) { } val Dispatcher = getString("use-dispatcher") + 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._ diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 38845a4e3d..0d50d89305 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -959,14 +959,18 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R new InboundCompressionsImpl(system, inboundContext, settings.Advanced.Compression) def createEncoder(pool: EnvelopeBufferPool): Flow[OutboundEnvelope, EnvelopeBuffer, ChangeOutboundCompression] = - Flow.fromGraph(new Encoder(localAddress, system, outboundEnvelopePool, pool)) + Flow.fromGraph(new Encoder(localAddress, system, outboundEnvelopePool, pool, settings.LogSend)) def aeronSource(streamId: Int, pool: EnvelopeBufferPool): Source[EnvelopeBuffer, NotUsed] = Source.fromGraph(new AeronSource(inboundChannel, streamId, aeron, taskRunner, pool, createFlightRecorderEventSink())) val messageDispatcherSink: Sink[InboundEnvelope, Future[Done]] = Sink.foreach[InboundEnvelope] { m ⇒ - messageDispatcher.dispatch(m.recipient.get, m.recipientAddress, m.message, m.sender) + 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 _ ⇒ diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 3e6ae64836..f2f90d51db 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -47,7 +47,8 @@ private[remote] class Encoder( uniqueLocalAddress: UniqueAddress, system: ExtendedActorSystem, outboundEnvelopePool: ObjectPool[ReusableOutboundEnvelope], - bufferPool: EnvelopeBufferPool) + bufferPool: EnvelopeBufferPool, + debugLogSend: Boolean) extends GraphStageWithMaterializedValue[FlowShape[OutboundEnvelope, EnvelopeBuffer], Encoder.ChangeOutboundCompression] { import Encoder._ @@ -89,6 +90,12 @@ private[remote] class Encoder( 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() @@ -120,6 +127,13 @@ private[remote] class Encoder( } 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 { diff --git a/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala b/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala index b9d17362ae..8caae6576b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala @@ -33,7 +33,6 @@ private[remote] object InboundEnvelope { */ private[remote] trait InboundEnvelope { def recipient: OptionVal[InternalActorRef] - def recipientAddress: Address def sender: OptionVal[ActorRef] def originUid: Long def association: OptionVal[OutboundContext] @@ -67,7 +66,6 @@ private[remote] object ReusableInboundEnvelope { */ private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { private var _recipient: OptionVal[InternalActorRef] = OptionVal.None - private var _recipientAddress: Address = null private var _sender: OptionVal[ActorRef] = OptionVal.None private var _originUid: Long = 0L private var _association: OptionVal[OutboundContext] = OptionVal.None @@ -78,7 +76,6 @@ private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { private var _envelopeBuffer: EnvelopeBuffer = null override def recipient: OptionVal[InternalActorRef] = _recipient - override def recipientAddress: Address = _recipientAddress override def sender: OptionVal[ActorRef] = _sender override def originUid: Long = _originUid override def association: OptionVal[OutboundContext] = _association @@ -107,7 +104,6 @@ private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { def clear(): Unit = { _recipient = OptionVal.None - _recipientAddress = null _message = null _sender = OptionVal.None _originUid = 0L @@ -124,7 +120,6 @@ private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { envelopeBuffer: EnvelopeBuffer, association: OptionVal[OutboundContext]): InboundEnvelope = { _recipient = recipient - _recipientAddress = recipientAddress _sender = sender _originUid = originUid _serializer = serializer @@ -136,5 +131,5 @@ private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { } override def toString: String = - s"InboundEnvelope($recipient, $recipientAddress, $message, $sender, $originUid, $association)" + s"InboundEnvelope($recipient, $message, $sender, $originUid, $association)" } diff --git a/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala b/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala index 4a61c4b191..7eed45c201 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala @@ -17,6 +17,7 @@ import akka.event.Logging import akka.remote.RemoteActorRefProvider import akka.remote.RemoteRef import akka.util.OptionVal +import akka.event.LoggingReceive /** * INTERNAL API @@ -27,58 +28,67 @@ private[akka] class MessageDispatcher( private val remoteDaemon = provider.remoteDaemon private val log = Logging(system, getClass.getName) + private val debugLogEnabled = log.isDebugEnabled def dispatch( - recipient: InternalActorRef, - recipientAddress: Address, - message: AnyRef, - senderOption: OptionVal[ActorRef]): Unit = { + recipient: InternalActorRef, + message: AnyRef, + senderOption: OptionVal[ActorRef], + originAddress: OptionVal[Address]): Unit = { - import provider.remoteSettings._ + import provider.remoteSettings.Artery._ + import Logging.messageClassName val sender: ActorRef = senderOption.getOrElse(system.deadLetters) val originalReceiver = recipient.path - def msgLog = s"RemoteMessage: [$message] to [$recipient]<+[$originalReceiver] from [$sender()]" - recipient match { case `remoteDaemon` ⇒ - if (UntrustedMode) log.debug("dropping daemon message in untrusted mode") - else { - if (LogReceive) log.debug("received daemon message {}", msgLog) + 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) log.debug("received local message {}", msgLog) + 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)) - log.debug( + 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 + } 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 ⇒ - log.debug( - "operating in UntrustedMode, dropping inbound PossiblyHarmful message of type [{}]", - Logging.messageClassName(msg)) + 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) log.debug("received remote-destined message {}", msgLog) + 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 [{}] arriving at [{}] inbound addresses are [{}]", - Logging.messageClassName(message), r, recipientAddress, provider.transport.addresses.mkString(", ")) + "dropping message [{}] for unknown recipient [{}] from [{}]", + messageClassName(message), r, senderOption.getOrElse(originAddress.getOrElse(""))) } } diff --git a/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala index 2ed521e292..2d901f492e 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/UntrustedSpec.scala @@ -63,8 +63,8 @@ object UntrustedSpec { class UntrustedSpec extends AkkaSpec(""" akka.actor.provider = remote - akka.remote.untrusted-mode = on - akka.remote.trusted-selection-paths = ["/user/receptionist", ] + 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 From 1408a47e0084634a5439237dd9541052cc995dd5 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 23 Sep 2016 08:03:26 +0200 Subject: [PATCH 162/186] freeSessionBuffer in AeronSource FragmentAssembler, #21401 --- .../akka/remote/artery/AeronSource.scala | 49 ++++++++++++++- .../akka/remote/artery/ArteryTransport.scala | 59 +++++++++++++------ .../akka/remote/artery/Association.scala | 44 +++++++++----- 3 files changed, 114 insertions(+), 38 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala index a6133d0104..2dda28a2c0 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala @@ -21,6 +21,8 @@ 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 object AeronSource { @@ -55,6 +57,10 @@ object AeronSource { onMessage(envelope) } }) + + trait ResourceLifecycle { + def onUnavailableImage(sessionId: Int): Unit + } } /** @@ -67,7 +73,7 @@ class AeronSource( taskRunner: TaskRunner, pool: EnvelopeBufferPool, flightRecorder: EventSink) - extends GraphStage[SourceShape[EnvelopeBuffer]] { + extends GraphStageWithMaterializedValue[SourceShape[EnvelopeBuffer], AeronSource.ResourceLifecycle] { import AeronSource._ import TaskRunner._ import FlightRecorderEvents._ @@ -75,8 +81,8 @@ class AeronSource( val out: Outlet[EnvelopeBuffer] = Outlet("AeronSource") override val shape: SourceShape[EnvelopeBuffer] = SourceShape(out) - override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = - new GraphStageLogic(shape) with OutHandler { + 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 @@ -91,6 +97,14 @@ class AeronSource( 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) } @@ -126,6 +140,7 @@ class AeronSource( } else { // delegate backoff to shared TaskRunner flightRecorder.hiFreq(AeronSource_DelegateToTaskRunner, countBeforeDelegate) + delegatingToTaskRunner = true delegateTaskStartTime = System.nanoTime() taskRunner.command(addPollTask) } @@ -134,7 +149,9 @@ class AeronSource( private def taskOnMessage(data: EnvelopeBuffer): Unit = { countBeforeDelegate = 0 + delegatingToTaskRunner = false flightRecorder.hiFreq(AeronSource_ReturnFromTaskRunner, System.nanoTime() - delegateTaskStartTime) + freeSessionBuffers() onMessage(data) } @@ -143,6 +160,32 @@ class AeronSource( 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/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 0d50d89305..d777708c21 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -68,6 +68,7 @@ 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 @@ -290,6 +291,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R 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 @@ -316,7 +318,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val codec: AkkaPduCodec = AkkaPduProtobufCodec private val killSwitch: SharedKillSwitch = KillSwitches.shared("transportKillSwitch") - private[this] val streamCompletions = new AtomicReference(Map.empty[String, Future[Done]]) + // 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 @@ -531,7 +534,11 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R override def onUnavailableImage(img: Image): Unit = { if (log.isDebugEnabled) log.debug(s"onUnavailableImage from ${img.sourceIdentity} session ${img.sessionId}") - // FIXME we should call FragmentAssembler.freeSessionBuffer when image is unavailable + + // freeSessionBuffer in AeronSource FragmentAssembler + streamMatValues.get.valuesIterator.foreach { + case InboundStreamMatValues(resourceLife, _) ⇒ resourceLife.onUnavailableImage(img.sessionId) + } } }) @@ -598,10 +605,10 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private def runInboundControlStream(compression: InboundCompressions): Unit = { if (isShutdown) throw ShuttingDown - val (ctrl, completed) = + val (resourceLife, (ctrl, completed)) = aeronSource(controlStreamId, envelopeBufferPool) .via(inboundFlow(compression)) - .toMat(inboundControlSink)(Keep.right) + .toMat(inboundControlSink)(Keep.both) .run()(materializer) controlSubject = ctrl @@ -669,16 +676,17 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } }) + updateStreamMatValues(controlStreamId, resourceLife, completed) attachStreamRestart("Inbound control stream", completed, () ⇒ runInboundControlStream(compression)) } private def runInboundOrdinaryMessagesStream(compression: InboundCompressions): Unit = { if (isShutdown) throw ShuttingDown - val completed = + val (resourceLife, completed) = if (inboundLanes == 1) { aeronSource(ordinaryStreamId, envelopeBufferPool) .via(inboundFlow(compression)) - .toMat(inboundSink(envelopeBufferPool))(Keep.right) + .toMat(inboundSink(envelopeBufferPool))(Keep.both) .run()(materializer) } else { @@ -688,7 +696,10 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R .via(inboundFlow(compression)) .map(env ⇒ (env.recipient, env)) - val broadcastHub = source.runWith(BroadcastHub.sink(bufferSize = settings.Advanced.InboundBroadcastHubBufferSize))(materializer) + val (resourceLife, broadcastHub) = + source + .toMat(BroadcastHub.sink(bufferSize = settings.Advanced.InboundBroadcastHubBufferSize))(Keep.both) + .run()(materializer) val lane = inboundSink(envelopeBufferPool) @@ -720,9 +731,10 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R case reason: Throwable ⇒ hubKillSwitch.abort(reason) } - completed + (resourceLife, completed) } + updateStreamMatValues(ordinaryStreamId, resourceLife, completed) attachStreamRestart("Inbound message stream", completed, () ⇒ runInboundOrdinaryMessagesStream(compression)) } @@ -730,17 +742,17 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R if (isShutdown) throw ShuttingDown val disableCompression = NoInboundCompressions // no compression on large message stream for now - val completed = aeronSource(largeStreamId, largeEnvelopeBufferPool) + val (resourceLife, completed) = aeronSource(largeStreamId, largeEnvelopeBufferPool) .via(inboundLargeFlow(disableCompression)) - .toMat(inboundSink(largeEnvelopeBufferPool))(Keep.right) + .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 - updateStreamCompletion(streamName, streamCompleted.recover { case _ ⇒ Done }) streamCompleted.onFailure { case ShutdownSignal ⇒ // shutdown as expected case _: AeronTerminated ⇒ // shutdown already in progress @@ -814,12 +826,15 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } } - // set the future that completes when the current stream for a given name completes - @tailrec - private def updateStreamCompletion(streamName: String, streamCompleted: Future[Done]): Unit = { - val prev = streamCompletions.get() - if (!streamCompletions.compareAndSet(prev, prev + (streamName → streamCompleted))) { - updateStreamCompletion(streamName, streamCompleted) + 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) } } @@ -831,7 +846,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R implicit val ec = remoteDispatcher for { _ ← Future.traverse(associationRegistry.allAssociations)(_.streamsCompleted) - _ ← Future.sequence(streamCompletions.get().valuesIterator) + _ ← Future.sequence(streamMatValues.get().valuesIterator.map { + case InboundStreamMatValues(_, done) ⇒ done + }) } yield Done } @@ -961,7 +978,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R 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, NotUsed] = + def aeronSource(streamId: Int, pool: EnvelopeBufferPool): Source[EnvelopeBuffer, AeronSource.ResourceLifecycle] = Source.fromGraph(new AeronSource(inboundChannel, streamId, aeron, taskRunner, pool, createFlightRecorderEventSink())) @@ -1078,6 +1095,10 @@ private[remote] object ArteryTransport { // 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)) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index eaaa875f3f..11e0db8dad 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -92,6 +92,8 @@ private[remote] object Association { final val OrdinaryQueueIndex = 2 private object OutboundStreamStopSignal extends RuntimeException with NoStackTrace + + final case class OutboundStreamMatValues(streamKillSwitch: SharedKillSwitch, completed: Future[Done]) } /** @@ -163,7 +165,8 @@ private[remote] class Association( else Future.sequence(c.map(_.changeActorRefCompression(table))).map(_ ⇒ Done) timeoutAfter(result, changeCompressionTimeout, new ChangeOutboundCompressionFailed) } - private[this] val streamCompletions = new AtomicReference(Map.empty[String, (SharedKillSwitch, Future[Done])]) + // 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[artery] def changeClassManifestCompression(table: CompressionTable[String]): Future[Done] = { @@ -471,8 +474,8 @@ private[remote] class Association( cancelIdleTimer() idle.set(Some(transport.system.scheduler.scheduleOnce(advancedSettings.StopQuarantinedAfterIdle) { if (associationState.isQuarantined()) - streamCompletions.get.valuesIterator.foreach { - case (killSwitch, _) ⇒ killSwitch.abort(OutboundStreamStopSignal) + streamMatValues.get.valuesIterator.foreach { + case OutboundStreamMatValues(killSwitch, _) ⇒ killSwitch.abort(OutboundStreamStopSignal) } }(transport.system.dispatcher))) } @@ -524,8 +527,10 @@ private[remote] class Association( 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, - streamKillSwitch, completed, () ⇒ runOutboundControlStream()) + completed, () ⇒ runOutboundControlStream()) } private def getOrCreateQueueWrapper(queueIndex: Int, capacity: Int): QueueWrapper = { @@ -562,8 +567,9 @@ private[remote] class Association( queuesVisibility = true // volatile write for visibility of the queues array changeOutboundCompression = Vector(changeCompression) + updateStreamMatValues(OrdinaryQueueIndex, streamKillSwitch, completed) attachStreamRestart("Outbound message stream", OrdinaryQueueIndex, queueSize, - streamKillSwitch, completed, () ⇒ runOutboundOrdinaryMessagesStream()) + completed, () ⇒ runOutboundOrdinaryMessagesStream()) } else { log.debug("Starting outbound message stream to [{}] with [{}] lanes", remoteAddress, outboundLanes) @@ -617,7 +623,7 @@ private[remote] class Association( changeOutboundCompression = changeCompressionValues attachStreamRestart("Outbound message stream", OrdinaryQueueIndex, queueSize, - streamKillSwitch, completed, () ⇒ runOutboundOrdinaryMessagesStream()) + completed, () ⇒ runOutboundOrdinaryMessagesStream()) } } @@ -640,12 +646,14 @@ private[remote] class Association( // 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, - streamKillSwitch, completed, () ⇒ runOutboundLargeMessagesStream()) + completed, () ⇒ runOutboundLargeMessagesStream()) } private def attachStreamRestart(streamName: String, queueIndex: Int, queueCapacity: Int, - streamKillSwitch: SharedKillSwitch, streamCompleted: Future[Done], restart: () ⇒ Unit): Unit = { + streamCompleted: Future[Done], restart: () ⇒ Unit): Unit = { def lazyRestart(): Unit = { changeOutboundCompression = Vector.empty @@ -659,7 +667,6 @@ private[remote] class Association( } implicit val ec = materializer.executionContext - updateStreamCompletion(streamName, (streamKillSwitch, streamCompleted.recover { case _ ⇒ Done })) streamCompleted.onFailure { case ArteryTransport.ShutdownSignal ⇒ // shutdown as expected @@ -699,12 +706,15 @@ private[remote] class Association( } } - // set the future that completes when the current stream for a given name completes - @tailrec - private def updateStreamCompletion(streamName: String, streamCompletion: (SharedKillSwitch, Future[Done])): Unit = { - val prev = streamCompletions.get() - if (!streamCompletions.compareAndSet(prev, prev + (streamName → streamCompletion))) { - updateStreamCompletion(streamName, streamCompletion) + 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) } } @@ -714,7 +724,9 @@ private[remote] class Association( */ def streamsCompleted: Future[Done] = { implicit val ec = materializer.executionContext - Future.sequence(streamCompletions.get().values.map(_._2)).map(_ ⇒ Done) + Future.sequence(streamMatValues.get().values.map { + case OutboundStreamMatValues(_, done) ⇒ done + }).map(_ ⇒ Done) } override def toString: String = From 531d6cb749a993deb53621d88761b25080507541 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 14 Sep 2016 12:07:15 +0200 Subject: [PATCH 163/186] akka.remote.artery.canonical.port in sample adjust artery config --- .../src/main/resources/application.conf | 5 +++-- .../scala/sample/cluster/factorial/FactorialBackend.scala | 2 +- .../main/scala/sample/cluster/simple/SimpleClusterApp.scala | 2 +- .../src/main/scala/sample/cluster/stats/StatsSample.scala | 2 +- .../scala/sample/cluster/stats/StatsSampleOneMaster.scala | 2 +- .../cluster/transformation/TransformationBackend.scala | 2 +- .../cluster/transformation/TransformationFrontend.scala | 2 +- 7 files changed, 9 insertions(+), 8 deletions(-) diff --git a/akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf b/akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf index 9b2aabe28c..86559812a1 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf +++ b/akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf @@ -1,12 +1,13 @@ #//#snippet akka { + actor { provider = "cluster" } remote.artery { enabled = on - hostname = 127.0.0.1 - port = 0 + canonical.hostname = 127.0.0.1 + canonical.port = 0 } cluster { diff --git a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/FactorialBackend.scala b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/FactorialBackend.scala index 273b2eaa58..e6629d73a0 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/FactorialBackend.scala +++ b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/FactorialBackend.scala @@ -34,7 +34,7 @@ object FactorialBackend { def main(args: Array[String]): Unit = { // Override the configuration of the port when specified as program argument val port = if (args.isEmpty) "0" else args(0) - val config = ConfigFactory.parseString(s"akka.remote.artery.port=$port"). + val config = ConfigFactory.parseString(s"akka.remote.artery.canonical.port=$port"). withFallback(ConfigFactory.parseString("akka.cluster.roles = [backend]")). withFallback(ConfigFactory.load("factorial")) diff --git a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/simple/SimpleClusterApp.scala b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/simple/SimpleClusterApp.scala index 343e56d86f..1f22869117 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/simple/SimpleClusterApp.scala +++ b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/simple/SimpleClusterApp.scala @@ -15,7 +15,7 @@ object SimpleClusterApp { def startup(ports: Seq[String]): Unit = { ports foreach { port => // Override the configuration of the port - val config = ConfigFactory.parseString("akka.remote.artery.port=" + port). + val config = ConfigFactory.parseString("akka.remote.artery.canonical.port=" + port). withFallback(ConfigFactory.load()) // Create an Akka system diff --git a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSample.scala b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSample.scala index 99741f1403..019d046bac 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSample.scala +++ b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSample.scala @@ -28,7 +28,7 @@ object StatsSample { ports foreach { port => // Override the configuration of the port when specified as program argument val config = - ConfigFactory.parseString(s"akka.remote.artery.port=" + port).withFallback( + ConfigFactory.parseString(s"akka.remote.artery.canonical.port=" + port).withFallback( ConfigFactory.parseString("akka.cluster.roles = [compute]")). withFallback(ConfigFactory.load("stats1")) diff --git a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSampleOneMaster.scala b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSampleOneMaster.scala index 3962f968db..7cda2d89df 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSampleOneMaster.scala +++ b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSampleOneMaster.scala @@ -23,7 +23,7 @@ object StatsSampleOneMaster { ports foreach { port => // Override the configuration of the port when specified as program argument val config = - ConfigFactory.parseString(s"akka.remote.artery.port=" + port).withFallback( + ConfigFactory.parseString(s"akka.remote.artery.canonical.port=" + port).withFallback( ConfigFactory.parseString("akka.cluster.roles = [compute]")). withFallback(ConfigFactory.load("stats2")) diff --git a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationBackend.scala b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationBackend.scala index 6d8cbbcd59..d7893fad5f 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationBackend.scala +++ b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationBackend.scala @@ -42,7 +42,7 @@ object TransformationBackend { def main(args: Array[String]): Unit = { // Override the configuration of the port when specified as program argument val port = if (args.isEmpty) "0" else args(0) - val config = ConfigFactory.parseString(s"akka.remote.artery.port=$port"). + val config = ConfigFactory.parseString(s"akka.remote.artery.canonical.port=$port"). withFallback(ConfigFactory.parseString("akka.cluster.roles = [backend]")). withFallback(ConfigFactory.load()) diff --git a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationFrontend.scala b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationFrontend.scala index e6679cd7c6..e4225106d7 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationFrontend.scala +++ b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationFrontend.scala @@ -40,7 +40,7 @@ object TransformationFrontend { def main(args: Array[String]): Unit = { // Override the configuration of the port when specified as program argument val port = if (args.isEmpty) "0" else args(0) - val config = ConfigFactory.parseString(s"akka.remote.artery.port=$port"). + val config = ConfigFactory.parseString(s"akka.remote.artery.canonical.port=$port"). withFallback(ConfigFactory.parseString("akka.cluster.roles = [frontend]")). withFallback(ConfigFactory.load()) From 5161e4d40aac79eb91c77173c307c1914b02c16e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andre=CC=81n?= Date: Fri, 23 Sep 2016 15:00:01 +0200 Subject: [PATCH 164/186] Testconductor allows more time to pass to give artery time to shutdown #21500 --- akka-multi-node-testkit/src/main/resources/reference.conf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 From d91ddb78913bba9b5a4dc7ae37d124043da8ecb9 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 23 Sep 2016 15:50:32 +0200 Subject: [PATCH 165/186] increase acceptable-heartbeat-pause in StressSpec, #21522 --- akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 1cbe452bcf..72d3c2e1b3 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala @@ -127,7 +127,7 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig { akka.actor.serialize-creators = off 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 } From e9c4393f7b82516302cef8762217137e05fe9b74 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Fri, 23 Sep 2016 21:42:07 +0200 Subject: [PATCH 166/186] Flight recorder improvements (#21532) * Little runnable app to dump .afr files * Merge the three sources of events into one time sorted list * The summary isn't that useful, let's not show it. --- .../remote/artery/FlightRecorderReader.scala | 38 +++++++++++++------ 1 file changed, 26 insertions(+), 12 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderReader.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderReader.scala index 43173bc8c6..722755f39f 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderReader.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderReader.scala @@ -2,12 +2,23 @@ package akka.remote.artery import java.io.{ IOException, RandomAccessFile } import java.nio.channels.FileChannel -import java.nio.file.Path +import java.nio.file.{ FileSystems, Path } import java.time.Instant import org.agrona.concurrent.MappedResizeableBuffer -import scala.collection.immutable +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 @@ -72,16 +83,15 @@ private[akka] object FlightRecorderReader { raFile = new RandomAccessFile(flightRecorderFile.toFile, "rw") channel = raFile.getChannel reader = new FlightRecorderReader(channel) - println(reader.structure) + 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) - println("--- ALERT ENTRIES") - reader.structure.alertLog.logs.foreach(log ⇒ println(log.richEntries.mkString("\n"))) + implicit val ordering = Ordering.fromLessThan[FlightRecorderReader#Entry]((a, b) ⇒ a.timeStamp.isBefore(b.timeStamp)) + val sorted = SortedSet[FlightRecorderReader#Entry](alerts: _*) ++ hiFreq ++ loFreq - println("--- HI FREQUENCY ENTRIES") - reader.structure.hiFreqLog.logs.foreach(log ⇒ println(log.compactEntries.mkString("\n"))) - - println("--- LO FREQUENCY ENTRIES") - reader.structure.loFreqLog.logs.foreach(log ⇒ println(log.richEntries.mkString("\n"))) + println("--- FLIGHT RECORDER LOG") + sorted.foreach(println) } finally { if (reader ne null) reader.close() @@ -196,7 +206,11 @@ private[akka] final class FlightRecorderReader(fileChannel: FileChannel) { } } - case class RichEntry(timeStamp: Instant, dirty: Boolean, code: Long, metadata: Array[Byte]) { + 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") @@ -204,7 +218,7 @@ private[akka] final class FlightRecorderReader(fileChannel: FileChannel) { } } - case class CompactEntry(timeStamp: Instant, dirty: Boolean, code: Long, param: Long) { + 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" From 4cccca8d52064ef0a00b63689f00478da334359e Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 26 Sep 2016 12:32:54 +0200 Subject: [PATCH 167/186] Better to send msg to unresolved ref than dropping it, #21422 * for the remote deployed actors * e.g. watch of really dead actor * this doesn't solve the ticket, we will make a better solution later --- .../scala/akka/remote/artery/Codecs.scala | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index f2f90d51db..b59ac96d51 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -412,10 +412,21 @@ private[remote] class Decoder( // down other messages. val recipientActorRefPath = headerBuilder.recipientActorRefPath.get if (bannedRemoteDeployedActorRefs.contains(recipientActorRefPath)) { - log.debug( - "Dropping message for banned (terminated) remote deployed recipient [{}].", - recipientActorRefPath) - pull(in) + + 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, From 90c1f04b4422ad01ed0bc0a18a29ec0b66be4dd4 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 26 Sep 2016 14:00:00 +0200 Subject: [PATCH 168/186] complete the Future[Done] in AeronSink in finally, #21538 * might not be possible to close the publication if the media driver has crashed * important to always complete the future, otherwise the shutdown process will not complete --- .../main/scala/akka/remote/artery/AeronSink.scala | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala index 9d3e5a2fd3..d19ecf5c7a 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala @@ -122,12 +122,15 @@ class AeronSink( } override def postStop(): Unit = { - taskRunner.command(Remove(addOfferTask.task)) - flightRecorder.loFreq(AeronSink_TaskRunnerRemoved, channelMetadata) - pub.close() - flightRecorder.loFreq(AeronSink_PublicationClosed, channelMetadata) - completed.complete(completedValue) - flightRecorder.loFreq(AeronSink_Stopped, channelMetadata) + 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 From ae860115ac24912f644c3a6fddc0ce9ddbdd1db0 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 26 Sep 2016 14:04:15 +0200 Subject: [PATCH 169/186] harden shutdown exception in sendTerminationHint --- .../scala/akka/remote/artery/ArteryTransport.scala | 6 +++++- .../main/scala/akka/remote/artery/Association.scala | 12 ++++++++---- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 0d50d89305..73f57d3506 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -253,9 +253,13 @@ private[remote] class FlushOnShutdown(done: Promise[Done], timeout: FiniteDurati 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) ⇒ - // send may throw + // sendTerminationHint may throw done.tryFailure(e) throw e } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index eaaa875f3f..928cddc7bf 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -403,11 +403,15 @@ private[remote] class Association( val msg = ActorSystemTerminating(localAddress) var sent = 0 queues.iterator.filter(_.isEnabled).foreach { queue ⇒ - val envelope = outboundEnvelopePool.acquire() - .init(OptionVal.None, msg, OptionVal.Some(replyTo)) + try { + val envelope = outboundEnvelopePool.acquire() + .init(OptionVal.None, msg, OptionVal.Some(replyTo)) - queue.offer(envelope) - sent += 1 + queue.offer(envelope) + sent += 1 + } catch { + case ShuttingDown ⇒ // can be thrown if `offer` triggers new materialization + } } sent } else 0 From 8ae0c9a8888aa06e898b00ec16be13e335484f80 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Mon, 26 Sep 2016 15:34:59 +0200 Subject: [PATCH 170/186] Use long uid in artery remoting and cluster #20644 --- .../akka/remote/artery/CodecBenchmark.scala | 2 +- .../cluster/protobuf/msg/ClusterMessages.java | 129 +++++- .../src/main/protobuf/ClusterMessages.proto | 2 + .../src/main/scala/akka/cluster/Cluster.scala | 2 +- .../scala/akka/cluster/ClusterDaemon.scala | 2 +- .../akka/cluster/ClusterRemoteWatcher.scala | 2 +- .../src/main/scala/akka/cluster/Member.scala | 35 +- .../protobuf/ClusterMessageSerializer.scala | 23 +- .../scala/akka/cluster/RestartNode3Spec.scala | 2 +- .../scala/akka/cluster/RestartNodeSpec.scala | 2 +- .../protobuf/msg/ReplicatorMessages.java | 129 +++++- .../main/protobuf/ReplicatorMessages.proto | 2 + .../ddata/protobuf/SerializationSupport.scala | 15 +- .../PiercingShouldKeepQuarantineSpec.scala | 4 +- .../remote/RemoteQuarantinePiercingSpec.scala | 8 +- .../artery/HandshakeRestartReceiverSpec.scala | 6 +- .../RemoteRestartedQuarantinedSpec.scala | 6 +- .../akka/remote/ArteryControlFormats.java | 430 +++++++++++++++++- .../main/protobuf/ArteryControlFormats.proto | 7 + .../akka/remote/AddressUidExtension.scala | 23 +- .../akka/remote/RemoteActorRefProvider.scala | 2 +- .../scala/akka/remote/RemoteTransport.scala | 2 +- .../scala/akka/remote/RemoteWatcher.scala | 29 +- .../src/main/scala/akka/remote/Remoting.scala | 7 +- .../akka/remote/RemotingLifecycleEvent.scala | 27 +- .../akka/remote/artery/ArteryTransport.scala | 7 +- .../akka/remote/artery/Association.scala | 3 +- .../scala/akka/remote/artery/BufferPool.scala | 2 +- .../ArteryMessageSerializer.scala | 48 +- .../akka/remote/transport/AkkaPduCodec.scala | 2 +- .../scala/akka/remote/RemoteWatcherSpec.scala | 4 +- .../test/scala/akka/remote/RemotingSpec.scala | 2 +- .../remote/artery/RemoteWatcherSpec.scala | 46 +- .../artery/SystemMessageDeliverySpec.scala | 4 +- .../ArteryMessageSerializerSpec.scala | 7 +- project/AkkaBuild.scala | 52 +-- project/MiMa.scala | 8 +- 37 files changed, 932 insertions(+), 151 deletions(-) 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 index f0903d4925..6203e053e2 100644 --- a/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala +++ b/akka-bench-jmh/src/main/scala/akka/remote/artery/CodecBenchmark.scala @@ -64,7 +64,7 @@ class CodecBenchmark { val uniqueLocalAddress = UniqueAddress( system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress, - AddressUidExtension(system).addressUid + AddressUidExtension(system).longAddressUid ) val payload = Array.ofDim[Byte](1000) 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/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/ClusterRemoteWatcher.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala index 5dd1ed3da8..534d23e273 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala @@ -93,7 +93,7 @@ private[cluster] class ClusterRemoteWatcher( // The reason we don't quarantine gracefully removed members (leaving) is that // Cluster Singleton need to exchange TakeOver/HandOver messages. if (previousStatus == MemberStatus.Down) { - quarantine(m.address, Some(m.uniqueAddress.uid), s"Cluster member removed, previous status [$previousStatus]") + quarantine(m.address, Some(m.uniqueAddress.longUid), s"Cluster member removed, previous status [$previousStatus]") } publishAddressTerminated(m.address) } 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/RestartNode3Spec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNode3Spec.scala index 895b3fb5bf..01cd5f8e00 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNode3Spec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNode3Spec.scala @@ -137,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-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-remote-tests/src/multi-jvm/scala/akka/remote/PiercingShouldKeepQuarantineSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/PiercingShouldKeepQuarantineSpec.scala index 12efbba067..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 @@ -32,7 +32,7 @@ class ArteryPiercingShouldKeepQuarantineSpecMultiJvmNode2 extends PiercingShould object PiercingShouldKeepQuarantineSpec { class Subject extends Actor { def receive = { - case "getuid" ⇒ sender() ! AddressUidExtension(context.system).addressUid + case "getuid" ⇒ sender() ! AddressUidExtension(context.system).longAddressUid } } } @@ -52,7 +52,7 @@ abstract class PiercingShouldKeepQuarantineSpec(multiNodeConfig: PiercingShouldK // 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 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 401a3bf104..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 @@ -45,7 +45,7 @@ object RemoteQuarantinePiercingSpec { class Subject extends Actor { def receive = { case "shutdown" ⇒ context.system.terminate() - case "identify" ⇒ sender() ! (AddressUidExtension(context.system).addressUid → self) + case "identify" ⇒ sender() ! (AddressUidExtension(context.system).longAddressUid → self) } } } @@ -57,10 +57,10 @@ abstract class RemoteQuarantinePiercingSpec(multiNodeConfig: RemoteQuarantinePie override def initialParticipants = roles.size - def identifyWithUid(role: RoleName, actorName: String, timeout: FiniteDuration = remainingOrDefault): (Int, ActorRef) = { + def identifyWithUid(role: RoleName, actorName: String, timeout: FiniteDuration = remainingOrDefault): (Long, ActorRef) = { within(timeout) { system.actorSelection(node(role) / "user" / actorName) ! "identify" - expectMsgType[(Int, ActorRef)] + expectMsgType[(Long, ActorRef)] } } @@ -90,7 +90,7 @@ abstract class RemoteQuarantinePiercingSpec(multiNodeConfig: RemoteQuarantinePie // 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) } 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 index 6e5ec4d812..a48cf210c6 100644 --- 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 @@ -34,7 +34,7 @@ object HandshakeRestartReceiverSpec extends MultiNodeConfig { class Subject extends Actor { def receive = { case "shutdown" ⇒ context.system.terminate() - case "identify" ⇒ sender() ! (AddressUidExtension(context.system).addressUid → self) + case "identify" ⇒ sender() ! (AddressUidExtension(context.system).longAddressUid → self) } } @@ -55,10 +55,10 @@ abstract class HandshakeRestartReceiverSpec super.afterAll() } - def identifyWithUid(rootPath: ActorPath, actorName: String, timeout: FiniteDuration = remainingOrDefault): (Int, ActorRef) = { + def identifyWithUid(rootPath: ActorPath, actorName: String, timeout: FiniteDuration = remainingOrDefault): (Long, ActorRef) = { within(timeout) { system.actorSelection(rootPath / "user" / actorName) ! "identify" - expectMsgType[(Int, ActorRef)] + expectMsgType[(Long, ActorRef)] } } 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 index feb28d84a4..34d78ba270 100644 --- 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 @@ -36,7 +36,7 @@ object RemoteRestartedQuarantinedSpec extends MultiNodeConfig { class Subject extends Actor { def receive = { case "shutdown" ⇒ context.system.terminate() - case "identify" ⇒ sender() ! (AddressUidExtension(context.system).addressUid → self) + case "identify" ⇒ sender() ! (AddressUidExtension(context.system).longAddressUid → self) } } @@ -51,10 +51,10 @@ abstract class RemoteRestartedQuarantinedSpec extends RemotingMultiNodeSpec(Remo override def initialParticipants = 2 - def identifyWithUid(role: RoleName, actorName: String, timeout: FiniteDuration = remainingOrDefault): (Int, ActorRef) = { + def identifyWithUid(role: RoleName, actorName: String, timeout: FiniteDuration = remainingOrDefault): (Long, ActorRef) = { within(timeout) { system.actorSelection(node(role) / "user" / actorName) ! "identify" - expectMsgType[(Int, ActorRef)] + expectMsgType[(Long, ActorRef)] } } diff --git a/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java b/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java index 22deb5c3cd..f1ed38b3a9 100644 --- a/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java +++ b/akka-remote/src/main/java/akka/remote/ArteryControlFormats.java @@ -6821,6 +6821,420 @@ public final class ArteryControlFormats { // @@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 @@ -6866,6 +7280,11 @@ public final class ArteryControlFormats { 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() { @@ -6894,8 +7313,9 @@ public final class ArteryControlFormats { "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(\004B\017\n\013akka.r", - "emoteH\001" + "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() { @@ -6956,6 +7376,12 @@ public final class ArteryControlFormats { 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; } }; diff --git a/akka-remote/src/main/protobuf/ArteryControlFormats.proto b/akka-remote/src/main/protobuf/ArteryControlFormats.proto index d71d66fe63..8970092b9f 100644 --- a/akka-remote/src/main/protobuf/ArteryControlFormats.proto +++ b/akka-remote/src/main/protobuf/ArteryControlFormats.proto @@ -78,3 +78,10 @@ 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/scala/akka/remote/AddressUidExtension.scala b/akka-remote/src/main/scala/akka/remote/AddressUidExtension.scala index e83d6f7f42..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,15 +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 { + + private def arteryEnabled = system.provider.asInstanceOf[RemoteActorRefProvider].remoteSettings.Artery.Enabled + val longAddressUid: Long = { - // FIXME we should use a long here, but then we need to change in Cluster and RemoteWatcher also - //ThreadLocalRandom.current.nextLong() - ThreadLocalRandom.current.nextInt() + 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") - val addressUid: Int = longAddressUid.toInt + 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/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 28ced7a91b..187f9f1389 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -455,7 +455,7 @@ private[akka] class RemoteActorRefProvider( * @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], reason: String): Unit = + def quarantine(address: Address, uid: Option[Long], reason: String): Unit = transport.quarantine(address, uid, reason) } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala index c8abdabcba..c3cc61cf5d 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala @@ -90,6 +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], reason: String): Unit + 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 33c8730f40..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._ @@ -34,6 +35,10 @@ private[akka] object RemoteWatcher { @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 case object ReapUnreachableTick @@ -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)) @@ -167,7 +172,7 @@ private[akka] class RemoteWatcher( def publishAddressTerminated(address: Address): Unit = AddressTerminatedTopic(context.system).publish(AddressTerminated(address)) - def quarantine(address: Address, uid: Option[Int], reason: String): Unit = + def quarantine(address: Address, uid: Option[Long], reason: String): Unit = remoteProvider.quarantine(address, uid, reason) def addWatch(watchee: InternalActorRef, watcher: InternalActorRef): Unit = { @@ -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 } } diff --git a/akka-remote/src/main/scala/akka/remote/Remoting.scala b/akka-remote/src/main/scala/akka/remote/Remoting.scala index 415925a42f..127c04ac9b 100644 --- a/akka-remote/src/main/scala/akka/remote/Remoting.scala +++ b/akka-remote/src/main/scala/akka/remote/Remoting.scala @@ -230,10 +230,11 @@ 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], reason: String): 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) } private[akka] def boundAddresses: Map[String, Set[Address]] = { 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/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 0d50d89305..b1da8af71b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -431,7 +431,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R runInboundStreams() topLevelFREvents.loFreq(Transport_StartupFinished, NoMetaData) - log.info("Remoting started; listening on address: [{}] with uid [{}]", localAddress.address, localAddress.uid) + log.info("Remoting started; listening on address: [{}] with UID [{}]", localAddress.address, localAddress.uid) } private lazy val shutdownHook = new Thread { @@ -891,10 +891,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R } } - override def quarantine(remoteAddress: Address, uid: Option[Int], reason: String): Unit = { + override def quarantine(remoteAddress: Address, uid: Option[Long], reason: String): Unit = { try { - // FIXME use Long uid - association(remoteAddress).quarantine(reason, uid.map(_.toLong)) + association(remoteAddress).quarantine(reason, uid) } catch { case ShuttingDown ⇒ // silence it } diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index eaaa875f3f..1da210f872 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -434,8 +434,7 @@ private[remote] class Association( "messages to this UID will be delivered to dead letters. " + "Remote actorsystem must be restarted to recover from this situation. {}", remoteAddress, u, reason) - // FIXME when we complete the switch to Long UID we must use Long here also, issue #20644 - transport.system.eventStream.publish(QuarantinedEvent(remoteAddress, u.toInt)) + transport.system.eventStream.publish(QuarantinedEvent(remoteAddress, u)) clearOutboundCompression() clearInboundCompression(u) // end delivery of system messages to that incarnation after this point diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index 0c21194e43..06fed362fe 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -356,7 +356,7 @@ private[remote] final class HeaderBuilderImpl( "HeaderBuilderImpl(" + "version:" + version + ", " + "flags:" + ByteFlag.binaryLeftPad(flags) + ", " + - "uid:" + uid + ", " + + "UID:" + uid + ", " + "_senderActorRef:" + _senderActorRef + ", " + "_senderActorRefIdx:" + _senderActorRefIdx + ", " + "_recipientActorRef:" + _recipientActorRef + ", " + diff --git a/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala index ee2694a3ac..8e31f0ffdf 100644 --- a/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala @@ -5,11 +5,12 @@ 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.{ ArteryControlFormats, MessageSerializer, UniqueAddress, WireFormats } +import akka.remote._ import akka.serialization.{ BaseSerializer, Serialization, SerializationExtension, SerializerWithStringManifest } /** INTERNAL API */ @@ -27,6 +28,9 @@ private[akka] object ArteryMessageSerializer { private val SystemMessageDeliveryAckManifest = "k" private val SystemMessageDeliveryNackManifest = "l" + private val ArteryHeartbeatManifest = "m" + private val ArteryHeartbeatRspManifest = "n" + private final val DeadLettersRepresentation = "" } @@ -41,6 +45,8 @@ private[akka] final class ArteryMessageSerializer(val system: ExtendedActorSyste 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 @@ -53,20 +59,22 @@ private[akka] final class ArteryMessageSerializer(val system: ExtendedActorSyste 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) - case SystemMessageDelivery.Ack(seqNo, from) ⇒ serializeSystemMessageDeliveryAck(seqNo, from) - case HandshakeReq(from, to) ⇒ serializeHandshakeReq(from, to) - case HandshakeRsp(from) ⇒ serializeWithAddress(from) - case SystemMessageDelivery.Nack(seqNo, from) ⇒ serializeSystemMessageDeliveryAck(seqNo, from) - case q: Quarantined ⇒ serializeQuarantined(q) - case ActorSystemTerminating(from) ⇒ serializeWithAddress(from) - case ActorSystemTerminatingAck(from) ⇒ serializeWithAddress(from) - case adv: ActorRefCompressionAdvertisement ⇒ serializeActorRefCompressionAdvertisement(adv) - case ActorRefCompressionAdvertisementAck(from, id) ⇒ serializeCompressionTableAdvertisementAck(from, id) - case adv: ClassManifestCompressionAdvertisement ⇒ serializeCompressionAdvertisement(adv)(identity) - case ClassManifestCompressionAdvertisementAck(from, id) ⇒ serializeCompressionTableAdvertisementAck(from, id) - }).toByteArray + 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) @@ -81,6 +89,8 @@ private[akka] final class ArteryMessageSerializer(val system: ExtendedActorSyste 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)") } @@ -226,4 +236,12 @@ private[akka] final class ArteryMessageSerializer(val system: ExtendedActorSyste 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/transport/AkkaPduCodec.scala b/akka-remote/src/main/scala/akka/remote/transport/AkkaPduCodec.scala index 4777778a97..6a209512f8 100644 --- a/akka-remote/src/main/scala/akka/remote/transport/AkkaPduCodec.scala +++ b/akka-remote/src/main/scala/akka/remote/transport/AkkaPduCodec.scala @@ -151,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)) } diff --git a/akka-remote/src/test/scala/akka/remote/RemoteWatcherSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteWatcherSpec.scala index 56ff462e8e..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], reason: String): 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)) } diff --git a/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala b/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala index 701cc5c670..252a76299e 100644 --- a/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala @@ -773,7 +773,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D inboundHandleProbe.expectNoMsg(1.second) // Quarantine the connection - RARP(thisSystem).provider.quarantine(remoteAddress, Some(remoteUID), "test") + 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/artery/RemoteWatcherSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala index 97a8e22872..dc236b272f 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteWatcherSpec.scala @@ -38,7 +38,7 @@ object RemoteWatcherSpec { object TestRemoteWatcher { final case class AddressTerm(address: Address) extends JavaSerializable - final case class Quarantined(address: Address, uid: Option[Int]) extends JavaSerializable + final case class Quarantined(address: Address, uid: Option[Long]) extends JavaSerializable } class TestRemoteWatcher(heartbeatExpectedResponseAfter: FiniteDuration) extends RemoteWatcher( @@ -54,7 +54,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], reason: String): 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)) } @@ -80,7 +80,7 @@ class RemoteWatcherSpec extends AkkaSpec( val remoteSystem = ActorSystem("RemoteSystem", system.settings.config) val remoteAddress = RARP(remoteSystem).provider.getDefaultAddress - def remoteAddressUid = AddressUidExtension(remoteSystem).addressUid + def remoteAddressUid = AddressUidExtension(remoteSystem).longAddressUid Seq(system, remoteSystem).foreach(muteDeadLetters( akka.remote.transport.AssociationHandle.Disassociated.getClass, @@ -90,7 +90,7 @@ class RemoteWatcherSpec extends AkkaSpec( shutdown(remoteSystem) } - val heartbeatRspB = HeartbeatRsp(remoteAddressUid) + val heartbeatRspB = ArteryHeartbeatRsp(remoteAddressUid) def createRemoteActor(props: Props, name: String): InternalActorRef = { remoteSystem.actorOf(props, name) @@ -119,14 +119,14 @@ class RemoteWatcherSpec extends AkkaSpec( expectMsg(Stats.counts(watching = 3, watchingNodes = 1)) expectNoMsg(100 millis) monitorA ! HeartbeatTick - expectMsg(Heartbeat) + expectMsg(ArteryHeartbeat) expectNoMsg(100 millis) monitorA ! HeartbeatTick - expectMsg(Heartbeat) + expectMsg(ArteryHeartbeat) expectNoMsg(100 millis) monitorA.tell(heartbeatRspB, monitorB) monitorA ! HeartbeatTick - expectMsg(Heartbeat) + expectMsg(ArteryHeartbeat) expectNoMsg(100 millis) monitorA ! UnwatchRemote(b1, a1) @@ -135,7 +135,7 @@ class RemoteWatcherSpec extends AkkaSpec( expectMsg(Stats.counts(watching = 2, watchingNodes = 1)) expectNoMsg(100 millis) monitorA ! HeartbeatTick - expectMsg(Heartbeat) + expectMsg(ArteryHeartbeat) expectNoMsg(100 millis) monitorA ! UnwatchRemote(b2, a2) @@ -144,7 +144,7 @@ class RemoteWatcherSpec extends AkkaSpec( expectMsg(Stats.counts(watching = 1, watchingNodes = 1)) expectNoMsg(100 millis) monitorA ! HeartbeatTick - expectMsg(Heartbeat) + expectMsg(ArteryHeartbeat) expectNoMsg(100 millis) monitorA ! UnwatchRemote(b2, a1) @@ -176,17 +176,17 @@ class RemoteWatcherSpec extends AkkaSpec( monitorA ! WatchRemote(b, a) monitorA ! HeartbeatTick - expectMsg(Heartbeat) + expectMsg(ArteryHeartbeat) monitorA.tell(heartbeatRspB, monitorB) expectNoMsg(1 second) monitorA ! HeartbeatTick - expectMsg(Heartbeat) + expectMsg(ArteryHeartbeat) monitorA.tell(heartbeatRspB, monitorB) within(10 seconds) { awaitAssert { monitorA ! HeartbeatTick - expectMsg(Heartbeat) + expectMsg(ArteryHeartbeat) // but no HeartbeatRsp monitorA ! ReapUnreachableTick p.expectMsg(1 second, TestRemoteWatcher.AddressTerm(b.path.address)) @@ -215,13 +215,13 @@ class RemoteWatcherSpec extends AkkaSpec( monitorA ! WatchRemote(b, a) monitorA ! HeartbeatTick - expectMsg(Heartbeat) + expectMsg(ArteryHeartbeat) // no HeartbeatRsp sent within(20 seconds) { awaitAssert { monitorA ! HeartbeatTick - expectMsg(Heartbeat) + expectMsg(ArteryHeartbeat) // but no HeartbeatRsp monitorA ! ReapUnreachableTick p.expectMsg(1 second, TestRemoteWatcher.AddressTerm(b.path.address)) @@ -249,17 +249,17 @@ class RemoteWatcherSpec extends AkkaSpec( monitorA ! WatchRemote(b, a) monitorA ! HeartbeatTick - expectMsg(Heartbeat) + expectMsg(ArteryHeartbeat) monitorA.tell(heartbeatRspB, monitorB) expectNoMsg(1 second) monitorA ! HeartbeatTick - expectMsg(Heartbeat) + expectMsg(ArteryHeartbeat) monitorA.tell(heartbeatRspB, monitorB) within(10 seconds) { awaitAssert { monitorA ! HeartbeatTick - expectMsg(Heartbeat) + expectMsg(ArteryHeartbeat) // but no HeartbeatRsp monitorA ! ReapUnreachableTick p.expectMsg(1 second, TestRemoteWatcher.AddressTerm(b.path.address)) @@ -281,21 +281,21 @@ class RemoteWatcherSpec extends AkkaSpec( monitorA ! WatchRemote(c, a) monitorA ! HeartbeatTick - expectMsg(Heartbeat) + expectMsg(ArteryHeartbeat) monitorA.tell(heartbeatRspB, monitorB) expectNoMsg(1 second) monitorA ! HeartbeatTick - expectMsg(Heartbeat) + expectMsg(ArteryHeartbeat) monitorA.tell(heartbeatRspB, monitorB) monitorA ! HeartbeatTick - expectMsg(Heartbeat) + expectMsg(ArteryHeartbeat) monitorA ! ReapUnreachableTick p.expectNoMsg(1 second) monitorA ! HeartbeatTick - expectMsg(Heartbeat) + expectMsg(ArteryHeartbeat) monitorA.tell(heartbeatRspB, monitorB) monitorA ! HeartbeatTick - expectMsg(Heartbeat) + expectMsg(ArteryHeartbeat) monitorA ! ReapUnreachableTick p.expectNoMsg(1 second) q.expectNoMsg(1 second) @@ -304,7 +304,7 @@ class RemoteWatcherSpec extends AkkaSpec( within(10 seconds) { awaitAssert { monitorA ! HeartbeatTick - expectMsg(Heartbeat) + expectMsg(ArteryHeartbeat) // but no HeartbeatRsp monitorA ! ReapUnreachableTick p.expectMsg(1 second, TestRemoteWatcher.AddressTerm(c.path.address)) diff --git a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala index cc7ed61608..a702272148 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/SystemMessageDeliverySpec.scala @@ -52,11 +52,11 @@ class SystemMessageDeliverySpec extends AkkaSpec(SystemMessageDeliverySpec.confi val addressA = UniqueAddress( RARP(system).provider.getDefaultAddress, - AddressUidExtension(system).addressUid) + AddressUidExtension(system).longAddressUid) val systemB = ActorSystem("systemB", system.settings.config) val addressB = UniqueAddress( RARP(systemB).provider.getDefaultAddress, - AddressUidExtension(systemB).addressUid) + AddressUidExtension(systemB).longAddressUid) val rootB = RootActorPath(addressB.address) val matSettings = ActorMaterializerSettings(system).withFuzzing(true) implicit val mat = ActorMaterializer(matSettings)(system) diff --git a/akka-remote/src/test/scala/akka/remote/serialization/ArteryMessageSerializerSpec.scala b/akka-remote/src/test/scala/akka/remote/serialization/ArteryMessageSerializerSpec.scala index bdf5333632..cf7d409c47 100644 --- a/akka-remote/src/test/scala/akka/remote/serialization/ArteryMessageSerializerSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/serialization/ArteryMessageSerializerSpec.scala @@ -5,7 +5,7 @@ package akka.remote.serialization import akka.actor._ -import akka.remote.UniqueAddress +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 @@ -30,7 +30,10 @@ class ArteryMessageSerializerSpec extends AkkaSpec { "ClassManifestCompressionAdvertisementAck" → ClassManifestCompressionAdvertisementAck(uniqueAddress(), 23), "SystemMessageDelivery.SystemMessageEnvelop" → SystemMessageDelivery.SystemMessageEnvelope("test", 1234567890123L, uniqueAddress()), "SystemMessageDelivery.Ack" → SystemMessageDelivery.Ack(98765432109876L, uniqueAddress()), - "SystemMessageDelivery.Nack" → SystemMessageDelivery.Nack(98765432109876L, uniqueAddress())).foreach { + "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) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 220bdbdd06..0b4bcfad89 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -168,19 +168,19 @@ object AkkaBuild extends Build { id = "akka-cluster", base = file("akka-cluster"), dependencies = Seq(remote, remoteTests % "test->test" , testkit % "test->test") - ).configs(MultiJvm).disablePlugins(ValidatePullRequest, MimaPlugin) + ).configs(MultiJvm).disablePlugins(ValidatePullRequest) lazy val clusterMetrics = Project( id = "akka-cluster-metrics", base = file("akka-cluster-metrics"), dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm", slf4j % "test->compile") - ).configs(MultiJvm).disablePlugins(ValidatePullRequest, MimaPlugin) + ).configs(MultiJvm).disablePlugins(ValidatePullRequest) lazy val clusterTools = Project( id = "akka-cluster-tools", base = file("akka-cluster-tools"), dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm") - ).configs(MultiJvm).disablePlugins(ValidatePullRequest, MimaPlugin) + ).configs(MultiJvm).disablePlugins(ValidatePullRequest) lazy val clusterSharding = Project( id = "akka-cluster-sharding", @@ -191,13 +191,13 @@ object AkkaBuild extends Build { // provided. dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm", persistence % "compile;test->provided", distributedData % "provided;test", clusterTools) - ).configs(MultiJvm).disablePlugins(ValidatePullRequest, MimaPlugin) + ).configs(MultiJvm).disablePlugins(ValidatePullRequest) lazy val distributedData = Project( id = "akka-distributed-data-experimental", base = file("akka-distributed-data"), dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm") - ).configs(MultiJvm).disablePlugins(ValidatePullRequest, MimaPlugin) + ).configs(MultiJvm).disablePlugins(ValidatePullRequest) lazy val slf4j = Project( id = "akka-slf4j", @@ -215,7 +215,7 @@ object AkkaBuild extends Build { id = "akka-persistence", base = file("akka-persistence"), dependencies = Seq(actor, testkit % "test->test", protobuf) - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ).disablePlugins(ValidatePullRequest) lazy val persistenceQuery = Project( id = "akka-persistence-query-experimental", @@ -225,37 +225,37 @@ object AkkaBuild extends Build { persistence % "compile;provided->provided;test->test", testkit % "compile;test->test", streamTestkit % "compile;test->test") - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ).disablePlugins(ValidatePullRequest) lazy val persistenceTck = Project( id = "akka-persistence-tck", base = file("akka-persistence-tck"), dependencies = Seq(persistence % "compile;provided->provided;test->test", testkit % "compile;test->test") - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ).disablePlugins(ValidatePullRequest) lazy val persistenceShared = Project( id = "akka-persistence-shared", base = file("akka-persistence-shared"), dependencies = Seq(persistence % "test->test", testkit % "test->test", remote % "test", protobuf) - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ).disablePlugins(ValidatePullRequest) lazy val httpCore = Project( id = "akka-http-core", base = file("akka-http-core"), dependencies = Seq(stream, parsing, streamTestkit % "test->test") - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ).disablePlugins(ValidatePullRequest) lazy val http = Project( id = "akka-http-experimental", base = file("akka-http"), dependencies = Seq(httpCore) - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ).disablePlugins(ValidatePullRequest) lazy val httpTestkit = Project( id = "akka-http-testkit", base = file("akka-http-testkit"), dependencies = Seq(http, streamTestkit) - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ).disablePlugins(ValidatePullRequest) lazy val httpTests = Project( id = "akka-http-tests", @@ -263,12 +263,12 @@ object AkkaBuild extends Build { dependencies = Seq( httpTestkit % "test", streamTestkit % "test->test", testkit % "test->test", httpSprayJson, httpXml, httpJackson, multiNodeTestkit, remoteTests % "test->test") // required for multi-node latency/throughput Spec - ).configs(MultiJvm).disablePlugins(ValidatePullRequest, MimaPlugin) + ).configs(MultiJvm).disablePlugins(ValidatePullRequest) lazy val httpMarshallersScala = Project( id = "akka-http-marshallers-scala-experimental", base = file("akka-http-marshallers-scala") - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ).disablePlugins(ValidatePullRequest) .settings(parentSettings: _*) .aggregate(httpSprayJson, httpXml) @@ -281,7 +281,7 @@ object AkkaBuild extends Build { lazy val httpMarshallersJava = Project( id = "akka-http-marshallers-java-experimental", base = file("akka-http-marshallers-java") - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ).disablePlugins(ValidatePullRequest) .settings(parentSettings: _*) .aggregate(httpJackson) @@ -293,61 +293,61 @@ object AkkaBuild extends Build { id = s"akka-http-$name-experimental", base = file(s"akka-http-marshallers-scala/akka-http-$name"), dependencies = Seq(http) - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ).disablePlugins(ValidatePullRequest) def httpMarshallersJavaSubproject(name: String) = Project( id = s"akka-http-$name-experimental", base = file(s"akka-http-marshallers-java/akka-http-$name"), dependencies = Seq(http) - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ).disablePlugins(ValidatePullRequest) lazy val parsing = Project( id = "akka-parsing", base = file("akka-parsing") - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ).disablePlugins(ValidatePullRequest) lazy val stream = Project( id = "akka-stream", base = file("akka-stream"), dependencies = Seq(actor) - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ).disablePlugins(ValidatePullRequest) lazy val streamTestkit = Project( id = "akka-stream-testkit", base = file("akka-stream-testkit"), dependencies = Seq(stream, testkit % "compile;test->test") - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ).disablePlugins(ValidatePullRequest) lazy val streamTests = Project( id = "akka-stream-tests", base = file("akka-stream-tests"), dependencies = Seq(streamTestkit % "test->test", stream) - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ).disablePlugins(ValidatePullRequest) lazy val streamTestsTck = Project( id = "akka-stream-tests-tck", base = file("akka-stream-tests-tck"), dependencies = Seq(streamTestkit % "test->test", stream) - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ).disablePlugins(ValidatePullRequest) lazy val kernel = Project( id = "akka-kernel", base = file("akka-kernel"), dependencies = Seq(actor, testkit % "test->test") - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ).disablePlugins(ValidatePullRequest) lazy val camel = Project( id = "akka-camel", base = file("akka-camel"), dependencies = Seq(actor, slf4j, testkit % "test->test") - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ).disablePlugins(ValidatePullRequest) lazy val osgi = Project( id = "akka-osgi", base = file("akka-osgi"), dependencies = Seq(actor) - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ).disablePlugins(ValidatePullRequest) lazy val docs = Project( id = "akka-docs", @@ -368,7 +368,7 @@ object AkkaBuild extends Build { id = "akka-contrib", base = file("akka-contrib"), dependencies = Seq(remote, remoteTests % "test->test", cluster, clusterTools, persistence % "compile;test->provided") - ).configs(MultiJvm).disablePlugins(ValidatePullRequest, MimaPlugin) + ).configs(MultiJvm).disablePlugins(ValidatePullRequest) lazy val samplesSettings = parentSettings ++ ActivatorDist.settings diff --git a/project/MiMa.scala b/project/MiMa.scala index 3d896c3d31..59c076e4c3 100644 --- a/project/MiMa.scala +++ b/project/MiMa.scala @@ -981,7 +981,13 @@ object MiMa extends AutoPlugin { ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.http.scaladsl.model.ws.TextMessage.asScala"), ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.http.scaladsl.model.ws.TextMessage.getStreamedText"), ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.http.scaladsl.model.ws.BinaryMessage.asScala"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.http.scaladsl.model.ws.BinaryMessage.getStreamedData") + ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.http.scaladsl.model.ws.BinaryMessage.getStreamedData"), + + // #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[IncompatibleMethTypeProblem]("akka.remote.RemoteWatcher.receiveHeartbeatRsp"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.remote.RemoteWatcher.selfHeartbeatRspMsg") ) ) } From b7475b50f24c0130c59adc8ffcea8b9481cb2841 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 26 Sep 2016 15:16:26 +0200 Subject: [PATCH 171/186] increase timeout for fist message in RemoteSendConsistencySpec, #21529 --- .../scala/akka/remote/artery/RemoteSendConsistencySpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala index fd874fc152..2b6100be6a 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteSendConsistencySpec.scala @@ -54,7 +54,7 @@ abstract class AbstractRemoteSendConsistencySpec(config: Config) extends AkkaSpe val remoteRef = { system.actorSelection(rootB / "user" / "echo") ! Identify(None) - expectMsgType[ActorIdentity].ref.get + expectMsgType[ActorIdentity](5.seconds).ref.get } remoteRef ! "ping" From 8cbeb662d980f25d3a6a33422c32f53cf263afa9 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 27 Sep 2016 13:03:07 +0200 Subject: [PATCH 172/186] multi lanes not supported yet --- akka-remote/src/main/resources/reference.conf | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 447efb3c8b..721d1bad78 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -202,12 +202,16 @@ akka { # 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 From af7010766701a86cc403b183a95a9f2514faa0d5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Martynas=20Mickevi=C4=8Dius?= Date: Wed, 21 Sep 2016 10:22:08 +0300 Subject: [PATCH 173/186] #20872 Change tableVersion to Byte --- .../akka/remote/artery/ArteryTransport.scala | 5 +- .../scala/akka/remote/artery/BufferPool.scala | 16 ++-- .../artery/compress/CompressionProtocol.scala | 5 +- .../artery/compress/CompressionTable.scala | 2 +- .../artery/compress/DecompressionTable.scala | 4 +- .../artery/compress/InboundCompressions.scala | 50 ++++++------ .../ArteryMessageSerializer.scala | 6 +- .../remote/artery/EnvelopeBufferSpec.scala | 17 ++-- .../compress/CompressionIntegrationSpec.scala | 81 +++++++++++++++---- 9 files changed, 118 insertions(+), 68 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index b1da8af71b..06677b9316 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -13,11 +13,10 @@ import java.util.concurrent.atomic.{ AtomicLong, AtomicReference } import java.util.concurrent.atomic.AtomicBoolean import scala.annotation.tailrec -import scala.concurrent.{ Await, ExecutionContext, Future, Promise } +import scala.concurrent.{ Await, Future, Promise } import scala.concurrent.duration._ import scala.util.Failure import scala.util.Success -import scala.util.Try import scala.util.control.NoStackTrace import scala.util.control.NonFatal import akka.Done @@ -32,7 +31,6 @@ import akka.remote.AddressUidExtension import akka.remote.RemoteActorRef import akka.remote.RemoteActorRefProvider import akka.remote.RemoteTransport -import akka.remote.RemotingLifecycleEvent import akka.remote.ThisActorSystemQuarantinedEvent import akka.remote.UniqueAddress import akka.remote.artery.ArteryTransport.ShuttingDown @@ -48,7 +46,6 @@ 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.ActorAttributes.Dispatcher import akka.stream.ActorMaterializer import akka.stream.KillSwitches import akka.stream.Materializer diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index 06fed362fe..159bf29748 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -120,8 +120,8 @@ private[remote] sealed trait HeaderBuilder { def flag(byteFlag: ByteFlag): Boolean def setFlag(byteFlag: ByteFlag, value: Boolean): Unit - def inboundActorRefCompressionTableVersion: Int - def inboundClassManifestCompressionTableVersion: Int + def inboundActorRefCompressionTableVersion: Byte + def inboundClassManifestCompressionTableVersion: Byte def useOutboundCompression(on: Boolean): Unit @@ -214,8 +214,8 @@ private[remote] final class HeaderBuilderImpl( var _version: Byte = 0 var _flags: Byte = 0 var _uid: Long = 0 - var _inboundActorRefCompressionTableVersion: Int = 0 - var _inboundClassManifestCompressionTableVersion: Int = 0 + var _inboundActorRefCompressionTableVersion: Byte = 0 + var _inboundClassManifestCompressionTableVersion: Byte = 0 var _useOutboundCompression: Boolean = true var _senderActorRef: String = null @@ -254,8 +254,8 @@ private[remote] final class HeaderBuilderImpl( override def setUid(uid: Long) = _uid = uid override def uid: Long = _uid - override def inboundActorRefCompressionTableVersion: Int = _inboundActorRefCompressionTableVersion - override def inboundClassManifestCompressionTableVersion: Int = _inboundClassManifestCompressionTableVersion + override def inboundActorRefCompressionTableVersion: Byte = _inboundActorRefCompressionTableVersion + override def inboundClassManifestCompressionTableVersion: Byte = _inboundClassManifestCompressionTableVersion def useOutboundCompression(on: Boolean): Unit = _useOutboundCompression = on @@ -442,11 +442,11 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { // compression table versions (stored in the Tag) val refCompressionVersionTag = byteBuffer.getInt(ActorRefCompressionTableVersionTagOffset) if ((refCompressionVersionTag & TagTypeMask) != 0) { - header._inboundActorRefCompressionTableVersion = refCompressionVersionTag & TagValueMask + header._inboundActorRefCompressionTableVersion = (refCompressionVersionTag & TagValueMask).byteValue } val manifestCompressionVersionTag = byteBuffer.getInt(ClassManifestCompressionTableVersionTagOffset) if ((manifestCompressionVersionTag & TagTypeMask) != 0) { - header._inboundClassManifestCompressionTableVersion = manifestCompressionVersionTag & TagValueMask + header._inboundClassManifestCompressionTableVersion = (manifestCompressionVersionTag & TagValueMask).byteValue } if (header.flag(MetadataPresentFlag)) { 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 index 21e928b8f8..3c8fc463a6 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionProtocol.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionProtocol.scala @@ -4,7 +4,6 @@ package akka.remote.artery.compress -import scala.language.existentials import akka.actor.ActorRef import akka.remote.UniqueAddress import akka.remote.artery.ControlMessage @@ -36,7 +35,7 @@ object CompressionProtocol { * 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: Int) + private[remote] final case class ActorRefCompressionAdvertisementAck(from: UniqueAddress, tableVersion: Byte) extends ControlMessage with CompressionMessage /** @@ -53,7 +52,7 @@ object CompressionProtocol { * 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: Int) + private[remote] final case class ClassManifestCompressionAdvertisementAck(from: UniqueAddress, tableVersion: Byte) extends ControlMessage with CompressionMessage /** INTERNAL API */ 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 index 106b733c16..06da2785e3 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala @@ -8,7 +8,7 @@ import java.util import java.util.Comparator /** INTERNAL API: Versioned compression table to be advertised between systems */ -private[remote] final case class CompressionTable[T](originUid: Long, version: Int, dictionary: Map[T, Int]) { +private[remote] final case class CompressionTable[T](originUid: Long, version: Byte, dictionary: Map[T, Int]) { import CompressionTable.NotCompressedId def compress(value: T): Int = 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 index a53b5e7258..aa848d3051 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala @@ -5,8 +5,8 @@ package akka.remote.artery.compress /** INTERNAL API */ -private[artery] final case class DecompressionTable[T](originUid: Long, version: Int, table: Array[T]) { - // TODO version maybe better as Long? // OR implement roll-over +private[artery] final case class DecompressionTable[T](originUid: Long, version: Byte, table: Array[T]) { + private[this] val length = table.length def get(idx: Int): T = { 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 index 75dd11bfe9..5f7339dd6c 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala @@ -9,14 +9,12 @@ import java.util.function.LongFunction import scala.concurrent.duration.{ Duration, FiniteDuration } import akka.actor.{ ActorRef, ActorSystem, Address } -import akka.event.{ Logging, NoLogging } +import akka.event.Logging import akka.remote.artery.{ ArterySettings, InboundContext, OutboundContext } -import akka.util.{ OptionVal, PrettyDuration } +import akka.util.OptionVal import org.agrona.collections.Long2ObjectHashMap import scala.annotation.tailrec -import scala.concurrent.Future -import akka.Done import akka.actor.Cancellable import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicInteger @@ -29,12 +27,12 @@ import java.util.concurrent.atomic.AtomicInteger */ private[remote] trait InboundCompressions { def hitActorRef(originUid: Long, remote: Address, ref: ActorRef, n: Int): Unit - def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] - def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: 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: Int, idx: Int): OptionVal[String] - def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit + def decompressClassManifest(originUid: Long, tableVersion: Byte, idx: Int): OptionVal[String] + def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Byte): Unit /** * Cancel advertisement scheduling @@ -85,7 +83,7 @@ private[remote] final class InboundCompressionsImpl( // actor ref compression --- - override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = + 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 @@ -99,7 +97,7 @@ private[remote] final class InboundCompressionsImpl( } } - override def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = { + override def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: Byte): Unit = { _actorRefsIns.get(originUid) match { case null ⇒ // ignore case Some(a) ⇒ a.confirmAdvertisement(tableVersion) @@ -109,7 +107,7 @@ private[remote] final class InboundCompressionsImpl( // class manifest compression --- - override def decompressClassManifest(originUid: Long, tableVersion: Int, idx: Int): OptionVal[String] = + 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 @@ -122,7 +120,7 @@ private[remote] final class InboundCompressionsImpl( case None ⇒ // closed } } - override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Int): Unit = { + override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Byte): Unit = { _classManifestsIns.get(originUid) match { case null ⇒ // ignore case Some(a) ⇒ a.confirmAdvertisement(tableVersion) @@ -191,7 +189,7 @@ private[remote] final class InboundActorRefCompression( stopped: AtomicBoolean) extends InboundCompression[ActorRef](system, settings, originUid, inboundContext, heavyHitters, stopped) { - override def decompress(tableVersion: Int, idx: Int): OptionVal[ActorRef] = + override def decompress(tableVersion: Byte, idx: Int): OptionVal[ActorRef] = super.decompressInternal(tableVersion, idx, 0) override protected def tableAdvertisementInterval = settings.ActorRefs.AdvertisementInterval @@ -223,7 +221,7 @@ final class InboundManifestCompression( override def increment(remoteAddress: Address, value: String, n: Long): Unit = if (value != "") super.increment(remoteAddress, value, n) - override def decompress(incomingTableVersion: Int, idx: Int): OptionVal[String] = + override def decompress(incomingTableVersion: Byte, idx: Int): OptionVal[String] = decompressInternal(incomingTableVersion, idx, 0) } /** @@ -245,12 +243,16 @@ private[remote] object InboundCompression { nextTable: DecompressionTable[T], advertisementInProgress: Option[CompressionTable[T]]) { - def startUsingNextTable(): State[T] = + def startUsingNextTable(): State[T] = { + // wrap around to positive values + val nextVersion = (nextTable.version + 1) & 0x7F State( oldTable = activeTable, activeTable = nextTable, - nextTable = DecompressionTable.empty[T].copy(version = nextTable.version + 1), + // skip 0 when wrapped around + nextTable = DecompressionTable.empty[T].copy(version = (if (nextVersion == 0) 1 else nextVersion).byteValue), advertisementInProgress = None) + } } } @@ -295,7 +297,7 @@ private[remote] abstract class InboundCompression[T >: Null]( /* ==== COMPRESSION ==== */ /** Override and specialize if needed, for default compression logic delegate to 3-param overload */ - def decompress(incomingTableVersion: Int, idx: Int): OptionVal[T] + def decompress(incomingTableVersion: Byte, idx: Int): OptionVal[T] /** * Decompress given identifier into its original representation. @@ -304,7 +306,7 @@ private[remote] abstract class InboundCompression[T >: Null]( * * @throws UnknownCompressedIdException if given id is not known, this may indicate a bug – such situation should not happen. */ - @tailrec final def decompressInternal(incomingTableVersion: Int, idx: Int, attemptCounter: Int): OptionVal[T] = { + @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}") @@ -345,7 +347,7 @@ private[remote] abstract class InboundCompression[T >: Null]( } } - @tailrec final def confirmAdvertisement(tableVersion: Int): Unit = { + @tailrec final def confirmAdvertisement(tableVersion: Byte): Unit = { val current = state.get current.advertisementInProgress match { case Some(inProgress) if tableVersion == inProgress.version ⇒ @@ -455,7 +457,7 @@ private[remote] abstract class InboundCompression[T >: Null]( */ protected def advertiseCompressionTable(association: OutboundContext, table: CompressionTable[T]): Unit - private def prepareCompressionAdvertisement(nextTableVersion: Int): CompressionTable[T] = { + 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: _*)) } @@ -479,16 +481,16 @@ final class UnknownCompressedIdException(id: Long) */ case object NoInboundCompressions extends InboundCompressions { override def hitActorRef(originUid: Long, remote: Address, ref: ActorRef, n: Int): Unit = () - override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = + 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: Int): Unit = () + 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: Int, idx: Int): OptionVal[String] = + 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: Int): Unit = () + override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: Byte): Unit = () override def close(): Unit = () diff --git a/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala index 8e31f0ffdf..e11be5f28f 100644 --- a/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/serialization/ArteryMessageSerializer.scala @@ -143,7 +143,7 @@ private[akka] final class ArteryMessageSerializer(val system: ExtendedActorSyste 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, kvs.toMap) + val table = CompressionTable(protoAdv.getOriginUid, protoAdv.getTableVersion.byteValue, kvs.toMap) create(deserializeUniqueAddress(protoAdv.getFrom), table) } @@ -153,9 +153,9 @@ private[akka] final class ArteryMessageSerializer(val system: ExtendedActorSyste .setVersion(version) .build() - def deserializeCompressionTableAdvertisementAck(bytes: Array[Byte], create: (UniqueAddress, Int) ⇒ AnyRef): AnyRef = { + def deserializeCompressionTableAdvertisementAck(bytes: Array[Byte], create: (UniqueAddress, Byte) ⇒ AnyRef): AnyRef = { val msg = ArteryControlFormats.CompressionTableAdvertisementAck.parseFrom(bytes) - create(deserializeUniqueAddress(msg.getFrom), msg.getVersion) + create(deserializeUniqueAddress(msg.getFrom), msg.getVersion.toByte) } def serializeSystemMessageEnvelope(env: SystemMessageDelivery.SystemMessageEnvelope): ArteryControlFormats.SystemMessageEnvelope = { diff --git a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala index ecd049b445..b2c2be8b6b 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala @@ -33,19 +33,18 @@ class EnvelopeBufferSpec extends AkkaSpec { val idxToManifest = manifestToIdx.map(_.swap) val outboundActorRefTable: CompressionTable[ActorRef] = - CompressionTable(17L, version = 0xCAFE, refToIdx) + CompressionTable(17L, version = 0xCA.byteValue, refToIdx) val outboundClassManifestTable: CompressionTable[String] = - CompressionTable(17L, version = 0xBABE, manifestToIdx) + CompressionTable(17L, version = 0xBA.byteValue, manifestToIdx) override def hitActorRef(originUid: Long, remote: Address, ref: ActorRef, n: Int): Unit = () - override def decompressActorRef(originUid: Long, tableVersion: Int, idx: Int): OptionVal[ActorRef] = OptionVal(idxToRef(idx)) - override def confirmActorRefCompressionAdvertisement(originUid: Long, tableVersion: 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: Int, idx: Int): OptionVal[String] = OptionVal(idxToManifest(idx)) - override def confirmClassManifestCompressionAdvertisement(originUid: Long, tableVersion: 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 = () } @@ -79,8 +78,8 @@ class EnvelopeBufferSpec extends AkkaSpec { headerOut.version should ===(1) headerOut.uid should ===(42) - headerOut.inboundActorRefCompressionTableVersion should ===(0xCAFE) - headerOut.inboundClassManifestCompressionTableVersion should ===(0xBABE) + headerOut.inboundActorRefCompressionTableVersion should ===(0xCA.byteValue) + headerOut.inboundClassManifestCompressionTableVersion should ===(0xBA.byteValue) headerOut.serializer should ===(4) headerOut.senderActorRef(originUid).get.path.toSerializationFormat should ===("akka://EnvelopeBufferSpec/compressable0") headerOut.senderActorRefPath should ===(OptionVal.None) 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 index 49337d6bb1..497677317c 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala @@ -77,13 +77,13 @@ class CompressionIntegrationSpec extends ArteryMultiNodeSpec(CompressionIntegrat awaitAssert { val a1 = aManifestProbe.expectMsgType[Events.ReceivedClassManifestCompressionTable](2.seconds) info("System [A] received: " + a1) - a1.table.version should be >= (1) + 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 should be >= (1) + a1.table.version.toInt should be >= (1) a1.table.dictionary.keySet should contain(echoRefA) // recipient a1.table.dictionary.keySet should contain(testActor) // sender } @@ -92,13 +92,13 @@ class CompressionIntegrationSpec extends ArteryMultiNodeSpec(CompressionIntegrat awaitAssert { val b1 = bManifestProbe.expectMsgType[Events.ReceivedClassManifestCompressionTable](2.seconds) info("System [B] received: " + b1) - b1.table.version should be >= (1) + 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 should be >= (1) + b1.table.version.toInt should be >= (1) b1.table.dictionary.keySet should contain(echoRefB) } } @@ -110,29 +110,30 @@ class CompressionIntegrationSpec extends ArteryMultiNodeSpec(CompressionIntegrat echoRefA.tell(TestMessage("hello2"), ignore.ref) val a2 = aManifestProbe.expectMsgType[Events.ReceivedClassManifestCompressionTable](2.seconds) info("System [A] received more: " + a2) - a2.table.version should be >= (3) + 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 should be >= (3) + 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 should be >= (3) + 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 should be >= (3) + b2.table.version.toInt should be >= (3) } } } + } "handle noSender sender" in { @@ -229,15 +230,15 @@ class CompressionIntegrationSpec extends ArteryMultiNodeSpec(CompressionIntegrat awaitAssert { val a2 = aManifestProbe.expectMsgType[Events.ReceivedClassManifestCompressionTable](2.seconds) info("System [A] received: " + a2) - a2.table.version should be >= (1) - a2.table.version should be < (3) + 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 should be >= (1) - a2.table.version should be < (3) + 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 } @@ -246,17 +247,69 @@ class CompressionIntegrationSpec extends ArteryMultiNodeSpec(CompressionIntegrat awaitAssert { val b2 = bManifestProbe.expectMsgType[Events.ReceivedClassManifestCompressionTable](2.seconds) info("System [B2] received: " + b2) - b2.table.version should be >= (1) + 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 should be >= (1) + 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 wrapAroundCount = 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 + + if ((tableVersionsSeen & 0x7F) == 0) { + wrapAroundCount += 1 + } + } + currentTableVersion should ===((tableVersionsSeen & 0x7F) + wrapAroundCount) + } } } From c3a69e3c422fa7d23082dd246ef46bc13839eb54 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 27 Sep 2016 17:56:15 +0200 Subject: [PATCH 174/186] fix wrong afr id --- .../scala/akka/remote/artery/FlightRecorderEvents.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala index ea0ccff368..5aaa2a77c8 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala @@ -37,8 +37,8 @@ object FlightRecorderEvents { val AeronSource_Started = 70 val AeronSource_Stopped = 71 val AeronSource_Received = 72 - val AeronSource_DelegateToTaskRunner = 72 - val AeronSource_ReturnFromTaskRunner = 73 + val AeronSource_DelegateToTaskRunner = 73 + val AeronSource_ReturnFromTaskRunner = 74 // Compression events val Compression_CompressedActorRef = 90 @@ -85,6 +85,5 @@ object FlightRecorderEvents { 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 } + Compression_AllocatedManifestCompressionId → "Compression: Allocated manifest compression id").map { case (int, str) ⇒ int.toLong → str } } From e61ffd1132092514c1c3fda8f53fc04711227b7b Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 28 Sep 2016 11:14:33 +0200 Subject: [PATCH 175/186] protobuf serialization of Status.Succes, Status.Failure, #21202 (#21565) * also support for serialization of exceptions, see comment in reference.conf * extract Throwable and Payload methods to helper classes * add security checks before creating instance from class name --- .../java/akka/remote/ContainerFormats.java | 2142 ++++++++++++++++- .../src/main/protobuf/ContainerFormats.proto | 14 + akka-remote/src/main/resources/reference.conf | 17 + .../serialization/MiscMessageSerializer.scala | 71 +- .../serialization/ThrowableSupport.scala | 73 + .../serialization/WrappedPayloadSupport.scala | 49 + .../MiscMessageSerializerSpec.scala | 41 + 7 files changed, 2366 insertions(+), 41 deletions(-) create mode 100644 akka-remote/src/main/scala/akka/remote/serialization/ThrowableSupport.scala create mode 100644 akka-remote/src/main/scala/akka/remote/serialization/WrappedPayloadSupport.scala diff --git a/akka-remote/src/main/java/akka/remote/ContainerFormats.java b/akka-remote/src/main/java/akka/remote/ContainerFormats.java index 1e63dd0f4b..677071914c 100644 --- a/akka-remote/src/main/java/akka/remote/ContainerFormats.java +++ b/akka-remote/src/main/java/akka/remote/ContainerFormats.java @@ -4563,6 +4563,2115 @@ public final class ContainerFormats { // @@protoc_insertion_point(class_scope:Payload) } + 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 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 +6707,16 @@ public final class ContainerFormats { private static akka.protobuf.GeneratedMessage.FieldAccessorTable internal_static_Payload_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_StackTraceElement_descriptor; + private static + akka.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_StackTraceElement_fieldAccessorTable; public static akka.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -4619,9 +6738,14 @@ 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\"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.Paylo" + + "ad\022&\n\nstackTrace\030\004 \003(\0132\022.StackTraceEleme" + + "nt\"`\n\021StackTraceElement\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\006P" + + "ARENT\020\000\022\016\n\nCHILD_NAME\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 +6794,18 @@ public final class ContainerFormats { akka.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_Payload_descriptor, new java.lang.String[] { "EnclosedMessage", "SerializerId", "MessageManifest", }); + internal_static_Throwable_descriptor = + getDescriptor().getMessageTypes().get(7); + internal_static_Throwable_fieldAccessorTable = new + akka.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_Throwable_descriptor, + new java.lang.String[] { "ClassName", "Message", "Cause", "StackTrace", }); + internal_static_StackTraceElement_descriptor = + getDescriptor().getMessageTypes().get(8); + 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/protobuf/ContainerFormats.proto b/akka-remote/src/main/protobuf/ContainerFormats.proto index 84785740c4..6ff9fda613 100644 --- a/akka-remote/src/main/protobuf/ContainerFormats.proto +++ b/akka-remote/src/main/protobuf/ContainerFormats.proto @@ -50,3 +50,17 @@ message Payload { required int32 serializerId = 2; optional bytes messageManifest = 4; } + +message Throwable { + required string className = 1; + optional string message = 2; + optional Payload cause = 3; + repeated StackTraceElement stackTrace = 4; +} + +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/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 721d1bad78..c28ee7ae2c 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -48,6 +48,23 @@ akka { "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 + + # 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 { 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..266773c78e 100644 --- a/akka-remote/src/main/scala/akka/remote/serialization/MiscMessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/serialization/MiscMessageSerializer.scala @@ -11,6 +11,8 @@ import akka.serialization.{ Serialization, BaseSerializer, SerializationExtensio class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest with BaseSerializer { private lazy val serialization = SerializationExtension(system) + private val payloadSupport = new WrappedPayloadSupport(system) + private val throwableSupport = new ThrowableSupport(system) private val NoneSerialized = Array.empty[Byte] @@ -19,19 +21,22 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW case identity: ActorIdentity ⇒ serializeActorIdentity(identity) case Some(value) ⇒ serializeSome(value) case None ⇒ NoneSerialized + case s: Status.Success ⇒ serializeStatusSuccess(s) + case f: Status.Failure ⇒ serializeStatusFailure(f) + case t: Throwable ⇒ throwableSupport.serializeThrowable(t) 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,7 +49,7 @@ 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 @@ -52,43 +57,35 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW 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) + private def serializeStatusSuccess(success: Status.Success): Array[Byte] = + payloadSupport.payloadBuilder(success.status).build().toByteArray - 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 - } + private def serializeStatusFailure(failure: Status.Failure): Array[Byte] = + payloadSupport.payloadBuilder(failure.cause).build().toByteArray private val IdentifyManifest = "A" private val ActorIdentifyManifest = "B" private val OptionManifest = "C" + private val StatusSuccessManifest = "D" + private val StatusFailureManifest = "E" + private val ThrowableManifest = "F" private val fromBinaryMap = Map[String, Array[Byte] ⇒ AnyRef]( IdentifyManifest → deserializeIdentify, ActorIdentifyManifest → deserializeActorIdentity, - OptionManifest → deserializeOption - ) + OptionManifest → deserializeOption, + StatusSuccessManifest → deserializeStatusSuccess, + StatusFailureManifest → deserializeStatusFailure, + ThrowableManifest → throwableSupport.deserializeThrowable) override def manifest(o: AnyRef): String = o match { - case _: Identify ⇒ IdentifyManifest - case _: ActorIdentity ⇒ ActorIdentifyManifest - case _: Option[Any] ⇒ OptionManifest + case _: Identify ⇒ IdentifyManifest + case _: ActorIdentity ⇒ ActorIdentifyManifest + case _: Option[Any] ⇒ OptionManifest + case _: Status.Success ⇒ StatusSuccessManifest + case _: Status.Failure ⇒ StatusFailureManifest + case _: Throwable ⇒ ThrowableManifest case _ ⇒ throw new IllegalArgumentException(s"Can't serialize object of type ${o.getClass} in [${getClass.getName}]") } @@ -102,13 +99,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)) @@ -125,16 +122,14 @@ 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]) } 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..136f822da2 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/serialization/ThrowableSupport.scala @@ -0,0 +1,73 @@ +/** + * 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] = { + 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.build().toByteArray + } + + 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 = { + val protoT = ContainerFormats.Throwable.parseFrom(bytes) + 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/test/scala/akka/remote/serialization/MiscMessageSerializerSpec.scala b/akka-remote/src/test/scala/akka/remote/serialization/MiscMessageSerializerSpec.scala index a5ed4c45b7..d6c6366684 100644 --- a/akka-remote/src/test/scala/akka/remote/serialization/MiscMessageSerializerSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/serialization/MiscMessageSerializerSpec.scala @@ -9,6 +9,7 @@ import akka.remote.MessageSerializer import akka.serialization.SerializationExtension import akka.testkit.AkkaSpec import com.typesafe.config.ConfigFactory +import scala.util.control.NoStackTrace object MiscMessageSerializerSpec { val serializationTestOverrides = @@ -16,12 +17,46 @@ object MiscMessageSerializerSpec { 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._ "MiscMessageSerializer" must { Seq( @@ -30,6 +65,12 @@ 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 "Some" → Some("value"), "None" → None).foreach { case (scenario, item) ⇒ From be319afcd20f0c74bda655ae0ffe0194f6ec8d24 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 27 Sep 2016 15:46:22 +0200 Subject: [PATCH 176/186] serialization of ActorRef, #21202 --- akka-remote/src/main/resources/reference.conf | 2 ++ .../remote/serialization/MiscMessageSerializer.scala | 12 +++++++++++- .../serialization/MiscMessageSerializerSpec.scala | 3 +++ 3 files changed, 16 insertions(+), 1 deletion(-) diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index c28ee7ae2c..3692405d55 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -65,6 +65,8 @@ akka { "akka.actor.InvalidActorNameException" = akka-misc "akka.actor.InvalidMessageException" = akka-misc + "akka.actor.ActorRef" = akka-misc + } serialization-identifiers { 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 266773c78e..2883018822 100644 --- a/akka-remote/src/main/scala/akka/remote/serialization/MiscMessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/serialization/MiscMessageSerializer.scala @@ -21,6 +21,7 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW case identity: ActorIdentity ⇒ serializeActorIdentity(identity) case Some(value) ⇒ serializeSome(value) case None ⇒ NoneSerialized + case r: ActorRef ⇒ serializeActorRef(r) case s: Status.Success ⇒ serializeStatusSuccess(s) case f: Status.Failure ⇒ serializeStatusFailure(f) case t: Throwable ⇒ throwableSupport.serializeThrowable(t) @@ -53,6 +54,9 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW .build() .toByteArray + private def serializeActorRef(ref: ActorRef): Array[Byte] = + actorRefBuilder(ref).build().toByteArray + private def actorRefBuilder(actorRef: ActorRef): ContainerFormats.ActorRef.Builder = ContainerFormats.ActorRef.newBuilder() .setPath(Serialization.serializedActorPath(actorRef)) @@ -69,6 +73,7 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW private val StatusSuccessManifest = "D" private val StatusFailureManifest = "E" private val ThrowableManifest = "F" + private val ActorRefManifest = "G" private val fromBinaryMap = Map[String, Array[Byte] ⇒ AnyRef]( IdentifyManifest → deserializeIdentify, @@ -76,13 +81,15 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW OptionManifest → deserializeOption, StatusSuccessManifest → deserializeStatusSuccess, StatusFailureManifest → deserializeStatusFailure, - ThrowableManifest → throwableSupport.deserializeThrowable) + ThrowableManifest → throwableSupport.deserializeThrowable, + ActorRefManifest → deserializeActorRefBytes) override def manifest(o: AnyRef): String = o match { case _: Identify ⇒ IdentifyManifest case _: ActorIdentity ⇒ ActorIdentifyManifest case _: Option[Any] ⇒ OptionManifest + case _: ActorRef ⇒ ActorRefManifest case _: Status.Success ⇒ StatusSuccessManifest case _: Status.Failure ⇒ StatusFailureManifest case _: Throwable ⇒ ThrowableManifest @@ -114,6 +121,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) 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 d6c6366684..869b87bf10 100644 --- a/akka-remote/src/test/scala/akka/remote/serialization/MiscMessageSerializerSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/serialization/MiscMessageSerializerSpec.scala @@ -58,6 +58,8 @@ object MiscMessageSerializerSpec { class MiscMessageSerializerSpec extends AkkaSpec(MiscMessageSerializerSpec.testConfig) { import MiscMessageSerializerSpec._ + val ref = system.actorOf(Props.empty, "hello") + "MiscMessageSerializer" must { Seq( "Identify" → Identify("some-message"), @@ -71,6 +73,7 @@ class MiscMessageSerializerSpec extends AkkaSpec(MiscMessageSerializerSpec.testC "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 { case (scenario, item) ⇒ From 1cae346f4da6a7cf3711b8e0924fe838ee124aca Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 27 Sep 2016 16:34:43 +0200 Subject: [PATCH 177/186] add thread local LRU cache for resolveActorRef --- .../akka/remote/RemoteActorRefProvider.scala | 21 +++++- .../scala/akka/remote/artery/Codecs.scala | 6 +- .../serialization/ActorRefResolveCache.scala | 64 +++++++++++++++++++ .../artery/RemoteActorRefProviderSpec.scala | 34 ++++++++++ 4 files changed, 121 insertions(+), 4 deletions(-) create mode 100644 akka-remote/src/main/scala/akka/remote/serialization/ActorRefResolveCache.scala diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 187f9f1389..7b8b871941 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -21,6 +21,8 @@ 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 @@ -175,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") @@ -389,7 +395,20 @@ private[akka] class RemoteActorRefProvider( } } - 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 { diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index b59ac96d51..26b2b0bb43 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -255,7 +255,7 @@ private[remote] object Decoder { /** * INTERNAL API */ -private[akka] final class ActorRefResolveCache(provider: RemoteActorRefProvider, localAddress: UniqueAddress) +private[akka] final class ActorRefResolveCacheWithAddress(provider: RemoteActorRefProvider, localAddress: UniqueAddress) extends LruBoundedCache[String, InternalActorRef](capacity = 1024, evictAgeThreshold = 600) { override protected def compute(k: String): InternalActorRef = @@ -286,8 +286,8 @@ private[remote] class Decoder( import Decoder.RetryResolveRemoteDeployedRecipient private val localAddress = inboundContext.localAddress.address private val headerBuilder = HeaderBuilder.in(compression) - private val actorRefResolver: ActorRefResolveCache = - new ActorRefResolveCache(system.provider.asInstanceOf[RemoteActorRefProvider], uniqueLocalAddress) + private val actorRefResolver: ActorRefResolveCacheWithAddress = + new ActorRefResolveCacheWithAddress(system.provider.asInstanceOf[RemoteActorRefProvider], uniqueLocalAddress) private val bannedRemoteDeployedActorRefs = new java.util.HashSet[String] private val retryResolveRemoteDeployedRecipientInterval = 50.millis 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/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala index 96f7cef146..586f41e8cb 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/RemoteActorRefProviderSpec.scala @@ -6,6 +6,9 @@ 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 { @@ -29,6 +32,37 @@ class RemoteActorRefProviderSpec extends ArteryMultiNodeSpec { 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") From 14e0188a1ccdc57370ea67786d098a61268cdad1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Endre=20S=C3=A1ndor=20Varga?= Date: Mon, 26 Sep 2016 15:04:53 +0200 Subject: [PATCH 178/186] 21202: Added more serializers for built-in stuff --- .../scala/akka/serialization/Serializer.scala | 4 +- .../serialization/MiscMessageSerializer.scala | 18 ++- .../serialization/PrimitiveSerializers.scala | 89 ++++++++++++ .../MiscMessageSerializerSpec.scala | 4 +- .../PrimitivesSerializationSpec.scala | 129 ++++++++++++++++++ 5 files changed, 239 insertions(+), 5 deletions(-) create mode 100644 akka-remote/src/main/scala/akka/remote/serialization/PrimitiveSerializers.scala create mode 100644 akka-remote/src/test/scala/akka/remote/serialization/PrimitivesSerializationSpec.scala diff --git a/akka-actor/src/main/scala/akka/serialization/Serializer.scala b/akka-actor/src/main/scala/akka/serialization/Serializer.scala index d550a769ca..3c7c742da6 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serializer.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serializer.scala @@ -303,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 } @@ -322,7 +322,7 @@ 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( 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 2883018822..0403515c70 100644 --- a/akka-remote/src/main/scala/akka/remote/serialization/MiscMessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/serialization/MiscMessageSerializer.scala @@ -14,7 +14,7 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW private val payloadSupport = new WrappedPayloadSupport(system) private val throwableSupport = new ThrowableSupport(system) - private val NoneSerialized = Array.empty[Byte] + private val ParameterlessSerialized = Array.empty[Byte] def toBinary(obj: AnyRef): Array[Byte] = obj match { case identify: Identify ⇒ serializeIdentify(identify) @@ -25,6 +25,9 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW case s: Status.Success ⇒ serializeStatusSuccess(s) case f: Status.Failure ⇒ serializeStatusFailure(f) case t: Throwable ⇒ throwableSupport.serializeThrowable(t) + case None ⇒ ParameterlessSerialized + case PoisonPill ⇒ ParameterlessSerialized + case Kill ⇒ ParameterlessSerialized case _ ⇒ throw new IllegalArgumentException(s"Cannot serialize object of type [${obj.getClass.getName}]") } @@ -68,12 +71,14 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW payloadSupport.payloadBuilder(failure.cause).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 fromBinaryMap = Map[String, Array[Byte] ⇒ AnyRef]( IdentifyManifest → deserializeIdentify, @@ -83,6 +88,10 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW StatusFailureManifest → deserializeStatusFailure, ThrowableManifest → throwableSupport.deserializeThrowable, ActorRefManifest → deserializeActorRefBytes) + OptionManifest → deserializeOption, + PoisonPillManifest → ((_) ⇒ PoisonPill), + KillManifest → ((_) ⇒ Kill) + ) override def manifest(o: AnyRef): String = o match { @@ -93,6 +102,11 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW case _: Status.Success ⇒ StatusSuccessManifest case _: Status.Failure ⇒ StatusFailureManifest case _: Throwable ⇒ ThrowableManifest + case _: Identify ⇒ IdentifyManifest + case _: ActorIdentity ⇒ ActorIdentityManifest + case _: Option[Any] ⇒ OptionManifest + case _: PoisonPill.type ⇒ PoisonPillManifest + case _: Kill.type ⇒ KillManifest case _ ⇒ throw new IllegalArgumentException(s"Can't serialize object of type ${o.getClass} in [${getClass.getName}]") } 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..9952652ae4 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/serialization/PrimitiveSerializers.scala @@ -0,0 +1,89 @@ +package akka.remote.serialization + +import java.nio.ByteBuffer + +import akka.actor.{ ExtendedActorSystem, Kill, PoisonPill } +import akka.serialization.{ BaseSerializer, ByteBufferSerializer } + +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") + +} 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 869b87bf10..32054ddbf6 100644 --- a/akka-remote/src/test/scala/akka/remote/serialization/MiscMessageSerializerSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/serialization/MiscMessageSerializerSpec.scala @@ -75,7 +75,9 @@ class MiscMessageSerializerSpec extends AkkaSpec(MiscMessageSerializerSpec.testC "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).foreach { case (scenario, item) ⇒ s"resolve serializer for $scenario" in { val serializer = SerializationExtension(system) 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..075e839df2 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/serialization/PrimitivesSerializationSpec.scala @@ -0,0 +1,129 @@ +/** + * 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 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) + } + } + +} From 0f376e751ee7d4d0c1c34fa355dc59da3df9e860 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Wed, 28 Sep 2016 14:04:58 +0200 Subject: [PATCH 179/186] Quarantine gracefully downed node after some time (#21534) * New setting for quarantining after graceful leave --- .../src/main/resources/reference.conf | 5 ++++ .../akka/cluster/ClusterRemoteWatcher.scala | 26 ++++++++++++++----- .../scala/akka/cluster/ClusterSettings.scala | 3 +++ 3 files changed, 27 insertions(+), 7 deletions(-) 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/ClusterRemoteWatcher.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala index 534d23e273..af6d3cba2f 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterRemoteWatcher.scala @@ -52,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 = { @@ -74,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 = @@ -90,14 +94,22 @@ private[cluster] class ClusterRemoteWatcher( def memberRemoved(m: Member, previousStatus: MemberStatus): Unit = if (m.address != selfAddress) { clusterNodes -= m.address - // The reason we don't quarantine gracefully removed members (leaving) is that - // Cluster Singleton need to exchange TakeOver/HandOver messages. + if (previousStatus == MemberStatus.Down) { - quarantine(m.address, Some(m.uniqueAddress.longUid), s"Cluster member removed, previous status [$previousStatus]") + 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 From bf5b6077391926c7073282116f0ebf5b54deda72 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Endre=20S=C3=A1ndor=20Varga?= Date: Wed, 28 Sep 2016 16:00:50 +0200 Subject: [PATCH 180/186] Serialization for system messages, RemoteWatcher and ByteString --- .../http/impl/util/One2OneBidiFlowSpec.scala | 2 +- .../java/akka/remote/ContainerFormats.java | 1340 ++++- .../akka/remote/SystemMessageFormats.java | 4354 +++++++++++++++++ .../src/main/protobuf/ContainerFormats.proto | 10 + .../main/protobuf/SystemMessageFormats.proto | 62 + akka-remote/src/main/resources/reference.conf | 36 +- .../serialization/MiscMessageSerializer.scala | 105 +- .../serialization/PrimitiveSerializers.scala | 31 +- .../SystemMessageSerializer.scala | 161 + .../serialization/ThrowableSupport.scala | 15 +- .../MiscMessageSerializerSpec.scala | 47 +- .../PrimitivesSerializationSpec.scala | 37 + .../SystemMessageSerializationSpec.scala | 79 + 13 files changed, 6225 insertions(+), 54 deletions(-) create mode 100644 akka-remote/src/main/java/akka/remote/SystemMessageFormats.java create mode 100644 akka-remote/src/main/protobuf/SystemMessageFormats.proto create mode 100644 akka-remote/src/main/scala/akka/remote/serialization/SystemMessageSerializer.scala create mode 100644 akka-remote/src/test/scala/akka/remote/serialization/SystemMessageSerializationSpec.scala 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-remote/src/main/java/akka/remote/ContainerFormats.java b/akka-remote/src/main/java/akka/remote/ContainerFormats.java index 677071914c..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,410 @@ 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 { @@ -5776,6 +6180,897 @@ public final class ContainerFormats { // @@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 { @@ -6707,11 +8002,21 @@ 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 @@ -6738,14 +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\"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.Paylo" + - "ad\022&\n\nstackTrace\030\004 \003(\0132\022.StackTraceEleme" + - "nt\"`\n\021StackTraceElement\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\006P" + - "ARENT\020\000\022\016\n\nCHILD_NAME\020\001\022\021\n\rCHILD_PATTERN" + - "\020\002B\017\n\013akka.remoteH\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() { @@ -6794,14 +8102,26 @@ public final class ContainerFormats { akka.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_Payload_descriptor, new java.lang.String[] { "EnclosedMessage", "SerializerId", "MessageManifest", }); - internal_static_Throwable_descriptor = + 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(8); + getDescriptor().getMessageTypes().get(10); internal_static_StackTraceElement_fieldAccessorTable = new akka.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_StackTraceElement_descriptor, 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/protobuf/ContainerFormats.proto b/akka-remote/src/main/protobuf/ContainerFormats.proto index 6ff9fda613..a760413f8d 100644 --- a/akka-remote/src/main/protobuf/ContainerFormats.proto +++ b/akka-remote/src/main/protobuf/ContainerFormats.proto @@ -51,6 +51,10 @@ message Payload { optional bytes messageManifest = 4; } +message WatcherHeartbeatResponse { + required uint64 uid = 1; +} + message Throwable { required string className = 1; optional string message = 2; @@ -58,6 +62,12 @@ message Throwable { 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; 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 3692405d55..390ad12028 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -18,6 +18,11 @@ akka { 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 { @@ -50,7 +55,24 @@ akka { "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 @@ -58,16 +80,13 @@ akka { # 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. + # 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 - - "akka.actor.ActorRef" = akka-misc - - } + } serialization-identifiers { "akka.remote.serialization.ProtobufSerializer" = 2 @@ -75,6 +94,11 @@ akka { "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 { 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 0403515c70..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,30 +5,33 @@ 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 ParameterlessSerialized = 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 r: ActorRef ⇒ serializeActorRef(r) - case s: Status.Success ⇒ serializeStatusSuccess(s) - case f: Status.Failure ⇒ serializeStatusFailure(f) - case t: Throwable ⇒ throwableSupport.serializeThrowable(t) - case None ⇒ ParameterlessSerialized - case PoisonPill ⇒ ParameterlessSerialized - case Kill ⇒ ParameterlessSerialized - 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] = @@ -60,6 +63,10 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW 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)) @@ -70,6 +77,17 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW 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 ActorIdentityManifest = "B" private val OptionManifest = "C" @@ -79,34 +97,40 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW 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, + ActorIdentityManifest → deserializeActorIdentity, OptionManifest → deserializeOption, StatusSuccessManifest → deserializeStatusSuccess, StatusFailureManifest → deserializeStatusFailure, ThrowableManifest → throwableSupport.deserializeThrowable, - ActorRefManifest → deserializeActorRefBytes) + ActorRefManifest → deserializeActorRefBytes, OptionManifest → deserializeOption, PoisonPillManifest → ((_) ⇒ PoisonPill), - KillManifest → ((_) ⇒ Kill) + 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 _: ActorRef ⇒ ActorRefManifest - case _: Status.Success ⇒ StatusSuccessManifest - case _: Status.Failure ⇒ StatusFailureManifest - case _: Throwable ⇒ ThrowableManifest - case _: Identify ⇒ IdentifyManifest - case _: ActorIdentity ⇒ ActorIdentityManifest - case _: Option[Any] ⇒ OptionManifest - case _: PoisonPill.type ⇒ PoisonPillManifest - case _: Kill.type ⇒ KillManifest + 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}]") } @@ -156,4 +180,25 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW 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 index 9952652ae4..4b71fd4d12 100644 --- a/akka-remote/src/main/scala/akka/remote/serialization/PrimitiveSerializers.scala +++ b/akka-remote/src/main/scala/akka/remote/serialization/PrimitiveSerializers.scala @@ -1,9 +1,11 @@ package akka.remote.serialization -import java.nio.ByteBuffer +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 @@ -87,3 +89,30 @@ class StringSerializer(val system: ExtendedActorSystem) extends BaseSerializer w 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 index 136f822da2..6d8a276af2 100644 --- a/akka-remote/src/main/scala/akka/remote/serialization/ThrowableSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/serialization/ThrowableSupport.scala @@ -16,6 +16,10 @@ private[akka] class ThrowableSupport(system: ExtendedActorSystem) { 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) @@ -31,7 +35,7 @@ private[akka] class ThrowableSupport(system: ExtendedActorSystem) { } } - b.build().toByteArray + b } def stackTraceElementBuilder(elem: StackTraceElement): ContainerFormats.StackTraceElement.Builder = { @@ -43,7 +47,10 @@ private[akka] class ThrowableSupport(system: ExtendedActorSystem) { } def deserializeThrowable(bytes: Array[Byte]): Throwable = { - val protoT = ContainerFormats.Throwable.parseFrom(bytes) + 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] @@ -63,9 +70,9 @@ private[akka] class ThrowableSupport(system: ExtendedActorSystem) { import scala.collection.JavaConverters._ val stackTrace = - (protoT.getStackTraceList.asScala.map { elem ⇒ + protoT.getStackTraceList.asScala.map { elem ⇒ new StackTraceElement(elem.getClassName, elem.getMethodName, elem.getFileName, elem.getLineNumber) - }).toArray + }.toArray t.setStackTrace(stackTrace) t } 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 32054ddbf6..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,10 +5,11 @@ 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 { @@ -77,7 +78,10 @@ class MiscMessageSerializerSpec extends AkkaSpec(MiscMessageSerializerSpec.testC "Some" → Some("value"), "None" → None, "Kill" → Kill, - "PoisonPill" → PoisonPill).foreach { + "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) @@ -107,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 index 075e839df2..23f5229007 100644 --- a/akka-remote/src/test/scala/akka/remote/serialization/PrimitivesSerializationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/serialization/PrimitivesSerializationSpec.scala @@ -8,6 +8,7 @@ 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 @@ -126,4 +127,40 @@ class PrimitivesSerializationSpec extends AkkaSpec(PrimitivesSerializationSpec.t } } + "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) + + } + + } + +} From fb886f0df960d8b80d0db8c61b479bb6f023ec1c Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 26 Sep 2016 11:57:37 +0200 Subject: [PATCH 181/186] configuration properties for frame size --- akka-remote/src/main/resources/reference.conf | 25 +++++++++++++++++++ .../akka/remote/artery/ArterySettings.scala | 13 +++++++--- .../akka/remote/artery/ArteryTransport.scala | 4 +-- 3 files changed, 36 insertions(+), 6 deletions(-) diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 3692405d55..47352d7b1e 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -190,6 +190,31 @@ akka { 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 diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala index 3e68421f0b..d98319c8de 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala @@ -119,10 +119,15 @@ private[akka] final class ArterySettings private (config: Config) { val FlightRecorderDestination: String = getString("flight-recorder.destination") val Compression = new Compression(getConfig("compression")) - final val MaximumFrameSize = 1024 * 1024 - final val MaximumPooledBuffers = 128 - final val MaximumLargeFrameSize = MaximumFrameSize * 5 - final val InboundBroadcastHubBufferSize = MaximumPooledBuffers / 2 + 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") } } diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 81976057f4..89a58f37bf 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -353,8 +353,8 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val restartCounter = new RestartCounter(settings.Advanced.InboundMaxRestarts, settings.Advanced.InboundRestartTimeout) - private val envelopeBufferPool = new EnvelopeBufferPool(settings.Advanced.MaximumFrameSize, settings.Advanced.MaximumPooledBuffers) - private val largeEnvelopeBufferPool = new EnvelopeBufferPool(settings.Advanced.MaximumLargeFrameSize, settings.Advanced.MaximumPooledBuffers) + 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 From 00c5895e77d681bdd7f85938650f138474cf3dc8 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 26 Sep 2016 11:59:11 +0200 Subject: [PATCH 182/186] config of control stream dispatcher --- akka-remote/src/main/resources/reference.conf | 22 +++++++++++-------- .../akka/remote/artery/ArterySettings.scala | 16 ++++++++++++-- .../akka/remote/artery/ArteryTransport.scala | 17 +++++++------- .../akka/remote/artery/Association.scala | 1 + 4 files changed, 37 insertions(+), 19 deletions(-) diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 47352d7b1e..d262c30859 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -166,10 +166,6 @@ akka { # but must be resolved to ActorRefs first. large-message-destinations = [] - # If set to a nonempty string artery will use the given dispatcher for - # its internal actors otherwise the default dispatcher is used. - use-dispatcher = "akka.remote.default-remote-dispatcher" - # 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 @@ -220,10 +216,18 @@ akka { # Settings for the materializer that is used for the remote streams. materializer = ${akka.stream.materializer} - materializer { - dispatcher = "akka.remote.default-remote-dispatcher" - } - + + # 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'. @@ -441,7 +445,7 @@ 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 diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala index d98319c8de..2d13805edc 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala @@ -54,7 +54,6 @@ private[akka] final class ArterySettings private (config: Config) { val segments = entry.split('/').tail tree.insert(segments, NotUsed) } - val Dispatcher = getString("use-dispatcher") val UntrustedMode: Boolean = getBoolean("untrusted-mode") val TrustedSelectionPaths: Set[String] = immutableSeq(getStringList("trusted-selection-paths")).toSet @@ -67,7 +66,20 @@ private[akka] final class ArterySettings private (config: Config) { import config._ val TestMode: Boolean = getBoolean("test-mode") - val MaterializerSettings = ActorMaterializerSettings(config.getConfig("materializer")) + + 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") diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 89a58f37bf..93630d0918 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -300,6 +300,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R @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) @@ -327,8 +328,6 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R private val inboundLanes = settings.Advanced.InboundLanes - private val remoteDispatcher = system.dispatchers.lookup(settings.Dispatcher) - // TODO use WildcardIndex.isEmpty when merged from master val largeMessageChannelEnabled = !settings.LargeMessageDestinations.wildcardTree.isEmpty || !settings.LargeMessageDestinations.doubleWildcardTree.isEmpty @@ -384,6 +383,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R remoteAddress ⇒ new Association( this, materializer, + controlMaterializer, remoteAddress, controlSubject, settings.LargeMessageDestinations, @@ -413,8 +413,6 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R else ArteryTransport.autoSelectPort(settings.Bind.Hostname) } else settings.Bind.Port - // TODO: Configure materializer properly - // TODO: Have a supervisor actor _localAddress = UniqueAddress( Address(ArteryTransport.ProtocolName, system.name, settings.Canonical.Hostname, port), AddressUidExtension(system).longAddressUid) @@ -428,6 +426,9 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R 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) @@ -610,7 +611,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R aeronSource(controlStreamId, envelopeBufferPool) .via(inboundFlow(compression)) .toMat(inboundControlSink)(Keep.both) - .run()(materializer) + .run()(controlMaterializer) controlSubject = ctrl @@ -784,10 +785,10 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R else { val flushingPromise = Promise[Done]() system.systemActorOf(FlushOnShutdown.props(flushingPromise, settings.Advanced.ShutdownFlushTimeout, - this, allAssociations).withDispatcher(settings.Dispatcher), "remoteFlushOnShutdown") + this, allAssociations), "remoteFlushOnShutdown") flushingPromise.future } - implicit val ec = remoteDispatcher + implicit val ec = system.dispatcher flushing.recover { case _ ⇒ Done }.flatMap(_ ⇒ internalShutdown()) } else { Future.successful(Done) @@ -844,7 +845,7 @@ private[remote] class ArteryTransport(_system: ExtendedActorSystem, _provider: R * Will complete successfully even if one of the stream completion futures failed */ private def streamsCompleted: Future[Done] = { - implicit val ec = remoteDispatcher + implicit val ec = system.dispatcher for { _ ← Future.traverse(associationRegistry.allAssociations)(_.streamsCompleted) _ ← Future.sequence(streamMatValues.get().valuesIterator.map { diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 855a90fc32..4532106c43 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -105,6 +105,7 @@ private[remote] object Association { private[remote] class Association( val transport: ArteryTransport, val materializer: Materializer, + val controlMaterializer: Materializer, override val remoteAddress: Address, override val controlSubject: ControlMessageSubject, largeMessageDestinations: WildcardIndex[NotUsed], From 74df8226deeaea8da59a940184f9f9041dc1e2ff Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 29 Sep 2016 10:50:37 +0200 Subject: [PATCH 183/186] add/change private visibility --- .../scala/akka/remote/artery/AeronSink.scala | 8 +++++-- .../akka/remote/artery/AeronSource.scala | 8 +++++-- .../akka/remote/artery/ArterySettings.scala | 2 +- .../akka/remote/artery/ArteryTransport.scala | 8 +++---- .../akka/remote/artery/Association.scala | 4 ++-- .../scala/akka/remote/artery/BufferPool.scala | 5 ++++- .../scala/akka/remote/artery/Codecs.scala | 2 +- .../scala/akka/remote/artery/Control.scala | 22 +++++++++---------- .../akka/remote/artery/FlightRecorder.scala | 10 ++++----- .../remote/artery/FlightRecorderEvents.scala | 5 ++++- .../scala/akka/remote/artery/Handshake.scala | 6 ++--- .../akka/remote/artery/InboundEnvelope.scala | 2 +- .../artery/InboundQuarantineCheck.scala | 2 +- .../akka/remote/artery/LruBoundedCache.scala | 5 ++++- .../remote/artery/MessageDispatcher.scala | 2 +- .../akka/remote/artery/OutboundEnvelope.scala | 8 +++---- .../akka/remote/artery/RemoteInstrument.scala | 5 ++++- .../akka/remote/artery/RestartCounter.scala | 4 ++-- .../remote/artery/SystemMessageDelivery.scala | 6 ++--- .../artery/compress/CompressionProtocol.scala | 9 +++++--- .../artery/compress/DecompressionTable.scala | 4 ++-- .../artery/compress/InboundCompressions.scala | 16 +++++++++----- .../artery/compress/TopHeavyHitters.scala | 5 ++++- 23 files changed, 90 insertions(+), 58 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala index d19ecf5c7a..cf6d56107c 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSink.scala @@ -27,7 +27,10 @@ import io.aeron.Publication import org.agrona.concurrent.UnsafeBuffer import org.agrona.hints.ThreadHints -object AeronSink { +/** + * INTERNAL API + */ +private[remote] object AeronSink { final class GaveUpMessageException(msg: String) extends RuntimeException(msg) with NoStackTrace @@ -73,9 +76,10 @@ object AeronSink { } /** + * INTERNAL API * @param channel eg. "aeron:udp?endpoint=localhost:40123" */ -class AeronSink( +private[remote] class AeronSink( channel: String, streamId: Int, aeron: Aeron, diff --git a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala index 2dda28a2c0..931998a8f1 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/AeronSource.scala @@ -24,7 +24,10 @@ import org.agrona.hints.ThreadHints import akka.stream.stage.GraphStageWithMaterializedValue import scala.util.control.NonFatal -object AeronSource { +/** + * INTERNAL API + */ +private[remote] object AeronSource { private def pollTask(sub: Subscription, handler: MessageHandler, onMessage: AsyncCallback[EnvelopeBuffer]): () ⇒ Boolean = { () ⇒ @@ -64,9 +67,10 @@ object AeronSource { } /** + * INTERNAL API * @param channel eg. "aeron:udp?endpoint=localhost:40123" */ -class AeronSource( +private[remote] class AeronSource( channel: String, streamId: Int, aeron: Aeron, diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala index 2d13805edc..fe94f28c8a 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala @@ -148,7 +148,7 @@ private[akka] object ArterySettings { def apply(config: Config) = new ArterySettings(config) /** INTERNAL API */ - private[akka] final class Compression private[ArterySettings] (config: Config) { + private[remote] final class Compression private[ArterySettings] (config: Config) { import config._ final val Enabled = true diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala index 93630d0918..c92684c727 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArteryTransport.scala @@ -72,7 +72,7 @@ import akka.remote.artery.Association.OutboundStreamMatValues * Inbound API that is used by the stream stages. * Separate trait to facilitate testing without real transport. */ -private[akka] trait InboundContext { +private[remote] trait InboundContext { /** * The local inbound address. */ @@ -105,7 +105,7 @@ private[akka] trait InboundContext { /** * INTERNAL API */ -private[akka] object AssociationState { +private[remote] object AssociationState { def apply(): AssociationState = new AssociationState( incarnation = 1, @@ -121,7 +121,7 @@ private[akka] object AssociationState { /** * INTERNAL API */ -private[akka] final class AssociationState( +private[remote] final class AssociationState( val incarnation: Int, val uniqueRemoteAddressPromise: Promise[UniqueAddress], val quarantined: ImmutableLongMap[AssociationState.QuarantinedTimestamp]) { @@ -188,7 +188,7 @@ private[akka] final class AssociationState( * Outbound association API that is used by the stream stages. * Separate trait to facilitate testing without real transport. */ -private[akka] trait OutboundContext { +private[remote] trait OutboundContext { /** * The local inbound address. */ diff --git a/akka-remote/src/main/scala/akka/remote/artery/Association.scala b/akka-remote/src/main/scala/akka/remote/artery/Association.scala index 4532106c43..6527ed7e17 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Association.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Association.scala @@ -157,7 +157,7 @@ private[remote] class Association( // in case there is a restart at the same time as a compression table update private val changeCompressionTimeout = 5.seconds - private[artery] def changeActorRefCompression(table: CompressionTable[ActorRef]): Future[Done] = { + private[remote] def changeActorRefCompression(table: CompressionTable[ActorRef]): Future[Done] = { import transport.system.dispatcher val c = changeOutboundCompression val result = @@ -170,7 +170,7 @@ private[remote] class Association( private[this] val streamMatValues = new AtomicReference(Map.empty[Int, OutboundStreamMatValues]) private[this] val idle = new AtomicReference[Option[Cancellable]](None) - private[artery] def changeClassManifestCompression(table: CompressionTable[String]): Future[Done] = { + private[remote] def changeClassManifestCompression(table: CompressionTable[String]): Future[Done] = { import transport.system.dispatcher val c = changeOutboundCompression val result = diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index 159bf29748..cc9aeab3f9 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -52,7 +52,10 @@ 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)})" } -object ByteFlag { +/** + * INTERNAL API + */ +private[remote] object ByteFlag { def binaryLeftPad(byte: Byte): String = { val string = Integer.toBinaryString(byte) val pad = "0" * (8 - string.length) // leftPad diff --git a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala index 26b2b0bb43..8ca2a0eae6 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Codecs.scala @@ -255,7 +255,7 @@ private[remote] object Decoder { /** * INTERNAL API */ -private[akka] final class ActorRefResolveCacheWithAddress(provider: RemoteActorRefProvider, localAddress: UniqueAddress) +private[remote] final class ActorRefResolveCacheWithAddress(provider: RemoteActorRefProvider, localAddress: UniqueAddress) extends LruBoundedCache[String, InternalActorRef](capacity = 1024, evictAgeThreshold = 600) { override protected def compute(k: String): InternalActorRef = diff --git a/akka-remote/src/main/scala/akka/remote/artery/Control.scala b/akka-remote/src/main/scala/akka/remote/artery/Control.scala index 0e89ca46c5..3c8a2b38cc 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Control.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Control.scala @@ -21,39 +21,39 @@ import akka.util.OptionVal import akka.event.Logging /** INTERNAL API: marker trait for protobuf-serializable artery messages */ -private[akka] trait ArteryMessage extends Serializable +private[remote] trait ArteryMessage extends Serializable /** * INTERNAL API: Marker trait for reply messages */ -private[akka] trait Reply extends ControlMessage +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[akka] trait ControlMessage extends ArteryMessage +private[remote] trait ControlMessage extends ArteryMessage /** * INTERNAL API */ -private[akka] final case class Quarantined(from: UniqueAddress, to: UniqueAddress) extends ControlMessage +private[remote] final case class Quarantined(from: UniqueAddress, to: UniqueAddress) extends ControlMessage /** * INTERNAL API */ -private[akka] case class ActorSystemTerminating(from: UniqueAddress) extends ControlMessage +private[remote] case class ActorSystemTerminating(from: UniqueAddress) extends ControlMessage /** * INTERNAL API */ -private[akka] case class ActorSystemTerminatingAck(from: UniqueAddress) extends ArteryMessage +private[remote] case class ActorSystemTerminatingAck(from: UniqueAddress) extends ArteryMessage /** * INTERNAL API */ -private[akka] object InboundControlJunction { +private[remote] object InboundControlJunction { /** * Observer subject for inbound control messages. @@ -86,7 +86,7 @@ private[akka] object InboundControlJunction { /** * INTERNAL API */ -private[akka] class InboundControlJunction +private[remote] class InboundControlJunction extends GraphStageWithMaterializedValue[FlowShape[InboundEnvelope, InboundEnvelope], InboundControlJunction.ControlMessageSubject] { import InboundControlJunction._ @@ -154,8 +154,8 @@ private[akka] class InboundControlJunction /** * INTERNAL API */ -private[akka] object OutboundControlJunction { - private[akka] trait OutboundControlIngress { +private[remote] object OutboundControlJunction { + private[remote] trait OutboundControlIngress { def sendControlMessage(message: ControlMessage): Unit } } @@ -163,7 +163,7 @@ private[akka] object OutboundControlJunction { /** * INTERNAL API */ -private[akka] class OutboundControlJunction( +private[remote] class OutboundControlJunction( outboundContext: OutboundContext, outboundEnvelopePool: ObjectPool[ReusableOutboundEnvelope]) extends GraphStageWithMaterializedValue[FlowShape[OutboundEnvelope, OutboundEnvelope], OutboundControlJunction.OutboundControlIngress] { import OutboundControlJunction._ diff --git a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala index c6d63817a1..0a20fe437d 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorder.scala @@ -263,15 +263,15 @@ private[remote] object FlightRecorder { /** * INTERNAL API */ -private[akka] sealed trait FlightRecorderStatus -case object Running extends FlightRecorderStatus -case object ShutDown extends FlightRecorderStatus -final case class SnapshotInProgress(latch: CountDownLatch) extends FlightRecorderStatus +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[akka] class FlightRecorder(val fileChannel: FileChannel) extends AtomicReference[FlightRecorderStatus](Running) { +private[remote] class FlightRecorder(val fileChannel: FileChannel) extends AtomicReference[FlightRecorderStatus](Running) { import FlightRecorder._ private[this] val globalSection = new MappedResizeableBuffer(fileChannel, 0, GlobalSectionSize) diff --git a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala index 5aaa2a77c8..1285aa6a36 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/FlightRecorderEvents.scala @@ -1,6 +1,9 @@ package akka.remote.artery -object FlightRecorderEvents { +/** + * INTERNAL API + */ +private[remote] object FlightRecorderEvents { // Note: Remember to update dictionary when adding new events! diff --git a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala index 3448c26e78..1c3abacbbc 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/Handshake.scala @@ -25,7 +25,7 @@ import akka.actor.Address /** * INTERNAL API */ -private[akka] object OutboundHandshake { +private[remote] object OutboundHandshake { /** * Stream is failed with this exception if the handshake is not completed @@ -50,7 +50,7 @@ private[akka] object OutboundHandshake { /** * INTERNAL API */ -private[akka] class OutboundHandshake( +private[remote] class OutboundHandshake( system: ActorSystem, outboundContext: OutboundContext, outboundEnvelopePool: ObjectPool[ReusableOutboundEnvelope], @@ -162,7 +162,7 @@ private[akka] class OutboundHandshake( /** * INTERNAL API */ -private[akka] class InboundHandshake(inboundContext: InboundContext, inControlStream: Boolean) extends GraphStage[FlowShape[InboundEnvelope, InboundEnvelope]] { +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) diff --git a/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala b/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala index 8caae6576b..cf2e8b3b54 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/InboundEnvelope.scala @@ -64,7 +64,7 @@ private[remote] object ReusableInboundEnvelope { /** * INTERNAL API */ -private[akka] final class ReusableInboundEnvelope extends InboundEnvelope { +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 diff --git a/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala b/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala index 8ac426bdca..edbbc289e5 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/InboundQuarantineCheck.scala @@ -20,7 +20,7 @@ import akka.actor.ActorSelectionMessage /** * INTERNAL API */ -private[akka] class InboundQuarantineCheck(inboundContext: InboundContext) extends GraphStage[FlowShape[InboundEnvelope, InboundEnvelope]] { +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) diff --git a/akka-remote/src/main/scala/akka/remote/artery/LruBoundedCache.scala b/akka-remote/src/main/scala/akka/remote/artery/LruBoundedCache.scala index 40e382348f..ae4fa13b29 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/LruBoundedCache.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/LruBoundedCache.scala @@ -8,7 +8,10 @@ import akka.util.{ OptionVal, Unsafe } import scala.annotation.tailrec import scala.reflect.ClassTag -object FastHash { +/** + * 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 = { diff --git a/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala b/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala index 7eed45c201..d60f43f476 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/MessageDispatcher.scala @@ -22,7 +22,7 @@ import akka.event.LoggingReceive /** * INTERNAL API */ -private[akka] class MessageDispatcher( +private[remote] class MessageDispatcher( system: ExtendedActorSystem, provider: RemoteActorRefProvider) { diff --git a/akka-remote/src/main/scala/akka/remote/artery/OutboundEnvelope.scala b/akka-remote/src/main/scala/akka/remote/artery/OutboundEnvelope.scala index 3625f612dd..45e8fc8ea5 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/OutboundEnvelope.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/OutboundEnvelope.scala @@ -10,7 +10,7 @@ import akka.util.OptionVal /** * INTERNAL API */ -private[akka] object OutboundEnvelope { +private[remote] object OutboundEnvelope { def apply( recipient: OptionVal[RemoteActorRef], message: AnyRef, @@ -24,7 +24,7 @@ private[akka] object OutboundEnvelope { /** * INTERNAL API */ -private[akka] trait OutboundEnvelope { +private[remote] trait OutboundEnvelope { def recipient: OptionVal[RemoteActorRef] def message: AnyRef def sender: OptionVal[ActorRef] @@ -37,7 +37,7 @@ private[akka] trait OutboundEnvelope { /** * INTERNAL API */ -private[akka] object ReusableOutboundEnvelope { +private[remote] object ReusableOutboundEnvelope { def createObjectPool(capacity: Int) = new ObjectPool[ReusableOutboundEnvelope]( capacity, create = () ⇒ new ReusableOutboundEnvelope, clear = outEnvelope ⇒ outEnvelope.clear()) @@ -46,7 +46,7 @@ private[akka] object ReusableOutboundEnvelope { /** * INTERNAL API */ -private[akka] final class ReusableOutboundEnvelope extends OutboundEnvelope { +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 diff --git a/akka-remote/src/main/scala/akka/remote/artery/RemoteInstrument.scala b/akka-remote/src/main/scala/akka/remote/artery/RemoteInstrument.scala index e9a8764e35..9da058947e 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/RemoteInstrument.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/RemoteInstrument.scala @@ -184,6 +184,9 @@ private[remote] final class MetadataMap[T >: Null] { else s"MetadataMap(${backing.toList.mkString("[", ",", "]")})" } -object MetadataMap { +/** + * 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 index 6f23b239d7..e11637e4c8 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/RestartCounter.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/RestartCounter.scala @@ -11,14 +11,14 @@ import scala.annotation.tailrec /** * INTERNAL API */ -private[akka] object RestartCounter { +private[remote] object RestartCounter { final case class State(count: Int, deadline: Deadline) } /** * INTERNAL API: Thread safe "restarts with duration" counter */ -private[akka] class RestartCounter(maxRestarts: Int, restartTimeout: FiniteDuration) { +private[remote] class RestartCounter(maxRestarts: Int, restartTimeout: FiniteDuration) { import RestartCounter._ private val state = new AtomicReference[State](State(0, Deadline.now + restartTimeout)) diff --git a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala index 1a6eed90d7..931a2b3b5b 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/SystemMessageDelivery.scala @@ -30,7 +30,7 @@ import scala.util.control.NoStackTrace /** * INTERNAL API */ -private[akka] object SystemMessageDelivery { +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 @@ -51,7 +51,7 @@ private[akka] object SystemMessageDelivery { /** * INTERNAL API */ -private[akka] class SystemMessageDelivery( +private[remote] class SystemMessageDelivery( outboundContext: OutboundContext, deadLetters: ActorRef, resendInterval: FiniteDuration, @@ -260,7 +260,7 @@ private[akka] class SystemMessageDelivery( /** * INTERNAL API */ -private[akka] class SystemMessageAcker(inboundContext: InboundContext) extends GraphStage[FlowShape[InboundEnvelope, InboundEnvelope]] { +private[remote] class SystemMessageAcker(inboundContext: InboundContext) extends GraphStage[FlowShape[InboundEnvelope, InboundEnvelope]] { import SystemMessageDelivery._ val in: Inlet[InboundEnvelope] = Inlet("SystemMessageAcker.in") 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 index 3c8fc463a6..2945bf70f8 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionProtocol.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionProtocol.scala @@ -10,7 +10,10 @@ import akka.remote.artery.ControlMessage // FIXME serialization /** INTERNAL API */ -object CompressionProtocol { +/** + * INTERNAL API + */ +private[remote] object CompressionProtocol { /** INTERNAL API */ sealed trait CompressionMessage @@ -56,9 +59,9 @@ object CompressionProtocol { extends ControlMessage with CompressionMessage /** INTERNAL API */ - private[akka] object Events { + private[remote] object Events { /** INTERNAL API */ - private[akka] sealed trait Event + private[remote] sealed trait Event /** INTERNAL API */ final case class HeavyHitterDetected(key: Any, id: Int, count: Long) extends Event 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 index aa848d3051..dfc6734f35 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala @@ -5,7 +5,7 @@ package akka.remote.artery.compress /** INTERNAL API */ -private[artery] final case class DecompressionTable[T](originUid: Long, version: Byte, table: Array[T]) { +private[remote] final case class DecompressionTable[T](originUid: Long, version: Byte, table: Array[T]) { private[this] val length = table.length @@ -26,7 +26,7 @@ private[artery] final case class DecompressionTable[T](originUid: Long, version: } /** INTERNAL API */ -private[artery] object DecompressionTable { +private[remote] object DecompressionTable { private[this] val _empty = DecompressionTable(0, 0, Array.empty) def empty[T] = _empty.asInstanceOf[DecompressionTable[T]] } 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 index 5f7339dd6c..9762fb8834 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala @@ -201,7 +201,10 @@ private[remote] final class InboundActorRefCompression( } } -final class InboundManifestCompression( +/** + * INTERNAL API + */ +private[remote] final class InboundManifestCompression( system: ActorSystem, settings: ArterySettings.Compression, originUid: Long, @@ -452,8 +455,8 @@ private[remote] abstract class InboundCompression[T >: Null]( } /** - * Must be implementeed by extending classes in order to send a [[akka.remote.artery.ControlMessage]] - * of apropriate type to the remote system in order to advertise the compression table to it. + * 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 @@ -467,7 +470,10 @@ private[remote] abstract class InboundCompression[T >: Null]( } -final class UnknownCompressedIdException(id: Long) +/** + * 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, " + @@ -479,7 +485,7 @@ final class UnknownCompressedIdException(id: Long) * * Literarily, no compression! */ -case object NoInboundCompressions extends InboundCompressions { +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") 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 index 715b081b71..638806bb58 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/TopHeavyHitters.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/TopHeavyHitters.scala @@ -285,7 +285,10 @@ private[remote] final class TopHeavyHitters[T >: Null](val max: Int)(implicit cl s"${getClass.getSimpleName}(max:$max)" } -object TopHeavyHitters { +/** + * 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 { From ba1cf38e53343111afb295f7436033c9b30f564f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Thu, 29 Sep 2016 13:52:48 +0200 Subject: [PATCH 184/186] Realign artery header data (#21581) * Minor cleanup in version calculation * Table versions before tag-fields * Single handling block of unknown compression table version #21580 * Compression table versions use -1 only as special number #21448 * Align to 4 byte boundaries in header --- .../akka/remote/artery/ArterySettings.scala | 2 +- .../scala/akka/remote/artery/BufferPool.scala | 51 +++++++------------ .../artery/compress/CompressionTable.scala | 6 ++- .../artery/compress/DecompressionTable.scala | 10 +++- .../artery/compress/InboundCompressions.scala | 20 +++----- .../remote/artery/EnvelopeBufferSpec.scala | 8 +-- .../compress/CompressionIntegrationSpec.scala | 7 +-- 7 files changed, 45 insertions(+), 59 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala index fe94f28c8a..23fee6d16c 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/ArterySettings.scala @@ -170,7 +170,7 @@ private[akka] object ArterySettings { } object Compression { // Compile time constants - final val Debug = false // unlocks additional very verbose debug logging of compression events (on DEBUG log level) + 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 { diff --git a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala index cc9aeab3f9..c2dcd094be 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/BufferPool.scala @@ -76,19 +76,19 @@ private[remote] object EnvelopeBuffer { val VersionOffset = 0 // Byte val FlagsOffset = 1 // Byte - // 2 bytes free // TODO re-align values to not have this empty space + 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 - val ActorRefCompressionTableVersionTagOffset = 28 // Int // TODO handle roll-over and move to Short - val ClassManifestCompressionTableVersionTagOffset = 32 // Int // TODO handle roll-over and move to Short // EITHER metadata followed by literals directly OR literals directly in this spot. // Mode depends on the `MetadataPresentFlag`. - val MetadataContainerAndLiteralSectionOffset = 36 // Int + val MetadataContainerAndLiteralSectionOffset = 28 // Int val UsAscii = Charset.forName("US-ASCII") @@ -386,29 +386,20 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { byteBuffer.clear() // Write fixed length parts - byteBuffer.put(header.version) - byteBuffer.put(header.flags) - // 1 empty byte slot // TODO avoid having these empty slots - // 1 empty byte slot - byteBuffer.position(UidOffset) // skips the above 2 empty slots - byteBuffer.putLong(header.uid) - byteBuffer.putInt(header.serializer) + 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.putInt(ActorRefCompressionTableVersionTagOffset, header.outboundActorRefCompression.version | TagTypeMask) - byteBuffer.putInt(ClassManifestCompressionTableVersionTagOffset, header.outboundClassManifestCompression.version | TagTypeMask) - byteBuffer.putInt(SenderActorRefTagOffset, header._senderActorRefIdx | TagTypeMask) + 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 - byteBuffer.position(MetadataContainerAndLiteralSectionOffset) - header.metadataContainer.copyToBuffer(byteBuffer) // after metadata is written, buffer is at correct position to continue writing literals (they "moved forward") - } else { - // Write compressable, variable-length parts always to the actual position of the buffer - // Write tag values explicitly in their proper offset - byteBuffer.position(MetadataContainerAndLiteralSectionOffset) } // Serialize sender @@ -435,22 +426,14 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) { val header = h.asInstanceOf[HeaderBuilderImpl] // Read fixed length parts - header setVersion byteBuffer.get() - header setFlags byteBuffer.get() - byteBuffer.get() // skip 1 byte - byteBuffer.get() // skip 1 byte - header setUid byteBuffer.getLong - header setSerializer byteBuffer.getInt + 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) - val refCompressionVersionTag = byteBuffer.getInt(ActorRefCompressionTableVersionTagOffset) - if ((refCompressionVersionTag & TagTypeMask) != 0) { - header._inboundActorRefCompressionTableVersion = (refCompressionVersionTag & TagValueMask).byteValue - } - val manifestCompressionVersionTag = byteBuffer.getInt(ClassManifestCompressionTableVersionTagOffset) - if ((manifestCompressionVersionTag & TagTypeMask) != 0) { - header._inboundClassManifestCompressionTableVersion = (manifestCompressionVersionTag & TagValueMask).byteValue - } + header._inboundActorRefCompressionTableVersion = byteBuffer.get(ActorRefCompressionTableVersionOffset) + header._inboundClassManifestCompressionTableVersion = byteBuffer.get(ClassManifestCompressionTableVersionOffset) if (header.flag(MetadataPresentFlag)) { byteBuffer.position(MetadataContainerAndLiteralSectionOffset) 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 index 06da2785e3..88444bb7f5 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/CompressionTable.scala @@ -7,7 +7,11 @@ package akka.remote.artery.compress import java.util import java.util.Comparator -/** INTERNAL API: Versioned compression table to be advertised between systems */ +/** + * 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 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 index dfc6734f35..efb02181f1 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/DecompressionTable.scala @@ -4,7 +4,11 @@ package akka.remote.artery.compress -/** INTERNAL API */ +/** + * 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 @@ -27,6 +31,10 @@ private[remote] final case class DecompressionTable[T](originUid: Long, version: /** 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 index 9762fb8834..8e326371de 100644 --- a/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala +++ b/akka-remote/src/main/scala/akka/remote/artery/compress/InboundCompressions.scala @@ -234,7 +234,7 @@ private[remote] object InboundCompression { object State { def empty[T] = State( - oldTable = DecompressionTable.empty[T].copy(version = -1), + oldTable = DecompressionTable.disabled[T], activeTable = DecompressionTable.empty[T], nextTable = DecompressionTable.empty[T].copy(version = 1), advertisementInProgress = None) @@ -247,13 +247,14 @@ private[remote] object InboundCompression { advertisementInProgress: Option[CompressionTable[T]]) { def startUsingNextTable(): State[T] = { - // wrap around to positive values - val nextVersion = (nextTable.version + 1) & 0x7F + def incrementTableVersion(version: Byte): Byte = + if (version == 127) 0 + else (version + 1).toByte + State( oldTable = activeTable, activeTable = nextTable, - // skip 0 when wrapped around - nextTable = DecompressionTable.empty[T].copy(version = (if (nextVersion == 0) 1 else nextVersion).byteValue), + nextTable = DecompressionTable.empty[T].copy(version = incrementTableVersion(nextTable.version)), advertisementInProgress = None) } } @@ -317,7 +318,7 @@ private[remote] abstract class InboundCompression[T >: Null]( val oldVersion = current.oldTable.version val activeVersion = current.activeTable.version - if (incomingTableVersion == -1) OptionVal.None // no compression, bail out early + 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) @@ -327,11 +328,6 @@ private[remote] abstract class InboundCompression[T >: Null]( val value: T = current.oldTable.get(idx) if (value != null) OptionVal.Some[T](value) else throw new UnknownCompressedIdException(idx) - } else if (incomingTableVersion < activeVersion) { - log.debug( - "Received value from originUid [{}] compressed with old table: [{}], current table version is: [{}]", - originUid, incomingTableVersion, activeVersion) - OptionVal.None } 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: {})", @@ -343,7 +339,7 @@ private[remote] abstract class InboundCompression[T >: Null]( // 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 was probably sent with compression table built for previous incarnation of this system. " + + "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 diff --git a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala index b2c2be8b6b..bb1865762a 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/EnvelopeBufferSpec.scala @@ -33,10 +33,10 @@ class EnvelopeBufferSpec extends AkkaSpec { val idxToManifest = manifestToIdx.map(_.swap) val outboundActorRefTable: CompressionTable[ActorRef] = - CompressionTable(17L, version = 0xCA.byteValue, refToIdx) + CompressionTable(17L, version = 28.toByte, refToIdx) val outboundClassManifestTable: CompressionTable[String] = - CompressionTable(17L, version = 0xBA.byteValue, manifestToIdx) + 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)) @@ -78,8 +78,8 @@ class EnvelopeBufferSpec extends AkkaSpec { headerOut.version should ===(1) headerOut.uid should ===(42) - headerOut.inboundActorRefCompressionTableVersion should ===(0xCA.byteValue) - headerOut.inboundClassManifestCompressionTableVersion should ===(0xBA.byteValue) + 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) 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 index 497677317c..655b8f7401 100644 --- a/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/artery/compress/CompressionIntegrationSpec.scala @@ -281,7 +281,6 @@ class CompressionIntegrationSpec extends ArteryMultiNodeSpec(CompressionIntegrat val maxDuplicateTables = 40 // max duplicate tables that will not fail the test var tableVersionsSeen = 0 var lastTableVersion = 0 - var wrapAroundCount = 0 var iteration = 0 while (tableVersionsSeen < maxTableVersions) { @@ -303,12 +302,8 @@ class CompressionIntegrationSpec extends ArteryMultiNodeSpec(CompressionIntegrat if (currentTableVersion != lastTableVersion) { // if we get a new table lastTableVersion = currentTableVersion tableVersionsSeen += 1 - - if ((tableVersionsSeen & 0x7F) == 0) { - wrapAroundCount += 1 - } } - currentTableVersion should ===((tableVersionsSeen & 0x7F) + wrapAroundCount) + currentTableVersion should ===(tableVersionsSeen & 0x7F) } } From dcd921369cc34c5c34d58a86851620d5402d4334 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 29 Sep 2016 11:29:38 +0200 Subject: [PATCH 185/186] adjust build for master --- project/AkkaBuild.scala | 213 ++++++++++++++++++------------------- project/Dependencies.scala | 5 +- project/Doc.scala | 3 +- project/MiMa.scala | 44 ++++---- project/Release.scala | 12 +-- 5 files changed, 138 insertions(+), 139 deletions(-) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 0b4bcfad89..ef30823021 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -82,7 +82,7 @@ object AkkaBuild extends Build { protobuf, remote, remoteTests, -// samples, // FIXME temporary in artery-dev branch + samples, slf4j, stream, streamTestkit, @@ -98,37 +98,36 @@ object AkkaBuild extends Build { aggregate = aggregatedProjects ).settings(rootSettings: _*) -// FIXME temporary in artery-dev branch -// lazy val akkaScalaNightly = Project( -// id = "akka-scala-nightly", -// base = file("akka-scala-nightly"), -// // remove dependencies that we have to build ourselves (Scala STM) -// // samples don't work with dbuild right now -// aggregate = aggregatedProjects diff List(agent, docs, samples) -// ).disablePlugins(ValidatePullRequest, MimaPlugin) + lazy val akkaScalaNightly = Project( + id = "akka-scala-nightly", + base = file("akka-scala-nightly"), + // remove dependencies that we have to build ourselves (Scala STM) + // samples don't work with dbuild right now + aggregate = aggregatedProjects diff List(agent, docs, samples) + ).disablePlugins(ValidatePullRequest, MimaPlugin) lazy val actor = Project( id = "akka-actor", base = file("akka-actor") - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ) lazy val testkit = Project( id = "akka-testkit", base = file("akka-testkit"), dependencies = Seq(actor) - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ) lazy val typed = Project( id = "akka-typed-experimental", base = file("akka-typed"), dependencies = Seq(testkit % "compile;test->test") - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ) lazy val actorTests = Project( id = "akka-actor-tests", base = file("akka-actor-tests"), dependencies = Seq(testkit % "compile;test->test") - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ) lazy val benchJmh = Project( id = "akka-bench-jmh", @@ -136,7 +135,7 @@ object AkkaBuild extends Build { dependencies = Seq( actor, http, stream, streamTests, - remote, persistence, distributedData, + persistence, distributedData, testkit ).map(_ % "compile;compile->test;provided->provided") ).disablePlugins(ValidatePullRequest) @@ -144,7 +143,7 @@ object AkkaBuild extends Build { lazy val protobuf = Project( id = "akka-protobuf", base = file("akka-protobuf") - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ) lazy val remote = Project( id = "akka-remote", @@ -156,31 +155,31 @@ object AkkaBuild extends Build { id = "akka-multi-node-testkit", base = file("akka-multi-node-testkit"), dependencies = Seq(remote, testkit) - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ) lazy val remoteTests = Project( id = "akka-remote-tests", base = file("akka-remote-tests"), - dependencies = Seq(actorTests % "test->test", remote % "test->test", streamTestkit % "test", multiNodeTestkit) + dependencies = Seq(actorTests % "test->test", remote % "test->test", streamTestkit % "test", multiNodeTestkit) ).configs(MultiJvm) lazy val cluster = Project( id = "akka-cluster", base = file("akka-cluster"), dependencies = Seq(remote, remoteTests % "test->test" , testkit % "test->test") - ).configs(MultiJvm).disablePlugins(ValidatePullRequest) + ).configs(MultiJvm) lazy val clusterMetrics = Project( id = "akka-cluster-metrics", base = file("akka-cluster-metrics"), dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm", slf4j % "test->compile") - ).configs(MultiJvm).disablePlugins(ValidatePullRequest) + ).configs(MultiJvm) lazy val clusterTools = Project( id = "akka-cluster-tools", base = file("akka-cluster-tools"), dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm") - ).configs(MultiJvm).disablePlugins(ValidatePullRequest) + ).configs(MultiJvm) lazy val clusterSharding = Project( id = "akka-cluster-sharding", @@ -191,31 +190,31 @@ object AkkaBuild extends Build { // provided. dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm", persistence % "compile;test->provided", distributedData % "provided;test", clusterTools) - ).configs(MultiJvm).disablePlugins(ValidatePullRequest) + ).configs(MultiJvm) lazy val distributedData = Project( id = "akka-distributed-data-experimental", base = file("akka-distributed-data"), dependencies = Seq(cluster % "compile->compile;test->test;multi-jvm->multi-jvm") - ).configs(MultiJvm).disablePlugins(ValidatePullRequest) + ).configs(MultiJvm) lazy val slf4j = Project( id = "akka-slf4j", base = file("akka-slf4j"), dependencies = Seq(actor, testkit % "test->test") - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ) lazy val agent = Project( id = "akka-agent", base = file("akka-agent"), dependencies = Seq(actor, testkit % "test->test") - ).disablePlugins(ValidatePullRequest, MimaPlugin) + ) lazy val persistence = Project( id = "akka-persistence", base = file("akka-persistence"), dependencies = Seq(actor, testkit % "test->test", protobuf) - ).disablePlugins(ValidatePullRequest) + ) lazy val persistenceQuery = Project( id = "akka-persistence-query-experimental", @@ -225,37 +224,37 @@ object AkkaBuild extends Build { persistence % "compile;provided->provided;test->test", testkit % "compile;test->test", streamTestkit % "compile;test->test") - ).disablePlugins(ValidatePullRequest) + ) lazy val persistenceTck = Project( id = "akka-persistence-tck", base = file("akka-persistence-tck"), dependencies = Seq(persistence % "compile;provided->provided;test->test", testkit % "compile;test->test") - ).disablePlugins(ValidatePullRequest) + ) lazy val persistenceShared = Project( id = "akka-persistence-shared", base = file("akka-persistence-shared"), dependencies = Seq(persistence % "test->test", testkit % "test->test", remote % "test", protobuf) - ).disablePlugins(ValidatePullRequest) + ) lazy val httpCore = Project( id = "akka-http-core", base = file("akka-http-core"), dependencies = Seq(stream, parsing, streamTestkit % "test->test") - ).disablePlugins(ValidatePullRequest) + ) lazy val http = Project( id = "akka-http-experimental", base = file("akka-http"), dependencies = Seq(httpCore) - ).disablePlugins(ValidatePullRequest) + ) lazy val httpTestkit = Project( id = "akka-http-testkit", base = file("akka-http-testkit"), dependencies = Seq(http, streamTestkit) - ).disablePlugins(ValidatePullRequest) + ) lazy val httpTests = Project( id = "akka-http-tests", @@ -263,12 +262,12 @@ object AkkaBuild extends Build { dependencies = Seq( httpTestkit % "test", streamTestkit % "test->test", testkit % "test->test", httpSprayJson, httpXml, httpJackson, multiNodeTestkit, remoteTests % "test->test") // required for multi-node latency/throughput Spec - ).configs(MultiJvm).disablePlugins(ValidatePullRequest) + ).configs(MultiJvm) lazy val httpMarshallersScala = Project( id = "akka-http-marshallers-scala-experimental", base = file("akka-http-marshallers-scala") - ).disablePlugins(ValidatePullRequest) + ) .settings(parentSettings: _*) .aggregate(httpSprayJson, httpXml) @@ -281,7 +280,7 @@ object AkkaBuild extends Build { lazy val httpMarshallersJava = Project( id = "akka-http-marshallers-java-experimental", base = file("akka-http-marshallers-java") - ).disablePlugins(ValidatePullRequest) + ) .settings(parentSettings: _*) .aggregate(httpJackson) @@ -293,61 +292,61 @@ object AkkaBuild extends Build { id = s"akka-http-$name-experimental", base = file(s"akka-http-marshallers-scala/akka-http-$name"), dependencies = Seq(http) - ).disablePlugins(ValidatePullRequest) + ) def httpMarshallersJavaSubproject(name: String) = Project( id = s"akka-http-$name-experimental", base = file(s"akka-http-marshallers-java/akka-http-$name"), dependencies = Seq(http) - ).disablePlugins(ValidatePullRequest) + ) lazy val parsing = Project( id = "akka-parsing", base = file("akka-parsing") - ).disablePlugins(ValidatePullRequest) + ) lazy val stream = Project( id = "akka-stream", base = file("akka-stream"), dependencies = Seq(actor) - ).disablePlugins(ValidatePullRequest) + ) lazy val streamTestkit = Project( id = "akka-stream-testkit", base = file("akka-stream-testkit"), dependencies = Seq(stream, testkit % "compile;test->test") - ).disablePlugins(ValidatePullRequest) + ) lazy val streamTests = Project( id = "akka-stream-tests", base = file("akka-stream-tests"), dependencies = Seq(streamTestkit % "test->test", stream) - ).disablePlugins(ValidatePullRequest) + ) lazy val streamTestsTck = Project( id = "akka-stream-tests-tck", base = file("akka-stream-tests-tck"), dependencies = Seq(streamTestkit % "test->test", stream) - ).disablePlugins(ValidatePullRequest) + ) lazy val kernel = Project( id = "akka-kernel", base = file("akka-kernel"), dependencies = Seq(actor, testkit % "test->test") - ).disablePlugins(ValidatePullRequest) + ) lazy val camel = Project( id = "akka-camel", base = file("akka-camel"), dependencies = Seq(actor, slf4j, testkit % "test->test") - ).disablePlugins(ValidatePullRequest) + ) lazy val osgi = Project( id = "akka-osgi", base = file("akka-osgi"), dependencies = Seq(actor) - ).disablePlugins(ValidatePullRequest) + ) lazy val docs = Project( id = "akka-docs", @@ -368,73 +367,71 @@ object AkkaBuild extends Build { id = "akka-contrib", base = file("akka-contrib"), dependencies = Seq(remote, remoteTests % "test->test", cluster, clusterTools, persistence % "compile;test->provided") - ).configs(MultiJvm).disablePlugins(ValidatePullRequest) + ).configs(MultiJvm) lazy val samplesSettings = parentSettings ++ ActivatorDist.settings -// FIXME temporary in artery-dev branch -// lazy val samples = Project( -// id = "akka-samples", -// base = file("akka-samples"), -// // FIXME osgiDiningHakkersSampleMavenTest temporarily removed from aggregate due to #16703 -// aggregate = if (!Sample.CliOptions.aggregateSamples) Nil else -// Seq(sampleCamelJava, sampleCamelScala, sampleClusterJava, sampleClusterScala, sampleFsmScala, sampleFsmJavaLambda, -// sampleMainJava, sampleMainScala, sampleMainJavaLambda, sampleMultiNodeScala, -// samplePersistenceJava, samplePersistenceScala, samplePersistenceJavaLambda, -// sampleRemoteJava, sampleRemoteScala, sampleSupervisionJavaLambda, -// sampleDistributedDataScala, sampleDistributedDataJava) -// ) -// .settings(samplesSettings: _*) -// .disablePlugins(MimaPlugin) -// -// lazy val sampleCamelJava = Sample.project("akka-sample-camel-java") -// lazy val sampleCamelScala = Sample.project("akka-sample-camel-scala") -// -// lazy val sampleClusterJava = Sample.project("akka-sample-cluster-java") -// lazy val sampleClusterScala = Sample.project("akka-sample-cluster-scala") -// -// lazy val sampleFsmScala = Sample.project("akka-sample-fsm-scala") -// lazy val sampleFsmJavaLambda = Sample.project("akka-sample-fsm-java-lambda") -// -// lazy val sampleMainJava = Sample.project("akka-sample-main-java") -// lazy val sampleMainScala = Sample.project("akka-sample-main-scala") -// lazy val sampleMainJavaLambda = Sample.project("akka-sample-main-java-lambda") -// -// lazy val sampleMultiNodeScala = Sample.project("akka-sample-multi-node-scala") -// -// lazy val samplePersistenceJava = Sample.project("akka-sample-persistence-java") -// lazy val samplePersistenceScala = Sample.project("akka-sample-persistence-scala") -// lazy val samplePersistenceJavaLambda = Sample.project("akka-sample-persistence-java-lambda") -// -// lazy val sampleRemoteJava = Sample.project("akka-sample-remote-java") -// lazy val sampleRemoteScala = Sample.project("akka-sample-remote-scala") -// -// lazy val sampleSupervisionJavaLambda = Sample.project("akka-sample-supervision-java-lambda") -// -// lazy val sampleDistributedDataScala = Sample.project("akka-sample-distributed-data-scala") -// lazy val sampleDistributedDataJava = Sample.project("akka-sample-distributed-data-java") -// -// lazy val osgiDiningHakkersSampleMavenTest = Project( -// id = "akka-sample-osgi-dining-hakkers-maven-test", -// base = file("akka-samples/akka-sample-osgi-dining-hakkers-maven-test") -// ) -// .settings( -// publishArtifact := false, -// // force publication of artifacts to local maven repo, so latest versions can be used when running maven tests -// compile in Compile <<= -// (publishM2 in actor, publishM2 in testkit, publishM2 in remote, publishM2 in cluster, publishM2 in osgi, -// publishM2 in slf4j, publishM2 in persistence, compile in Compile) map -// ((_, _, _, _, _, _, _, c) => c), -// test in Test ~= { x => { -// def executeMvnCommands(failureMessage: String, commands: String*) = { -// if ({List("sh", "-c", commands.mkString("cd akka-samples/akka-sample-osgi-dining-hakkers; mvn ", " ", "")) !} != 0) -// throw new Exception(failureMessage) -// } -// executeMvnCommands("Osgi sample Dining hakkers test failed", "clean", "install") -// }} -// ) -// .disablePlugins(ValidatePullRequest, MimaPlugin) -// .settings(dontPublishSettings: _*) + lazy val samples = Project( + id = "akka-samples", + base = file("akka-samples"), + // FIXME osgiDiningHakkersSampleMavenTest temporarily removed from aggregate due to #16703 + aggregate = if (!Sample.CliOptions.aggregateSamples) Nil else + Seq(sampleCamelJava, sampleCamelScala, sampleClusterJava, sampleClusterScala, sampleFsmScala, sampleFsmJavaLambda, + sampleMainJava, sampleMainScala, sampleMainJavaLambda, sampleMultiNodeScala, + samplePersistenceJava, samplePersistenceScala, samplePersistenceJavaLambda, + sampleRemoteJava, sampleRemoteScala, sampleSupervisionJavaLambda, + sampleDistributedDataScala, sampleDistributedDataJava) + ) + .settings(samplesSettings: _*) + .disablePlugins(MimaPlugin) + + lazy val sampleCamelJava = Sample.project("akka-sample-camel-java") + lazy val sampleCamelScala = Sample.project("akka-sample-camel-scala") + + lazy val sampleClusterJava = Sample.project("akka-sample-cluster-java") + lazy val sampleClusterScala = Sample.project("akka-sample-cluster-scala") + + lazy val sampleFsmScala = Sample.project("akka-sample-fsm-scala") + lazy val sampleFsmJavaLambda = Sample.project("akka-sample-fsm-java-lambda") + + lazy val sampleMainJava = Sample.project("akka-sample-main-java") + lazy val sampleMainScala = Sample.project("akka-sample-main-scala") + lazy val sampleMainJavaLambda = Sample.project("akka-sample-main-java-lambda") + + lazy val sampleMultiNodeScala = Sample.project("akka-sample-multi-node-scala") + + lazy val samplePersistenceJava = Sample.project("akka-sample-persistence-java") + lazy val samplePersistenceScala = Sample.project("akka-sample-persistence-scala") + lazy val samplePersistenceJavaLambda = Sample.project("akka-sample-persistence-java-lambda") + + lazy val sampleRemoteJava = Sample.project("akka-sample-remote-java") + lazy val sampleRemoteScala = Sample.project("akka-sample-remote-scala") + + lazy val sampleSupervisionJavaLambda = Sample.project("akka-sample-supervision-java-lambda") + + lazy val sampleDistributedDataScala = Sample.project("akka-sample-distributed-data-scala") + lazy val sampleDistributedDataJava = Sample.project("akka-sample-distributed-data-java") + + lazy val osgiDiningHakkersSampleMavenTest = Project( + id = "akka-sample-osgi-dining-hakkers-maven-test", + base = file("akka-samples/akka-sample-osgi-dining-hakkers-maven-test") + ) + .settings( + publishArtifact := false, + // force publication of artifacts to local maven repo, so latest versions can be used when running maven tests + compile in Compile <<= + (publishM2 in actor, publishM2 in testkit, publishM2 in remote, publishM2 in cluster, publishM2 in osgi, + publishM2 in slf4j, publishM2 in persistence, compile in Compile) map + ((_, _, _, _, _, _, _, c) => c), + test in Test ~= { x => { + def executeMvnCommands(failureMessage: String, commands: String*) = { + if ({List("sh", "-c", commands.mkString("cd akka-samples/akka-sample-osgi-dining-hakkers; mvn ", " ", "")) !} != 0) + throw new Exception(failureMessage) + } + executeMvnCommands("Osgi sample Dining hakkers test failed", "clean", "install") + }} + ) + .settings(dontPublishSettings: _*) val dontPublishSettings = Seq( publishSigned := (), diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 247a42e198..43166b674c 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -16,7 +16,7 @@ object Dependencies { val junitVersion = "4.12" val Versions = Seq( - crossScalaVersions := Seq("2.11.8"), // "2.12.0-M4" + crossScalaVersions := Seq("2.11.8"), // "2.12.0-RC1" scalaVersion := crossScalaVersions.value.head, scalaStmVersion := sys.props.get("akka.build.scalaStmVersion").getOrElse("0.7"), scalaCheckVersion := sys.props.get("akka.build.scalaCheckVersion").getOrElse("1.13.2"), @@ -28,8 +28,7 @@ object Dependencies { }, java8CompatVersion := { scalaVersion.value match { - case "2.12.0-M4" => "0.8.0-RC1" - case "2.12.0-M5" => "0.8.0-RC3" + case x if x.startsWith("2.12.0-RC1") => "0.8.0-RC7" case _ => "0.7.0" } } diff --git a/project/Doc.scala b/project/Doc.scala index cb8b3b9b74..2294bd2315 100644 --- a/project/Doc.scala +++ b/project/Doc.scala @@ -118,10 +118,9 @@ object UnidocRoot extends AutoPlugin { )) } - // FIXME temporary removal of samples in artery-dev branch override lazy val projectSettings = CliOptions.genjavadocEnabled.ifTrue(scalaJavaUnidocSettings).getOrElse(scalaUnidocSettings) ++ - settings(Seq(), Seq(AkkaBuild.remoteTests, AkkaBuild.benchJmh, AkkaBuild.parsing, AkkaBuild.protobuf)) + settings(Seq(AkkaBuild.samples), Seq(AkkaBuild.remoteTests, AkkaBuild.benchJmh, AkkaBuild.parsing, AkkaBuild.protobuf, AkkaBuild.osgiDiningHakkersSampleMavenTest, AkkaBuild.akkaScalaNightly)) } /** diff --git a/project/MiMa.scala b/project/MiMa.scala index 59c076e4c3..6c392d243b 100644 --- a/project/MiMa.scala +++ b/project/MiMa.scala @@ -748,7 +748,6 @@ object MiMa extends AutoPlugin { ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.http.scaladsl.DefaultSSLContextCreation.validateAndWarnAboutLooseSettings") ), "2.4.4" -> Seq( - // #20080, #20081 remove race condition on HTTP client ProblemFilters.exclude[DirectMissingMethodProblem]("akka.http.scaladsl.Http#HostConnectionPool.gatewayFuture"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.http.scaladsl.Http#HostConnectionPool.copy"), @@ -849,19 +848,6 @@ object MiMa extends AutoPlugin { ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.cluster.client.ClusterClient.initialContactsSel") ), "2.4.6" -> Seq( - - // 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"), - // internal api FilterAnyProblemStartingWith("akka.stream.impl"), @@ -965,11 +951,6 @@ object MiMa extends AutoPlugin { // #21025 new orElse flow op ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOps.orElseGraph"), ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOps.orElse"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOpsMat.orElseMat") - ), - "2.4.10" -> Seq( - ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.RemoteActorRefProvider.quarantine"), - ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.RemoteWatcher.quarantine"), ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOpsMat.orElseMat"), // #21201 adding childActorOf to TestActor / TestKit / TestProbe @@ -983,11 +964,36 @@ object MiMa extends AutoPlugin { ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.http.scaladsl.model.ws.BinaryMessage.asScala"), ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.http.scaladsl.model.ws.BinaryMessage.getStreamedData"), + // #21131 new implementation for Akka Typed + ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.actor.dungeon.DeathWatch.isWatching") + ), + "2.4.10" -> Seq( + // #21290 new zipWithIndex flow op + 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") + + ) ) } diff --git a/project/Release.scala b/project/Release.scala index dd2d72830e..33c898faed 100644 --- a/project/Release.scala +++ b/project/Release.scala @@ -33,8 +33,7 @@ object Release { val (state2, Seq(api, japi)) = extracted.runTask(unidoc in Compile, state1) val (state3, docs) = extracted.runTask(generate in Sphinx, state2) val (state4, _) = extracted.runTask(Dist.dist, state3) -// FIXME temporary in artery-dev branch -// val (state5, activatorDist) = extracted.runTask(ActivatorDist.activatorDist in LocalProject(AkkaBuild.samples.id), state4) + val (state5, activatorDist) = extracted.runTask(ActivatorDist.activatorDist in LocalProject(AkkaBuild.samples.id), state4) IO.delete(release) IO.createDirectory(release) @@ -48,11 +47,10 @@ object Release { for (f <- (dist * "akka_*.zip").get) IO.copyFile(f, release / "downloads" / f.name) -// FIXME temporary in artery-dev branch -// for (f <- (activatorDist * "*.zip").get) -// IO.copyFile(f, release / "downloads" / f.name) -// state5 - state4 + for (f <- (activatorDist * "*.zip").get) + IO.copyFile(f, release / "downloads" / f.name) + + state5 } def uploadReleaseCommand = Command.command("uploadRelease") { state => From 5558caa1f016003ec7b4d33ce5e8f82106dcd020 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 29 Sep 2016 13:29:17 +0200 Subject: [PATCH 186/186] revert Artery config in akka-sample-cluster-scala --- .../src/main/resources/application.conf | 18 +++++++++--------- .../cluster/factorial/FactorialBackend.scala | 2 +- .../cluster/simple/SimpleClusterApp.scala | 2 +- .../sample/cluster/stats/StatsSample.scala | 2 +- .../cluster/stats/StatsSampleOneMaster.scala | 2 +- .../transformation/TransformationBackend.scala | 2 +- .../TransformationFrontend.scala | 2 +- .../stats/StatsSampleSingleMasterSpec.scala | 2 +- .../sample/cluster/stats/StatsSampleSpec.scala | 2 +- .../TransformationSampleSpec.scala | 2 +- .../tutorial/index.html | 2 +- 11 files changed, 19 insertions(+), 19 deletions(-) diff --git a/akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf b/akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf index 86559812a1..676a223350 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf +++ b/akka-samples/akka-sample-cluster-scala/src/main/resources/application.conf @@ -1,19 +1,20 @@ #//#snippet akka { - actor { - provider = "cluster" + provider = "akka.cluster.ClusterActorRefProvider" } - remote.artery { - enabled = on - canonical.hostname = 127.0.0.1 - canonical.port = 0 + remote { + log-remote-lifecycle-events = off + netty.tcp { + hostname = "127.0.0.1" + port = 0 + } } cluster { seed-nodes = [ - "akka://ClusterSystem@127.0.0.1:2551", - "akka://ClusterSystem@127.0.0.1:2552"] + "akka.tcp://ClusterSystem@127.0.0.1:2551", + "akka.tcp://ClusterSystem@127.0.0.1:2552"] #//#snippet # excluded from snippet @@ -36,4 +37,3 @@ akka.extensions=["akka.cluster.metrics.ClusterMetricsExtension"] # Note: use per-jvm-instance folder when running multiple jvm on one host. akka.cluster.metrics.native-library-extract-folder=${user.dir}/target/native #//#snippet - diff --git a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/FactorialBackend.scala b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/FactorialBackend.scala index e6629d73a0..11ee8c2074 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/FactorialBackend.scala +++ b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/FactorialBackend.scala @@ -34,7 +34,7 @@ object FactorialBackend { def main(args: Array[String]): Unit = { // Override the configuration of the port when specified as program argument val port = if (args.isEmpty) "0" else args(0) - val config = ConfigFactory.parseString(s"akka.remote.artery.canonical.port=$port"). + val config = ConfigFactory.parseString(s"akka.remote.netty.tcp.port=$port"). withFallback(ConfigFactory.parseString("akka.cluster.roles = [backend]")). withFallback(ConfigFactory.load("factorial")) diff --git a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/simple/SimpleClusterApp.scala b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/simple/SimpleClusterApp.scala index 1f22869117..1e87f49d25 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/simple/SimpleClusterApp.scala +++ b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/simple/SimpleClusterApp.scala @@ -15,7 +15,7 @@ object SimpleClusterApp { def startup(ports: Seq[String]): Unit = { ports foreach { port => // Override the configuration of the port - val config = ConfigFactory.parseString("akka.remote.artery.canonical.port=" + port). + val config = ConfigFactory.parseString("akka.remote.netty.tcp.port=" + port). withFallback(ConfigFactory.load()) // Create an Akka system diff --git a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSample.scala b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSample.scala index 019d046bac..b32dd961a8 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSample.scala +++ b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSample.scala @@ -28,7 +28,7 @@ object StatsSample { ports foreach { port => // Override the configuration of the port when specified as program argument val config = - ConfigFactory.parseString(s"akka.remote.artery.canonical.port=" + port).withFallback( + ConfigFactory.parseString(s"akka.remote.netty.tcp.port=" + port).withFallback( ConfigFactory.parseString("akka.cluster.roles = [compute]")). withFallback(ConfigFactory.load("stats1")) diff --git a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSampleOneMaster.scala b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSampleOneMaster.scala index 7cda2d89df..f4db5403be 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSampleOneMaster.scala +++ b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/stats/StatsSampleOneMaster.scala @@ -23,7 +23,7 @@ object StatsSampleOneMaster { ports foreach { port => // Override the configuration of the port when specified as program argument val config = - ConfigFactory.parseString(s"akka.remote.artery.canonical.port=" + port).withFallback( + ConfigFactory.parseString(s"akka.remote.netty.tcp.port=" + port).withFallback( ConfigFactory.parseString("akka.cluster.roles = [compute]")). withFallback(ConfigFactory.load("stats2")) diff --git a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationBackend.scala b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationBackend.scala index d7893fad5f..2d164b4bb0 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationBackend.scala +++ b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationBackend.scala @@ -42,7 +42,7 @@ object TransformationBackend { def main(args: Array[String]): Unit = { // Override the configuration of the port when specified as program argument val port = if (args.isEmpty) "0" else args(0) - val config = ConfigFactory.parseString(s"akka.remote.artery.canonical.port=$port"). + val config = ConfigFactory.parseString(s"akka.remote.netty.tcp.port=$port"). withFallback(ConfigFactory.parseString("akka.cluster.roles = [backend]")). withFallback(ConfigFactory.load()) diff --git a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationFrontend.scala b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationFrontend.scala index e4225106d7..deb34d5158 100644 --- a/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationFrontend.scala +++ b/akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/transformation/TransformationFrontend.scala @@ -40,7 +40,7 @@ object TransformationFrontend { def main(args: Array[String]): Unit = { // Override the configuration of the port when specified as program argument val port = if (args.isEmpty) "0" else args(0) - val config = ConfigFactory.parseString(s"akka.remote.artery.canonical.port=$port"). + val config = ConfigFactory.parseString(s"akka.remote.netty.tcp.port=$port"). withFallback(ConfigFactory.parseString("akka.cluster.roles = [frontend]")). withFallback(ConfigFactory.load()) diff --git a/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala b/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala index dbbdd32e11..e22a1a281f 100644 --- a/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala +++ b/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala @@ -48,7 +48,7 @@ object StatsSampleSingleMasterSpecConfig extends MultiNodeConfig { // note that no fixed host names and ports are used commonConfig(ConfigFactory.parseString(""" akka.loglevel = INFO - akka.actor.provider = "cluster" + akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.remote.log-remote-lifecycle-events = off akka.cluster.roles = [compute] #//#router-deploy-config diff --git a/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala b/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala index 9421e17cbc..dba0965de2 100644 --- a/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala +++ b/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala @@ -40,7 +40,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 = "cluster" + akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.remote.log-remote-lifecycle-events = off akka.cluster.roles = [compute] #//#router-lookup-config diff --git a/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala b/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala index 0eb2469d1a..cd9324181c 100644 --- a/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala +++ b/akka-samples/akka-sample-cluster-scala/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala @@ -42,7 +42,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 = "cluster" + akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.remote.log-remote-lifecycle-events = off """)) diff --git a/akka-samples/akka-sample-cluster-scala/tutorial/index.html b/akka-samples/akka-sample-cluster-scala/tutorial/index.html index 0567e6685f..1628cb5970 100644 --- a/akka-samples/akka-sample-cluster-scala/tutorial/index.html +++ b/akka-samples/akka-sample-cluster-scala/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 cluster for akka.actor.provider. The akka.cluster.seed-nodes should +and use akka.cluster.ClusterActorRefProvider. The akka.cluster.seed-nodes should normally also be added to your application.conf file.