2016-04-19 17:38:26 +02:00
|
|
|
/**
|
2018-01-04 17:26:29 +00:00
|
|
|
* Copyright (C) 2016-2018 Lightbend Inc. <https://www.lightbend.com>
|
2016-04-19 17:38:26 +02:00
|
|
|
*/
|
|
|
|
|
package akka.remote.artery
|
2017-10-21 13:35:05 +01:00
|
|
|
package aeron
|
2016-04-19 17:38:26 +02:00
|
|
|
|
2016-06-15 15:12:33 +02:00
|
|
|
import java.io.File
|
2016-04-19 17:38:26 +02:00
|
|
|
import java.util.concurrent.CyclicBarrier
|
|
|
|
|
import java.util.concurrent.Executors
|
2016-06-15 15:12:33 +02:00
|
|
|
import java.util.concurrent.atomic.AtomicBoolean
|
2016-04-19 17:38:26 +02:00
|
|
|
import java.util.concurrent.atomic.AtomicInteger
|
2016-06-15 15:12:33 +02:00
|
|
|
import java.util.concurrent.atomic.AtomicLong
|
2016-04-19 17:38:26 +02:00
|
|
|
import java.util.concurrent.atomic.AtomicLongArray
|
2016-06-15 15:12:33 +02:00
|
|
|
import java.util.concurrent.locks.LockSupport
|
2016-05-05 14:38:48 +02:00
|
|
|
|
2016-04-19 17:38:26 +02:00
|
|
|
import scala.concurrent.duration._
|
2016-06-15 15:12:33 +02:00
|
|
|
|
|
|
|
|
import akka.Done
|
2016-04-19 17:38:26 +02:00
|
|
|
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
|
2016-06-15 15:12:33 +02:00
|
|
|
import akka.stream.KillSwitches
|
2016-04-19 17:38:26 +02:00
|
|
|
import akka.stream.ThrottleMode
|
2016-06-15 15:12:33 +02:00
|
|
|
import akka.stream.scaladsl.Flow
|
|
|
|
|
import akka.stream.scaladsl.Keep
|
2016-04-19 17:38:26 +02:00
|
|
|
import akka.stream.scaladsl.Source
|
|
|
|
|
import akka.testkit._
|
2016-06-15 15:12:33 +02:00
|
|
|
import akka.util.ByteString
|
2016-04-19 17:38:26 +02:00
|
|
|
import com.typesafe.config.ConfigFactory
|
|
|
|
|
import io.aeron.Aeron
|
2016-06-15 15:12:33 +02:00
|
|
|
import io.aeron.CncFileDescriptor
|
2016-04-19 17:38:26 +02:00
|
|
|
import io.aeron.driver.MediaDriver
|
|
|
|
|
import org.HdrHistogram.Histogram
|
2016-05-03 16:44:33 +02:00
|
|
|
import org.agrona.IoUtil
|
2016-06-15 15:12:33 +02:00
|
|
|
import org.agrona.concurrent.ManyToOneConcurrentArrayQueue
|
2016-04-19 17:38:26 +02:00
|
|
|
|
|
|
|
|
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 {
|
2016-06-10 15:04:13 +02:00
|
|
|
provider = remote
|
2016-04-19 17:38:26 +02:00
|
|
|
serialize-creators = false
|
|
|
|
|
serialize-messages = false
|
|
|
|
|
}
|
2016-06-10 07:41:36 +02:00
|
|
|
remote.artery {
|
|
|
|
|
enabled = off
|
|
|
|
|
advanced.idle-cpu-level=8
|
|
|
|
|
}
|
2016-04-19 17:38:26 +02:00
|
|
|
}
|
|
|
|
|
""")))
|
|
|
|
|
|
|
|
|
|
final case class TestSettings(
|
2016-06-03 11:59:00 +02:00
|
|
|
testName: String,
|
2016-04-19 17:38:26 +02:00
|
|
|
messageRate: Int, // msg/s
|
|
|
|
|
payloadSize: Int,
|
2016-06-03 11:59:00 +02:00
|
|
|
repeat: Int)
|
2016-04-19 17:38:26 +02:00
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
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")
|
|
|
|
|
|
2016-04-22 16:24:12 +02:00
|
|
|
var plots = LatencyPlots()
|
|
|
|
|
|
2016-05-03 16:44:33 +02:00
|
|
|
val driver = MediaDriver.launchEmbedded()
|
|
|
|
|
|
2016-08-30 14:37:11 +02:00
|
|
|
val pool = new EnvelopeBufferPool(1024 * 1024, 128)
|
2016-05-05 14:38:48 +02:00
|
|
|
|
2017-10-02 15:23:04 +01:00
|
|
|
val cncByteBuffer = IoUtil.mapExistingFile(new File(driver.aeronDirectoryName, CncFileDescriptor.CNC_FILE), "cnc")
|
2016-04-25 08:40:58 +02:00
|
|
|
val stats =
|
2016-09-09 14:29:04 +02:00
|
|
|
new AeronStat(AeronStat.mapCounters(cncByteBuffer))
|
2016-04-25 08:40:58 +02:00
|
|
|
|
2016-04-19 17:38:26 +02:00
|
|
|
val aeron = {
|
|
|
|
|
val ctx = new Aeron.Context
|
|
|
|
|
ctx.aeronDirectoryName(driver.aeronDirectoryName)
|
|
|
|
|
Aeron.connect(ctx)
|
|
|
|
|
}
|
|
|
|
|
|
2016-06-10 07:41:36 +02:00
|
|
|
val idleCpuLevel = system.settings.config.getInt("akka.remote.artery.advanced.idle-cpu-level")
|
2016-05-03 16:44:33 +02:00
|
|
|
val taskRunner = {
|
2016-06-10 07:41:36 +02:00
|
|
|
val r = new TaskRunner(system.asInstanceOf[ExtendedActorSystem], idleCpuLevel)
|
2016-05-03 16:44:33 +02:00
|
|
|
r.start()
|
|
|
|
|
r
|
|
|
|
|
}
|
|
|
|
|
|
2016-04-19 17:38:26 +02:00
|
|
|
lazy implicit val mat = ActorMaterializer()(system)
|
|
|
|
|
|
|
|
|
|
override def initialParticipants = roles.size
|
|
|
|
|
|
|
|
|
|
def channel(roleName: RoleName) = {
|
2018-01-16 16:49:28 +01:00
|
|
|
val n = node(roleName)
|
|
|
|
|
system.actorSelection(n / "user" / "updPort") ! UdpPortActor.GetUdpPort
|
|
|
|
|
val port = expectMsgType[Int]
|
|
|
|
|
s"aeron:udp?endpoint=${n.address.host.get}:$port"
|
2016-04-19 17:38:26 +02:00
|
|
|
}
|
|
|
|
|
|
2016-05-04 13:31:08 +02:00
|
|
|
val streamId = 1
|
2016-09-09 07:45:21 +02:00
|
|
|
val giveUpMessageAfter = 30.seconds
|
2016-05-04 13:31:08 +02:00
|
|
|
|
2016-04-19 17:38:26 +02:00
|
|
|
lazy val reporterExecutor = Executors.newFixedThreadPool(1)
|
2016-05-03 21:16:30 +02:00
|
|
|
def reporter(name: String): TestRateReporter = {
|
|
|
|
|
val r = new TestRateReporter(name)
|
2016-04-19 17:38:26 +02:00
|
|
|
reporterExecutor.execute(r)
|
|
|
|
|
r
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def afterAll(): Unit = {
|
|
|
|
|
reporterExecutor.shutdown()
|
2016-05-03 16:44:33 +02:00
|
|
|
taskRunner.stop()
|
|
|
|
|
aeron.close()
|
2016-05-04 13:31:08 +02:00
|
|
|
driver.close()
|
2016-09-09 14:29:04 +02:00
|
|
|
IoUtil.unmap(cncByteBuffer)
|
2016-05-03 16:44:33 +02:00
|
|
|
IoUtil.delete(new File(driver.aeronDirectoryName), true)
|
2016-04-22 16:24:12 +02:00
|
|
|
runOn(first) {
|
|
|
|
|
println(plots.plot50.csv(system.name + "50"))
|
|
|
|
|
println(plots.plot90.csv(system.name + "90"))
|
|
|
|
|
println(plots.plot99.csv(system.name + "99"))
|
|
|
|
|
}
|
2016-04-19 17:38:26 +02:00
|
|
|
super.afterAll()
|
|
|
|
|
}
|
|
|
|
|
|
2016-06-15 15:12:33 +02:00
|
|
|
def printTotal(testName: String, payloadSize: Long, histogram: Histogram, totalDurationNanos: Long, lastRepeat: Boolean): Unit = {
|
2017-06-08 07:28:22 +02:00
|
|
|
def percentile(p: Double): Double = histogram.getValueAtPercentile(p) / 1000.0
|
2016-06-15 15:12:33 +02:00
|
|
|
val throughput = 1000.0 * histogram.getTotalCount / totalDurationNanos.nanos.toMillis
|
|
|
|
|
|
2016-04-19 17:38:26 +02:00
|
|
|
println(s"=== AeronStreamLatency $testName: RTT " +
|
|
|
|
|
f"50%%ile: ${percentile(50.0)}%.0f µs, " +
|
|
|
|
|
f"90%%ile: ${percentile(90.0)}%.0f µs, " +
|
2016-06-15 15:12:33 +02:00
|
|
|
f"99%%ile: ${percentile(99.0)}%.0f µs, " +
|
|
|
|
|
f"rate: ${throughput}%,.0f msg/s")
|
2016-04-19 17:38:26 +02:00
|
|
|
println("Histogram of RTT latencies in microseconds.")
|
|
|
|
|
histogram.outputPercentileDistribution(System.out, 1000.0)
|
2016-04-22 16:24:12 +02:00
|
|
|
|
|
|
|
|
// 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)))
|
|
|
|
|
}
|
2016-04-19 17:38:26 +02:00
|
|
|
}
|
|
|
|
|
|
2016-04-25 08:40:58 +02:00
|
|
|
def printStats(side: String): Unit = {
|
|
|
|
|
println(side + " stats:")
|
|
|
|
|
stats.print(System.out)
|
|
|
|
|
}
|
|
|
|
|
|
2017-11-20 15:15:17 +01:00
|
|
|
def sendToDeadLetters[T](pending: Vector[T]): Unit =
|
|
|
|
|
pending.foreach(system.deadLetters ! _)
|
|
|
|
|
|
2016-04-19 17:38:26 +02:00
|
|
|
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),
|
2016-06-15 15:12:33 +02:00
|
|
|
TestSettings(
|
|
|
|
|
testName = "rate-20000-size-100",
|
|
|
|
|
messageRate = 20000,
|
|
|
|
|
payloadSize = 100,
|
|
|
|
|
repeat = repeatCount),
|
2016-04-19 17:38:26 +02:00
|
|
|
TestSettings(
|
|
|
|
|
testName = "rate-1000-size-1k",
|
|
|
|
|
messageRate = 1000,
|
|
|
|
|
payloadSize = 1000,
|
|
|
|
|
repeat = repeatCount))
|
|
|
|
|
|
|
|
|
|
def test(testSettings: TestSettings): Unit = {
|
|
|
|
|
import testSettings._
|
|
|
|
|
|
|
|
|
|
runOn(first) {
|
2016-05-03 16:44:33 +02:00
|
|
|
val payload = ("1" * payloadSize).getBytes("utf-8")
|
2016-04-19 17:38:26 +02:00
|
|
|
// 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
|
2016-06-15 15:12:33 +02:00
|
|
|
val startTime = new AtomicLong
|
2016-04-22 16:24:12 +02:00
|
|
|
val lastRepeat = new AtomicBoolean(false)
|
2016-05-03 16:44:33 +02:00
|
|
|
val killSwitch = KillSwitches.shared(testName)
|
|
|
|
|
val started = TestProbe()
|
|
|
|
|
val startMsg = "0".getBytes("utf-8")
|
2016-12-30 15:05:21 +01:00
|
|
|
Source.fromGraph(new AeronSource(channel(first), streamId, aeron, taskRunner, pool, IgnoreEventSink, 0))
|
2016-05-03 16:44:33 +02:00
|
|
|
.via(killSwitch.flow)
|
2016-05-05 14:38:48 +02:00
|
|
|
.runForeach { envelope ⇒
|
|
|
|
|
val bytes = ByteString.fromByteBuffer(envelope.byteBuffer)
|
2016-05-03 16:44:33 +02:00
|
|
|
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) {
|
2016-06-15 15:12:33 +02:00
|
|
|
val totalDurationNanos = System.nanoTime() - startTime.get
|
|
|
|
|
printTotal(testName, bytes.length, histogram, totalDurationNanos, lastRepeat.get)
|
2016-05-03 16:44:33 +02:00
|
|
|
barrier.await() // this is always the last party
|
|
|
|
|
}
|
2016-04-19 17:38:26 +02:00
|
|
|
}
|
2016-05-05 14:38:48 +02:00
|
|
|
pool.release(envelope)
|
2016-04-19 17:38:26 +02:00
|
|
|
}
|
|
|
|
|
|
2016-05-03 16:44:33 +02:00
|
|
|
within(10.seconds) {
|
2016-05-05 14:38:48 +02:00
|
|
|
Source(1 to 50).map { _ ⇒
|
|
|
|
|
val envelope = pool.acquire()
|
|
|
|
|
envelope.byteBuffer.put(startMsg)
|
|
|
|
|
envelope.byteBuffer.flip()
|
|
|
|
|
envelope
|
|
|
|
|
}
|
2016-05-03 16:44:33 +02:00
|
|
|
.throttle(1, 200.milliseconds, 1, ThrottleMode.Shaping)
|
2016-09-09 07:45:21 +02:00
|
|
|
.runWith(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpMessageAfter, IgnoreEventSink))
|
2016-05-03 16:44:33 +02:00
|
|
|
started.expectMsg(Done)
|
|
|
|
|
}
|
|
|
|
|
|
2016-06-15 15:12:33 +02:00
|
|
|
for (rep ← 1 to repeat) {
|
2016-04-19 17:38:26 +02:00
|
|
|
histogram.reset()
|
|
|
|
|
count.set(0)
|
2016-06-15 15:12:33 +02:00
|
|
|
lastRepeat.set(rep == repeat)
|
2016-04-19 17:38:26 +02:00
|
|
|
|
2016-06-15 15:12:33 +02:00
|
|
|
val sendFlow = Flow[Unit]
|
|
|
|
|
.map { _ ⇒
|
2016-05-05 14:38:48 +02:00
|
|
|
val envelope = pool.acquire()
|
|
|
|
|
envelope.byteBuffer.put(payload)
|
|
|
|
|
envelope.byteBuffer.flip()
|
|
|
|
|
envelope
|
2016-04-19 17:38:26 +02:00
|
|
|
}
|
2016-06-15 15:12:33 +02:00
|
|
|
|
2017-11-20 15:15:17 +01:00
|
|
|
val queueValue = Source.fromGraph(new SendQueue[Unit](sendToDeadLetters))
|
2016-06-15 15:12:33 +02:00
|
|
|
.via(sendFlow)
|
2016-09-09 07:45:21 +02:00
|
|
|
.to(new AeronSink(channel(second), streamId, aeron, taskRunner, pool, giveUpMessageAfter, IgnoreEventSink))
|
2016-06-15 15:12:33 +02:00
|
|
|
.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
|
|
|
|
|
}
|
2016-04-19 17:38:26 +02:00
|
|
|
|
|
|
|
|
barrier.await((totalMessages / messageRate) + 10, SECONDS)
|
|
|
|
|
}
|
|
|
|
|
|
2016-05-03 16:44:33 +02:00
|
|
|
killSwitch.shutdown()
|
2016-04-19 17:38:26 +02:00
|
|
|
rep.halt()
|
|
|
|
|
}
|
|
|
|
|
|
2016-04-25 08:40:58 +02:00
|
|
|
printStats(myself.name)
|
2016-04-19 17:38:26 +02:00
|
|
|
enterBarrier("after-" + testName)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"Latency of Aeron Streams" must {
|
|
|
|
|
|
2018-01-16 16:49:28 +01:00
|
|
|
"start upd port" in {
|
|
|
|
|
system.actorOf(Props[UdpPortActor], "updPort")
|
|
|
|
|
enterBarrier("udp-port-started")
|
|
|
|
|
}
|
|
|
|
|
|
2016-04-19 17:38:26 +02:00
|
|
|
"start echo" in {
|
|
|
|
|
runOn(second) {
|
|
|
|
|
// just echo back
|
2016-12-30 15:05:21 +01:00
|
|
|
Source.fromGraph(new AeronSource(channel(second), streamId, aeron, taskRunner, pool, IgnoreEventSink, 0))
|
2016-09-09 07:45:21 +02:00
|
|
|
.runWith(new AeronSink(channel(first), streamId, aeron, taskRunner, pool, giveUpMessageAfter, IgnoreEventSink))
|
2016-04-19 17:38:26 +02:00
|
|
|
}
|
|
|
|
|
enterBarrier("echo-started")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
for (s ← scenarios) {
|
|
|
|
|
s"be low for ${s.testName}, at ${s.messageRate} msg/s, payloadSize = ${s.payloadSize}" in test(s)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
}
|