Rename sbt akka modules

Co-authored-by: Sean Glover <sean@seanglover.com>
This commit is contained in:
Matthew de Detrich 2023-01-05 11:10:50 +01:00 committed by GitHub
parent b92b749946
commit 24c03cde19
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
2930 changed files with 1466 additions and 1462 deletions

17
bench-jmh/README.md Normal file
View file

@ -0,0 +1,17 @@
# Akka Microbenchmarks
This subproject contains some microbenchmarks excercising key parts of Akka. (Excluding typed which has its
own jmh module)
You can run them like:
project bench-jmh
jmh:run -i 3 -wi 3 -f 1 .*ActorCreationBenchmark
Use 'jmh:run -h' to get an overview of the available options.
Some potentially out of date resources for writing JMH benchmarks:
* [Studying what's wrong with JMH benchmarks](https://www.researchgate.net/publication/333825812_What's_Wrong_With_My_Benchmark_Results_Studying_Bad_Practices_in_JMH_Benchmarks)
* [Writing good benchmarks](http://tutorials.jenkov.com/java-performance/jmh.html#writing-good-benchmarks)

View file

@ -0,0 +1,95 @@
/*
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.serialization.jackson;
import java.time.Duration;
import java.time.Instant;
import java.time.LocalDateTime;
import java.util.List;
import java.util.Map;
public class JavaMessages {
interface JTestMessage {}
public static class JSmall implements JTestMessage {
public final String name;
public final int num;
public JSmall(String name, int num) {
this.name = name;
this.num = num;
}
}
public static class JMedium implements JTestMessage {
public final String field1;
public final String field2;
public final String field3;
public final int num1;
public final int num2;
public final int num3;
public final boolean flag1;
public final boolean flag2;
public final Duration duration;
public final LocalDateTime date;
public final Instant instant;
public final JSmall nested1;
public final JSmall nested2;
public final JSmall nested3;
public JMedium(
String field1,
String field2,
String field3,
int num1,
int num2,
int num3,
boolean flag1,
boolean flag2,
Duration duration,
LocalDateTime date,
Instant instant,
JSmall nested1,
JSmall nested2,
JSmall nested3) {
this.field1 = field1;
this.field2 = field2;
this.field3 = field3;
this.num1 = num1;
this.num2 = num2;
this.num3 = num3;
this.flag1 = flag1;
this.flag2 = flag2;
this.duration = duration;
this.date = date;
this.instant = instant;
this.nested1 = nested1;
this.nested2 = nested2;
this.nested3 = nested3;
}
}
public static class JLarge implements JTestMessage {
public final JMedium nested1;
public final JMedium nested2;
public final JMedium nested3;
public final List<JMedium> list;
public final Map<String, JMedium> map;
public JLarge(
JMedium nested1,
JMedium nested2,
JMedium nested3,
List<JMedium> list,
Map<String, JMedium> map) {
this.nested1 = nested1;
this.nested2 = nested2;
this.nested3 = nested3;
this.list = list;
this.map = map;
}
}
}

View file

@ -0,0 +1,14 @@
<?xml version="1.0" encoding="UTF-8"?>
<configuration>
<!-- Silence initial setup logging from Logback -->
<statusListener class="ch.qos.logback.core.status.NopStatusListener" />
<appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
<encoder>
<pattern>%date{ISO8601} %-5level %logger %marker - %msg MDC: {%mdc}%n</pattern>
</encoder>
</appender>
<root level="INFO">
<appender-ref ref="STDOUT"/>
</root>
</configuration>

View file

@ -0,0 +1,36 @@
/*
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko
import org.openjdk.jmh.results.RunResult
import org.openjdk.jmh.runner.Runner
import org.openjdk.jmh.runner.options.CommandLineOptions
object BenchRunner {
def main(args: Array[String]) = {
import org.apache.pekko.util.ccompat.JavaConverters._
val args2 = args.toList.flatMap {
case "quick" => "-i 1 -wi 1 -f1 -t1".split(" ").toList
case "full" => "-i 10 -wi 4 -f3 -t1".split(" ").toList
case "jitwatch" => "-jvmArgs=-XX:+UnlockDiagnosticVMOptions -XX:+TraceClassLoading -XX:+LogCompilation" :: Nil
case other => other :: Nil
}
val opts = new CommandLineOptions(args2: _*)
val results = new Runner(opts).run()
val report = results.asScala.map { (result: RunResult) =>
val bench = result.getParams.getBenchmark
val params =
result.getParams.getParamsKeys.asScala.map(key => s"$key=${result.getParams.getParam(key)}").mkString("_")
val score = result.getAggregatedResult.getPrimaryResult.getScore.round
val unit = result.getAggregatedResult.getPrimaryResult.getScoreUnit
s"\t${bench}_${params}\t$score\t$unit"
}
report.toList.sorted.foreach(println)
}
}

View file

@ -0,0 +1,103 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import BenchmarkActors._
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
object ActorBenchmark {
// Constants because they are used in annotations
final val threads = 8 // update according to cpu
final val numMessagesPerActorPair = 1000000 // messages per actor pair
final val numActors = 512
final val totalMessages = numMessagesPerActorPair * numActors / 2
}
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(1)
@Threads(1)
@Warmup(iterations = 10, time = 5, timeUnit = TimeUnit.SECONDS, batchSize = 1)
@Measurement(iterations = 10, time = 15, timeUnit = TimeUnit.SECONDS, batchSize = 1)
class ActorBenchmark {
import ActorBenchmark._
@Param(Array("50"))
var tpt = 0
@Param(Array("50"))
var batchSize = 0
// @Param(Array("org.apache.pekko.actor.ManyToOneArrayMailbox"))
@Param(
Array(
"org.apache.pekko.dispatch.SingleConsumerOnlyUnboundedMailbox",
"org.apache.pekko.actor.ManyToOneArrayMailbox",
"org.apache.pekko.actor.JCToolsMailbox"))
var mailbox = ""
@Param(Array("fjp-dispatcher")) // @Param(Array("fjp-dispatcher", "affinity-dispatcher"))
var dispatcher = ""
implicit var system: ActorSystem = _
@Setup(Level.Trial)
def setup(): Unit = {
requireRightNumberOfCores(threads)
system = ActorSystem(
"ActorBenchmark",
ConfigFactory.parseString(s"""
pekko.actor {
default-mailbox.mailbox-capacity = 512
fjp-dispatcher {
executor = "fork-join-executor"
fork-join-executor {
parallelism-min = $threads
parallelism-factor = 1.0
parallelism-max = $threads
}
throughput = $tpt
mailbox-type = "$mailbox"
}
affinity-dispatcher {
executor = "affinity-pool-executor"
affinity-pool-executor {
parallelism-min = $threads
parallelism-factor = 1.0
parallelism-max = $threads
task-queue-size = 512
idle-cpu-level = 5
fair-work-distribution.threshold = 2048
}
throughput = $tpt
mailbox-type = "$mailbox"
}
}
"""))
}
@TearDown(Level.Trial)
def shutdown(): Unit = {
system.terminate()
Await.ready(system.whenTerminated, 15.seconds)
}
@Benchmark
@OperationsPerInvocation(totalMessages)
def echo(): Unit =
benchmarkEchoActors(numMessagesPerActorPair, numActors, dispatcher, batchSize, timeout)
}

View file

@ -0,0 +1,55 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import org.openjdk.jmh.annotations._
/*
regex checking:
[info] a.a.ActorCreationBenchmark.synchronousStarting ss 120000 28.285 0.481 us
hand checking:
[info] a.a.ActorCreationBenchmark.synchronousStarting ss 120000 21.496 0.502 us
*/
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.SingleShotTime))
@Fork(5)
@Warmup(iterations = 1000)
@Measurement(iterations = 4000)
class ActorCreationBenchmark {
implicit val system: ActorSystem = ActorSystem()
final val props = Props[MyActor]()
var i = 1
def name = {
i += 1
"some-rather-long-actor-name-actor-" + i
}
@TearDown(Level.Trial)
def shutdown(): Unit = {
system.terminate()
Await.ready(system.whenTerminated, 15.seconds)
}
@Benchmark
@OutputTimeUnit(TimeUnit.MICROSECONDS)
def synchronousStarting =
system.actorOf(props, name)
}
class MyActor extends Actor {
override def receive: Receive = {
case _ =>
}
}

View file

@ -0,0 +1,52 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor
import java.util.concurrent.TimeUnit
import org.openjdk.jmh.annotations.{ Scope => JmhScope }
import org.openjdk.jmh.annotations.Benchmark
import org.openjdk.jmh.annotations.BenchmarkMode
import org.openjdk.jmh.annotations.Fork
import org.openjdk.jmh.annotations.Measurement
import org.openjdk.jmh.annotations.Mode
import org.openjdk.jmh.annotations.OutputTimeUnit
import org.openjdk.jmh.annotations.State
import org.openjdk.jmh.annotations.Warmup
/*
[info] Benchmark Mode Samples Score Score error Units
[info] a.a.ActorPathValidationBenchmark.handLoop7000 thrpt 20 0.070 0.002 ops/us
[info] a.a.ActorPathValidationBenchmark.old7000 -- blows up (stack overflow) --
[info] a.a.ActorPathValidationBenchmark.handLoopActor_1 thrpt 20 38.825 3.378 ops/us
[info] a.a.ActorPathValidationBenchmark.oldActor_1 thrpt 20 1.585 0.090 ops/us
*/
@Fork(2)
@State(JmhScope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Warmup(iterations = 5)
@Measurement(iterations = 10)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
class ActorPathValidationBenchmark {
final val a = "actor-1"
final val s = "687474703a2f2f74686566727569742e636f6d2f26683d37617165716378357926656e" * 100
final val ElementRegex = """(?:[-\w:@&=+,.!~*'_;]|%\p{XDigit}{2})(?:[-\w:@&=+,.!~*'$_;]|%\p{XDigit}{2})*""".r
// @Benchmark // blows up with stack overflow, we know
def old7000: Option[List[String]] = ElementRegex.unapplySeq(s)
@Benchmark
def handLoop7000: Boolean = ActorPath.isValidPathElement(s)
@Benchmark
def oldActor_1: Option[List[String]] = ElementRegex.unapplySeq(a)
@Benchmark
def handLoopActor_1: Boolean = ActorPath.isValidPathElement(a)
}

View file

@ -0,0 +1,97 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor
import java.util.concurrent.TimeUnit
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.actor.BenchmarkActors._
import pekko.actor.ForkJoinActorBenchmark.cores
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(1)
@Threads(1)
@Warmup(iterations = 10, time = 5, timeUnit = TimeUnit.SECONDS, batchSize = 1)
@Measurement(iterations = 10, time = 15, timeUnit = TimeUnit.SECONDS, batchSize = 1)
class AffinityPoolComparativeBenchmark {
@Param(Array("1"))
var throughPut = 0
@Param(Array("affinity-dispatcher", "default-fj-dispatcher", "fixed-size-dispatcher"))
var dispatcher = ""
@Param(Array("SingleConsumerOnlyUnboundedMailbox")) // "default"
var mailbox = ""
final val numThreads, numActors = 8
final val numMessagesPerActorPair = 2000000
final val totalNumberOfMessages = numMessagesPerActorPair * (numActors / 2)
implicit var system: ActorSystem = _
@Setup(Level.Trial)
def setup(): Unit = {
requireRightNumberOfCores(cores)
val mailboxConf = mailbox match {
case "default" => ""
case "SingleConsumerOnlyUnboundedMailbox" =>
s"""default-mailbox.mailbox-type = "${classOf[pekko.dispatch.SingleConsumerOnlyUnboundedMailbox].getName}""""
}
system = ActorSystem(
"AffinityPoolComparativeBenchmark",
ConfigFactory.parseString(s"""| pekko {
| log-dead-letters = off
| actor {
| default-fj-dispatcher {
| executor = "fork-join-executor"
| fork-join-executor {
| parallelism-min = $numThreads
| parallelism-factor = 1.0
| parallelism-max = $numThreads
| }
| throughput = $throughPut
| }
|
| fixed-size-dispatcher {
| executor = "thread-pool-executor"
| thread-pool-executor {
| fixed-pool-size = $numThreads
| }
| throughput = $throughPut
| }
|
| affinity-dispatcher {
| executor = "affinity-pool-executor"
| affinity-pool-executor {
| parallelism-min = $numThreads
| parallelism-factor = 1.0
| parallelism-max = $numThreads
| task-queue-size = 512
| idle-cpu-level = 5
| fair-work-distribution.threshold = 2048
| }
| throughput = $throughPut
| }
| $mailboxConf
| }
| }
""".stripMargin))
}
@TearDown(Level.Trial)
def shutdown(): Unit = tearDownSystem()
@Benchmark
@OperationsPerInvocation(totalNumberOfMessages)
def pingPong(): Unit = benchmarkPingPongActors(numMessagesPerActorPair, numActors, dispatcher, throughPut, timeout)
}

View file

@ -0,0 +1,71 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor
import java.util.concurrent.TimeUnit
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
import org.apache.pekko.actor.BenchmarkActors._
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(1)
@Threads(1)
@Warmup(iterations = 10, time = 5, timeUnit = TimeUnit.SECONDS, batchSize = 1)
@Measurement(iterations = 10, time = 15, timeUnit = TimeUnit.SECONDS, batchSize = 1)
class AffinityPoolIdleCPULevelBenchmark {
final val numThreads, numActors = 8
final val numMessagesPerActorPair = 2000000
final val totalNumberOfMessages = numMessagesPerActorPair * (numActors / 2)
implicit var system: ActorSystem = _
@Param(Array("1", "3", "5", "7", "10"))
var idleCPULevel = ""
@Param(Array("25"))
var throughPut = 0
@Setup(Level.Trial)
def setup(): Unit = {
requireRightNumberOfCores(numThreads)
system = ActorSystem(
"AffinityPoolWaitingStrategyBenchmark",
ConfigFactory.parseString(s""" | pekko {
| log-dead-letters = off
| actor {
| affinity-dispatcher {
| executor = "affinity-pool-executor"
| affinity-pool-executor {
| parallelism-min = $numThreads
| parallelism-factor = 1.0
| parallelism-max = $numThreads
| task-queue-size = 512
| idle-cpu-level = $idleCPULevel
| fair-work-distribution.threshold = 2048
| }
| throughput = $throughPut
| }
|
| }
| }
""".stripMargin))
}
@TearDown(Level.Trial)
def shutdown(): Unit = tearDownSystem()
@Benchmark
@OutputTimeUnit(TimeUnit.NANOSECONDS)
@OperationsPerInvocation(8000000)
def pingPong(): Unit =
benchmarkPingPongActors(numMessagesPerActorPair, numActors, "affinity-dispatcher", throughPut, timeout)
}

View file

@ -0,0 +1,114 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor
import java.util.concurrent.{ CountDownLatch, TimeUnit }
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.actor.BenchmarkActors._
import pekko.actor.ForkJoinActorBenchmark.cores
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(1)
@Threads(1)
@Warmup(iterations = 10, time = 15, timeUnit = TimeUnit.SECONDS, batchSize = 1)
@Measurement(iterations = 10, time = 20, timeUnit = TimeUnit.SECONDS, batchSize = 1)
class AffinityPoolRequestResponseBenchmark {
@Param(Array("1", "5", "50"))
var throughPut = 0
@Param(Array("affinity-dispatcher", "default-fj-dispatcher", "fixed-size-dispatcher"))
var dispatcher = ""
@Param(Array("SingleConsumerOnlyUnboundedMailbox")) // "default"
var mailbox = ""
final val numThreads, numActors = 8
final val numQueriesPerActor = 400000
final val totalNumberOfMessages = numQueriesPerActor * numActors
final val numUsersInDB = 300000
implicit var system: ActorSystem = _
var actors: Vector[(ActorRef, ActorRef)] = null
var latch: CountDownLatch = null
@Setup(Level.Trial)
def setup(): Unit = {
requireRightNumberOfCores(cores)
val mailboxConf = mailbox match {
case "default" => ""
case "SingleConsumerOnlyUnboundedMailbox" =>
s"""default-mailbox.mailbox-type = "${classOf[pekko.dispatch.SingleConsumerOnlyUnboundedMailbox].getName}""""
}
system = ActorSystem(
"AffinityPoolComparativeBenchmark",
ConfigFactory.parseString(s"""| pekko {
| log-dead-letters = off
| actor {
| default-fj-dispatcher {
| executor = "fork-join-executor"
| fork-join-executor {
| parallelism-min = $numThreads
| parallelism-factor = 1.0
| parallelism-max = $numThreads
| }
| throughput = $throughPut
| }
|
| fixed-size-dispatcher {
| executor = "thread-pool-executor"
| thread-pool-executor {
| fixed-pool-size = $numThreads
| }
| throughput = $throughPut
| }
|
| affinity-dispatcher {
| executor = "affinity-pool-executor"
| affinity-pool-executor {
| parallelism-min = $numThreads
| parallelism-factor = 1.0
| parallelism-max = $numThreads
| task-queue-size = 512
| idle-cpu-level = 5
| fair-work-distribution.threshold = 2048
| }
| throughput = $throughPut
| }
| $mailboxConf
| }
| }
""".stripMargin))
}
@TearDown(Level.Trial)
def shutdown(): Unit = tearDownSystem()
@Setup(Level.Invocation)
def setupActors(): Unit = {
val (_actors, _latch) =
RequestResponseActors.startUserQueryActorPairs(numActors, numQueriesPerActor, numUsersInDB, dispatcher)
actors = _actors
latch = _latch
}
@Benchmark
@OperationsPerInvocation(totalNumberOfMessages)
def queryUserServiceActor(): Unit = {
val startNanoTime = System.nanoTime()
RequestResponseActors.initiateQuerySimulation(actors, throughPut * 2)
latch.await(BenchmarkActors.timeout.toSeconds, TimeUnit.SECONDS)
BenchmarkActors.printProgress(totalNumberOfMessages, numActors, startNanoTime)
}
}

View file

@ -0,0 +1,181 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor
import java.util.concurrent.{ CountDownLatch, TimeUnit }
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.concurrent.duration.Duration
object BenchmarkActors {
val timeout = 30.seconds
case object Message
case object Stop
class PingPong(val messagesPerPair: Int, latch: CountDownLatch) extends Actor {
var left = messagesPerPair / 2
def receive = {
case Message =>
if (left == 0) {
latch.countDown()
context.stop(self)
}
sender() ! Message
left -= 1
}
}
object PingPong {
def props(messagesPerPair: Int, latch: CountDownLatch) = Props(new PingPong(messagesPerPair, latch))
}
class Echo extends Actor {
def receive = {
case Message =>
sender() ! Message
}
}
object EchoSender {
def props(messagesPerPair: Int, latch: CountDownLatch, batchSize: Int): Props =
Props(new EchoSender(messagesPerPair, latch, batchSize))
}
class EchoSender(messagesPerPair: Int, latch: CountDownLatch, batchSize: Int) extends Actor {
private val echo = context.actorOf(Props[Echo]().withDispatcher(context.props.dispatcher), "echo")
private var left = messagesPerPair / 2
private var batch = 0
def receive = {
case Message =>
batch -= 1
if (batch <= 0) {
if (!sendBatch()) {
latch.countDown()
context.stop(self)
}
}
}
private def sendBatch(): Boolean = {
if (left > 0) {
var i = 0
while (i < batchSize) {
echo ! Message
i += 1
}
left -= batchSize
batch = batchSize
true
} else
false
}
}
class Pipe(next: Option[ActorRef]) extends Actor {
def receive = {
case Message =>
if (next.isDefined) next.get.forward(Message)
case Stop =>
context.stop(self)
if (next.isDefined) next.get.forward(Stop)
}
}
object Pipe {
def props(next: Option[ActorRef]) = Props(new Pipe(next))
}
private def startPingPongActorPairs(messagesPerPair: Int, numPairs: Int, dispatcher: String)(
implicit system: ActorSystem): (Vector[(ActorRef, ActorRef)], CountDownLatch) = {
val fullPathToDispatcher = "pekko.actor." + dispatcher
val latch = new CountDownLatch(numPairs * 2)
val actors = List
.fill(numPairs) {
val ping = system.actorOf(PingPong.props(messagesPerPair, latch).withDispatcher(fullPathToDispatcher))
val pong = system.actorOf(PingPong.props(messagesPerPair, latch).withDispatcher(fullPathToDispatcher))
(ping, pong)
}
.toVector
(actors, latch)
}
private def initiatePingPongForPairs(refs: Vector[(ActorRef, ActorRef)], inFlight: Int): Unit = {
for {
(ping, pong) <- refs
_ <- 1 to inFlight
} {
ping.tell(Message, pong)
}
}
private def startEchoActorPairs(messagesPerPair: Int, numPairs: Int, dispatcher: String, batchSize: Int)(
implicit system: ActorSystem): (Vector[ActorRef], CountDownLatch) = {
val fullPathToDispatcher = "pekko.actor." + dispatcher
val latch = new CountDownLatch(numPairs)
val actors = (1 to numPairs).map { _ =>
system.actorOf(EchoSender.props(messagesPerPair, latch, batchSize).withDispatcher(fullPathToDispatcher))
}.toVector
(actors, latch)
}
private def initiateEchoPairs(refs: Vector[ActorRef]): Unit = {
refs.foreach(_ ! Message)
}
def printProgress(totalMessages: Long, numActors: Int, startNanoTime: Long): Unit = {
val durationMicros = (System.nanoTime() - startNanoTime) / 1000
println(
f" $totalMessages messages by $numActors actors took ${durationMicros / 1000} ms, " +
f"${totalMessages.toDouble / durationMicros}%,.2f M msg/s")
}
def requireRightNumberOfCores(numCores: Int): Unit =
require(
Runtime.getRuntime.availableProcessors == numCores,
s"Update the cores constant to ${Runtime.getRuntime.availableProcessors}")
def benchmarkPingPongActors(
numMessagesPerActorPair: Int,
numActors: Int,
dispatcher: String,
throughPut: Int,
shutdownTimeout: Duration)(implicit system: ActorSystem): Unit = {
val numPairs = numActors / 2
val totalNumMessages = numPairs * numMessagesPerActorPair
val (actors, latch) = startPingPongActorPairs(numMessagesPerActorPair, numPairs, dispatcher)
val startNanoTime = System.nanoTime()
initiatePingPongForPairs(actors, inFlight = throughPut * 2)
latch.await(shutdownTimeout.toSeconds, TimeUnit.SECONDS)
printProgress(totalNumMessages, numActors, startNanoTime)
}
def benchmarkEchoActors(
numMessagesPerActorPair: Int,
numActors: Int,
dispatcher: String,
batchSize: Int,
shutdownTimeout: Duration)(implicit system: ActorSystem): Unit = {
val numPairs = numActors / 2
val totalNumMessages = numPairs * numMessagesPerActorPair
val (actors, latch) = startEchoActorPairs(numMessagesPerActorPair, numPairs, dispatcher, batchSize)
val startNanoTime = System.nanoTime()
initiateEchoPairs(actors)
latch.await(shutdownTimeout.toSeconds, TimeUnit.SECONDS)
printProgress(totalNumMessages, numActors, startNanoTime)
}
def tearDownSystem()(implicit system: ActorSystem): Unit = {
system.terminate()
Await.ready(system.whenTerminated, timeout)
}
}

View file

@ -0,0 +1,103 @@
/*
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor
import java.nio.ByteBuffer
import java.util.Random
import java.util.concurrent.TimeUnit
import org.openjdk.jmh.annotations._
import org.apache.pekko.io.DirectByteBufferPool
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.AverageTime))
@OutputTimeUnit(TimeUnit.NANOSECONDS)
class DirectByteBufferPoolBenchmark {
private val MAX_LIVE_BUFFERS = 8192
@Param(Array("00000", "00256", "01024", "04096", "16384", "65536"))
var size = 0
val random = new Random
private[pekko] var arteryPool: DirectByteBufferPool = _
@Setup(Level.Trial)
def setup(): Unit = {
arteryPool = new DirectByteBufferPool(size, MAX_LIVE_BUFFERS)
}
@TearDown(Level.Trial)
def tearDown(): Unit = {
var i = 0
while (i < MAX_LIVE_BUFFERS) {
arteryPool.release(pooledDirectBuffers(i))
pooledDirectBuffers(i) = null
DirectByteBufferPool.tryCleanDirectByteBuffer(unpooledDirectBuffers(i))
unpooledDirectBuffers(i) = null
DirectByteBufferPool.tryCleanDirectByteBuffer(unpooledHeapBuffers(i))
unpooledHeapBuffers(i) = null
i += 1
}
}
private val unpooledHeapBuffers = new Array[ByteBuffer](MAX_LIVE_BUFFERS)
private val pooledDirectBuffers = new Array[ByteBuffer](MAX_LIVE_BUFFERS)
private val unpooledDirectBuffers = new Array[ByteBuffer](MAX_LIVE_BUFFERS)
import org.openjdk.jmh.annotations.Benchmark
@Benchmark
def unpooledHeapAllocAndRelease(): Unit = {
val idx = random.nextInt(unpooledHeapBuffers.length)
val oldBuf = unpooledHeapBuffers(idx)
if (oldBuf != null) DirectByteBufferPool.tryCleanDirectByteBuffer(oldBuf)
unpooledHeapBuffers(idx) = ByteBuffer.allocateDirect(size)
}
@Benchmark
def unpooledDirectAllocAndRelease(): Unit = {
val idx = random.nextInt(unpooledDirectBuffers.length)
val oldBuf = unpooledDirectBuffers(idx)
if (oldBuf != null) DirectByteBufferPool.tryCleanDirectByteBuffer(oldBuf)
unpooledDirectBuffers(idx) = ByteBuffer.allocateDirect(size)
}
@Benchmark
def pooledDirectAllocAndRelease(): Unit = {
val idx = random.nextInt(pooledDirectBuffers.length)
val oldBuf = pooledDirectBuffers(idx)
if (oldBuf != null) arteryPool.release(oldBuf)
pooledDirectBuffers(idx) = arteryPool.acquire()
}
}
object DirectByteBufferPoolBenchmark {
final val numMessages = 2000000 // messages per actor pair
// Constants because they are used in annotations
// update according to cpu
final val cores = 8
final val coresStr = "8"
final val cores2xStr = "16"
final val cores4xStr = "24"
final val twoActors = 2
final val moreThanCoresActors = cores * 2
final val lessThanCoresActors = cores / 2
final val sameAsCoresActors = cores
final val totalMessagesTwoActors = numMessages
final val totalMessagesMoreThanCores = (moreThanCoresActors * numMessages) / 2
final val totalMessagesLessThanCores = (lessThanCoresActors * numMessages) / 2
final val totalMessagesSameAsCores = (sameAsCoresActors * numMessages) / 2
}

View file

@ -0,0 +1,141 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor
import java.util.concurrent.TimeUnit
import scala.annotation.tailrec
import scala.concurrent.Await
import scala.concurrent.duration._
import BenchmarkActors._
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
import org.apache.pekko.testkit.TestProbe
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(1)
@Threads(1)
@Warmup(iterations = 10, time = 5, timeUnit = TimeUnit.SECONDS, batchSize = 1)
@Measurement(iterations = 10, time = 15, timeUnit = TimeUnit.SECONDS, batchSize = 1)
class ForkJoinActorBenchmark {
import ForkJoinActorBenchmark._
@Param(Array("50"))
var tpt = 0
@Param(Array(coresStr)) // coresStr, cores2xStr, cores4xStr
var threads = ""
@Param(
Array(
"org.apache.pekko.dispatch.SingleConsumerOnlyUnboundedMailbox",
"org.apache.pekko.actor.ManyToOneArrayMailbox",
"org.apache.pekko.actor.JCToolsMailbox"))
var mailbox = ""
implicit var system: ActorSystem = _
@Setup(Level.Trial)
def setup(): Unit = {
requireRightNumberOfCores(cores)
system = ActorSystem(
"ForkJoinActorBenchmark",
ConfigFactory.parseString(s"""
pekko {
log-dead-letters = off
default-mailbox.mailbox-capacity = 512
actor {
fjp-dispatcher {
executor = "fork-join-executor"
fork-join-executor {
parallelism-min = $threads
parallelism-factor = 1.0
parallelism-max = $threads
}
throughput = $tpt
mailbox-type = "$mailbox"
}
}
}
"""))
}
@TearDown(Level.Trial)
def shutdown(): Unit = {
system.terminate()
Await.ready(system.whenTerminated, 15.seconds)
}
// @Benchmark
// @OperationsPerInvocation(totalMessagesTwoActors)
// def pingPong(): Unit = benchmarkPingPongActors(messages, twoActors, "fjp-dispatcher", tpt, timeout)
@Benchmark
@OperationsPerInvocation(totalMessagesLessThanCores)
def pingPongLessActorsThanCores(): Unit =
benchmarkPingPongActors(messages, lessThanCoresActors, "fjp-dispatcher", tpt, timeout)
// @Benchmark
// @OperationsPerInvocation(totalMessagesSameAsCores)
// def pingPongSameNumberOfActorsAsCores(): Unit = benchmarkPingPongActors(messages, sameAsCoresActors, "fjp-dispatcher", tpt, timeout)
@Benchmark
@OperationsPerInvocation(totalMessagesMoreThanCores)
def pingPongMoreActorsThanCores(): Unit =
benchmarkPingPongActors(messages, moreThanCoresActors, "fjp-dispatcher", tpt, timeout)
// @Benchmark
// @Measurement(timeUnit = TimeUnit.MILLISECONDS)
// @OperationsPerInvocation(messages)
def floodPipe(): Unit = {
val end = system.actorOf(Props(classOf[Pipe], None))
val middle = system.actorOf(Props(classOf[Pipe], Some(end)))
val penultimate = system.actorOf(Props(classOf[Pipe], Some(middle)))
val beginning = system.actorOf(Props(classOf[Pipe], Some(penultimate)))
val p = TestProbe()
p.watch(end)
@tailrec def send(left: Int): Unit =
if (left > 0) {
beginning ! Message
send(left - 1)
}
send(messages / 4) // we have 4 actors in the pipeline
beginning ! Stop
p.expectTerminated(end, timeout)
}
}
object ForkJoinActorBenchmark {
final val messages = 2000000 // messages per actor pair
// Constants because they are used in annotations
// update according to cpu
final val cores = 8
final val coresStr = "8"
final val cores2xStr = "16"
final val cores4xStr = "24"
final val twoActors = 2
final val moreThanCoresActors = cores * 2
final val lessThanCoresActors = cores / 2
final val sameAsCoresActors = cores
final val totalMessagesTwoActors = messages
final val totalMessagesMoreThanCores = (moreThanCoresActors * messages) / 2
final val totalMessagesLessThanCores = (lessThanCoresActors * messages) / 2
final val totalMessagesSameAsCores = (sameAsCoresActors * messages) / 2
}

View file

@ -0,0 +1,58 @@
/*
* Copyright (C) 2017-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor
import scala.annotation.tailrec
import scala.concurrent.duration.Duration
import com.typesafe.config.Config
import org.jctools.queues.MpscGrowableArrayQueue
import org.apache.pekko
import pekko.dispatch.BoundedMessageQueueSemantics
import pekko.dispatch.BoundedNodeMessageQueue
import pekko.dispatch.Envelope
import pekko.dispatch.MailboxType
import pekko.dispatch.MessageQueue
import pekko.dispatch.ProducesMessageQueue
case class JCToolsMailbox(val capacity: Int) extends MailboxType with ProducesMessageQueue[BoundedNodeMessageQueue] {
def this(settings: ActorSystem.Settings, config: Config) = this(config.getInt("mailbox-capacity"))
if (capacity < 0) throw new IllegalArgumentException("The capacity for JCToolsMailbox can not be negative")
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
new JCToolsMessageQueue(capacity)
}
class JCToolsMessageQueue(capacity: Int)
extends MpscGrowableArrayQueue[Envelope](capacity)
with MessageQueue
with BoundedMessageQueueSemantics {
final def pushTimeOut: Duration = Duration.Undefined
final def enqueue(receiver: ActorRef, handle: Envelope): Unit =
if (!offer(handle))
receiver
.asInstanceOf[InternalActorRef]
.provider
.deadLetters
.tell(DeadLetter(handle.message, handle.sender, receiver), handle.sender)
final def dequeue(): Envelope = poll()
final def numberOfMessages: Int = size()
final def hasMessages: Boolean = !isEmpty()
@tailrec final def cleanUp(owner: ActorRef, deadLetters: MessageQueue): Unit = {
val envelope = dequeue()
if (envelope ne null) {
deadLetters.enqueue(owner, envelope)
cleanUp(owner, deadLetters)
}
}
}

View file

@ -0,0 +1,72 @@
/*
* Copyright (C) 2017-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor
import scala.annotation.tailrec
import scala.concurrent.duration.Duration
import com.typesafe.config.Config
import org.agrona.concurrent.ManyToOneConcurrentArrayQueue
import org.apache.pekko
import pekko.dispatch.BoundedMessageQueueSemantics
import pekko.dispatch.BoundedNodeMessageQueue
import pekko.dispatch.Envelope
import pekko.dispatch.MailboxType
import pekko.dispatch.MessageQueue
import pekko.dispatch.ProducesMessageQueue
/**
* ManyToOneArrayMailbox is a high-performance, multiple-producer single-consumer, bounded MailboxType,
* Noteworthy is that it discards overflow as DeadLetters.
*
* It can't have multiple consumers, which rules out using it with BalancingPool (BalancingDispatcher) for instance.
*
* NOTE: ManyToOneArrayMailbox does not use `mailbox-push-timeout-time` as it is non-blocking.
*/
case class ManyToOneArrayMailbox(val capacity: Int)
extends MailboxType
with ProducesMessageQueue[BoundedNodeMessageQueue] {
def this(settings: ActorSystem.Settings, config: Config) = this(config.getInt("mailbox-capacity"))
if (capacity < 0) throw new IllegalArgumentException("The capacity for ManyToOneArrayMailbox can not be negative")
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
new ManyToOneArrayMessageQueue(capacity)
}
/**
* Lock-free bounded non-blocking multiple-producer single-consumer queue.
* Discards overflowing messages into DeadLetters.
* Allocation free, using `org.agrona.concurrent.ManyToOneConcurrentArrayQueue`.
*/
class ManyToOneArrayMessageQueue(capacity: Int) extends MessageQueue with BoundedMessageQueueSemantics {
final def pushTimeOut: Duration = Duration.Undefined
private val queue = new ManyToOneConcurrentArrayQueue[Envelope](capacity)
final def enqueue(receiver: ActorRef, handle: Envelope): Unit =
if (!queue.add(handle))
receiver
.asInstanceOf[InternalActorRef]
.provider
.deadLetters
.tell(DeadLetter(handle.message, handle.sender, receiver), handle.sender)
final def dequeue(): Envelope = queue.poll()
final def numberOfMessages: Int = queue.size()
final def hasMessages: Boolean = !queue.isEmpty()
@tailrec final def cleanUp(owner: ActorRef, deadLetters: MessageQueue): Unit = {
val envelope = dequeue()
if (envelope ne null) {
deadLetters.enqueue(owner, envelope)
cleanUp(owner, deadLetters)
}
}
}

View file

@ -0,0 +1,97 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor
import java.util.concurrent.CountDownLatch
import scala.collection.mutable
import scala.util.Random
object RequestResponseActors {
case class Request(userId: Int)
case class User(userId: Int, firstName: String, lastName: String, ssn: Int, friends: Seq[Int])
class UserQueryActor(latch: CountDownLatch, numQueries: Int, numUsersInDB: Int) extends Actor {
private var left = numQueries
private val receivedUsers: mutable.Map[Int, User] = mutable.Map()
private val randGenerator = new Random()
override def receive: Receive = {
case u: User => {
receivedUsers.put(u.userId, u)
if (left == 0) {
latch.countDown()
context.stop(self)
} else {
sender() ! Request(randGenerator.nextInt(numUsersInDB))
}
left -= 1
}
}
}
object UserQueryActor {
def props(latch: CountDownLatch, numQueries: Int, numUsersInDB: Int) = {
Props(new UserQueryActor(latch, numQueries, numUsersInDB))
}
}
class UserServiceActor(userDb: Map[Int, User], latch: CountDownLatch, numQueries: Int) extends Actor {
private var left = numQueries
def receive = {
case Request(id) =>
userDb.get(id) match {
case Some(u) => sender() ! u
case None =>
}
if (left == 0) {
latch.countDown()
context.stop(self)
}
left -= 1
}
}
object UserServiceActor {
def props(latch: CountDownLatch, numQueries: Int, numUsersInDB: Int) = {
val r = new Random()
val users = for {
id <- 0 until numUsersInDB
firstName = r.nextString(5)
lastName = r.nextString(7)
ssn = r.nextInt()
friendIds = for { _ <- 0 until 5 } yield r.nextInt(numUsersInDB)
} yield id -> User(id, firstName, lastName, ssn, friendIds)
Props(new UserServiceActor(users.toMap, latch, numQueries))
}
}
def startUserQueryActorPairs(numActors: Int, numQueriesPerActor: Int, numUsersInDBPerActor: Int, dispatcher: String)(
implicit system: ActorSystem) = {
val fullPathToDispatcher = "pekko.actor." + dispatcher
val latch = new CountDownLatch(numActors)
val actorsPairs = for {
i <- (1 to (numActors / 2)).toVector
userQueryActor = system.actorOf(
UserQueryActor.props(latch, numQueriesPerActor, numUsersInDBPerActor).withDispatcher(fullPathToDispatcher))
userServiceActor = system.actorOf(
UserServiceActor.props(latch, numQueriesPerActor, numUsersInDBPerActor).withDispatcher(fullPathToDispatcher))
} yield (userQueryActor, userServiceActor)
(actorsPairs, latch)
}
def initiateQuerySimulation(requestResponseActorPairs: Seq[(ActorRef, ActorRef)], inFlight: Int) = {
for {
(queryActor, serviceActor) <- requestResponseActorPairs
i <- 1 to inFlight
} {
serviceActor.tell(Request(i), queryActor)
}
}
}

View file

@ -0,0 +1,47 @@
/*
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.routing.RoundRobinPool
import pekko.testkit.TestActors
import pekko.testkit.TestProbe
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.SingleShotTime))
@Fork(3)
@Warmup(iterations = 20)
@Measurement(iterations = 100)
class RouterPoolCreationBenchmark {
implicit val system: ActorSystem = ActorSystem()
val probe = TestProbe()
Props[TestActors.EchoActor]()
@Param(Array("1000", "2000", "3000", "4000"))
var size = 0
@TearDown(Level.Trial)
def shutdown(): Unit = {
system.terminate()
Await.ready(system.whenTerminated, 15.seconds)
}
@Benchmark
@OutputTimeUnit(TimeUnit.MICROSECONDS)
def testCreation: Boolean = {
val pool = system.actorOf(RoundRobinPool(size).props(TestActors.echoActorProps))
pool.tell("hello", probe.ref)
probe.expectMsg(5.seconds, "hello")
true
}
}

View file

@ -0,0 +1,98 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor
import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicInteger
import scala.concurrent.{ Await, Promise }
import scala.concurrent.ExecutionContext.Implicits.global
import scala.concurrent.duration._
import org.openjdk.jmh.annotations._
import org.apache.pekko.util.Timeout
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(2)
@Warmup(iterations = 10, time = 1700, timeUnit = TimeUnit.MILLISECONDS)
@Measurement(iterations = 20, time = 1700, timeUnit = TimeUnit.MILLISECONDS)
class ScheduleBenchmark {
implicit val system: ActorSystem = ActorSystem()
val scheduler: Scheduler = system.scheduler
val interval: FiniteDuration = 25.millis
val within: FiniteDuration = 2.seconds
implicit val timeout: Timeout = Timeout(within)
@Param(Array("4", "16", "64"))
var to = 0
@Param(Array("0.1", "0.35", "0.9"))
var ratio = 0d
var winner: Int = _
var promise: Promise[Any] = _
@Setup(Level.Iteration)
def setup(): Unit = {
winner = (to * ratio + 1).toInt
promise = Promise[Any]()
}
@TearDown
def shutdown(): Unit = {
system.terminate()
Await.ready(system.whenTerminated, 15.seconds)
}
def op(idx: Int) = if (idx == winner) promise.trySuccess(idx) else idx
@Benchmark
def scheduleWithFixedDelay(): Unit = {
val aIdx = new AtomicInteger(1)
val tryWithNext = scheduler.scheduleWithFixedDelay(0.millis, interval) { () =>
val idx = aIdx.getAndIncrement
if (idx <= to) op(idx)
}
promise.future.onComplete {
case _ =>
tryWithNext.cancel()
}
Await.result(promise.future, within)
}
@Benchmark
def scheduleAtFixedRate(): Unit = {
val aIdx = new AtomicInteger(1)
val tryWithNext = scheduler.scheduleAtFixedRate(0.millis, interval) { () =>
val idx = aIdx.getAndIncrement
if (idx <= to) op(idx)
}
promise.future.onComplete {
case _ =>
tryWithNext.cancel()
}
Await.result(promise.future, within)
}
@Benchmark
def multipleScheduleOnce(): Unit = {
val tryWithNext = (1 to to)
.foldLeft(0.millis -> List[Cancellable]()) {
case ((interv, c), idx) =>
(interv + interval,
scheduler.scheduleOnce(interv) {
op(idx)
} :: c)
}
._2
promise.future.onComplete {
case _ =>
tryWithNext.foreach(_.cancel())
}
Await.result(promise.future, within)
}
}

View file

@ -0,0 +1,64 @@
/*
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
import org.apache.pekko.testkit.TestProbe
object StashCreationBenchmark {
class StashingActor extends Actor with Stash {
def receive = {
case msg => sender() ! msg
}
}
val props = Props[StashingActor]()
}
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.SampleTime))
@Fork(3)
@Warmup(iterations = 5)
@Measurement(iterations = 10)
class StashCreationBenchmark {
val conf = ConfigFactory.parseString("""
my-dispatcher = {
stash-capacity = 1000
}
""")
implicit val system: ActorSystem = ActorSystem("StashCreationBenchmark", conf)
val probe = TestProbe()
@TearDown(Level.Trial)
def shutdown(): Unit = {
system.terminate()
Await.ready(system.whenTerminated, 15.seconds)
}
@Benchmark
@OutputTimeUnit(TimeUnit.MICROSECONDS)
def testDefault: Boolean = {
val stash = system.actorOf(StashCreationBenchmark.props)
stash.tell("hello", probe.ref)
probe.expectMsg("hello")
true
}
@Benchmark
@OutputTimeUnit(TimeUnit.MICROSECONDS)
def testCustom: Boolean = {
val stash = system.actorOf(StashCreationBenchmark.props.withDispatcher("my-dispatcher"))
stash.tell("hello", probe.ref)
probe.expectMsg("hello")
true
}
}

View file

@ -0,0 +1,164 @@
/*
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import com.typesafe.config.{ Config, ConfigFactory }
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.dispatch._
import pekko.testkit.TestProbe
import pekko.util.Helpers.ConfigOps
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.SingleShotTime))
@Fork(5)
@Threads(1)
@Warmup(iterations = 10, batchSize = TellOnlyBenchmark.numMessages)
@Measurement(iterations = 10, batchSize = TellOnlyBenchmark.numMessages)
class TellOnlyBenchmark {
import TellOnlyBenchmark._
implicit var system: ActorSystem = _
@Setup(Level.Trial)
def setup(): Unit = {
system = ActorSystem(
"TellOnlyBenchmark",
ConfigFactory.parseString(s"""| pekko {
| log-dead-letters = off
| actor {
| default-dispatcher {
| executor = "fork-join-executor"
| fork-join-executor {
| parallelism-min = 1
| parallelism-max = 4
| }
| throughput = 1
| }
| }
| }
| dropping-dispatcher {
| fork-join-executor.parallelism-min = 1
| fork-join-executor.parallelism-max = 1
| type = "org.apache.pekko.actor.TellOnlyBenchmark$$DroppingDispatcherConfigurator"
| mailbox-type = "org.apache.pekko.actor.TellOnlyBenchmark$$UnboundedDroppingMailbox"
| }
| """.stripMargin))
}
@TearDown(Level.Trial)
def shutdown(): Unit = {
system.terminate()
Await.ready(system.whenTerminated, 15.seconds)
}
var actor: ActorRef = _
var probe: TestProbe = _
@Setup(Level.Iteration)
def setupIteration(): Unit = {
actor = system.actorOf(Props[TellOnlyBenchmark.Echo]().withDispatcher("dropping-dispatcher"))
probe = TestProbe()
probe.watch(actor)
probe.send(actor, message)
probe.expectMsg(message)
probe.send(actor, flipDrop)
probe.expectNoMessage(200.millis)
System.gc()
}
@TearDown(Level.Iteration)
def shutdownIteration(): Unit = {
probe.send(actor, flipDrop)
probe.expectNoMessage(200.millis)
actor ! stop
probe.expectTerminated(actor, timeout)
actor = null
probe = null
}
@Benchmark
@OutputTimeUnit(TimeUnit.MICROSECONDS)
def tell(): Unit = {
probe.send(actor, message)
}
}
object TellOnlyBenchmark {
final val stop = "stop"
final val message = "message"
final val flipDrop = "flipDrop"
final val timeout = 5.seconds
final val numMessages = 1000000
class Echo extends Actor {
def receive = {
case `stop` =>
context.stop(self)
case m => sender() ! m
}
}
class DroppingMessageQueue extends UnboundedMailbox.MessageQueue {
@volatile var dropping = false
override def enqueue(receiver: ActorRef, handle: Envelope): Unit = {
if (handle.message == flipDrop) dropping = !dropping
else if (!dropping) super.enqueue(receiver, handle)
}
}
case class UnboundedDroppingMailbox() extends MailboxType with ProducesMessageQueue[DroppingMessageQueue] {
def this(settings: ActorSystem.Settings, config: Config) = this()
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
new DroppingMessageQueue
}
class DroppingDispatcher(
_configurator: MessageDispatcherConfigurator,
_id: String,
_throughput: Int,
_throughputDeadlineTime: Duration,
_executorServiceFactoryProvider: ExecutorServiceFactoryProvider,
_shutdownTimeout: FiniteDuration)
extends Dispatcher(
_configurator,
_id,
_throughput,
_throughputDeadlineTime,
_executorServiceFactoryProvider,
_shutdownTimeout) {
override protected[pekko] def dispatch(receiver: ActorCell, invocation: Envelope): Unit = {
val mbox = receiver.mailbox
mbox.enqueue(receiver.self, invocation)
mbox.messageQueue match {
case mb: DroppingMessageQueue if mb.dropping => // do nothing
case _ => registerForExecution(mbox, true, false)
}
}
}
class DroppingDispatcherConfigurator(config: Config, prerequisites: DispatcherPrerequisites)
extends MessageDispatcherConfigurator(config, prerequisites) {
override def dispatcher(): MessageDispatcher =
new DroppingDispatcher(
this,
config.getString("id"),
config.getInt("throughput"),
config.getNanosDuration("throughput-deadline-time"),
configureExecutor(),
config.getMillisDuration("shutdown-timeout"))
}
}

View file

@ -0,0 +1,104 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor.typed
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.actor.typed.scaladsl.AskPattern._
object TypedActorBenchmark {
// Constants because they are used in annotations
final val threads = 8 // update according to cpu
final val numMessagesPerActorPair = 1000000 // messages per actor pair
final val numActors = 512
final val totalMessages = numMessagesPerActorPair * numActors / 2
final val timeout = 30.seconds
}
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(1)
@Threads(1)
@Warmup(iterations = 10, time = 5, timeUnit = TimeUnit.SECONDS, batchSize = 1)
@Measurement(iterations = 10, time = 15, timeUnit = TimeUnit.SECONDS, batchSize = 1)
class TypedActorBenchmark {
import TypedActorBenchmark._
import TypedBenchmarkActors._
@Param(Array("50"))
var tpt = 0
@Param(Array("50"))
var batchSize = 0
@Param(Array("pekko.dispatch.SingleConsumerOnlyUnboundedMailbox", "pekko.dispatch.UnboundedMailbox"))
var mailbox = ""
@Param(Array("fjp-dispatcher")) // @Param(Array("fjp-dispatcher", "affinity-dispatcher"))
var dispatcher = ""
implicit var system: ActorSystem[Start] = _
implicit val askTimeout: pekko.util.Timeout = pekko.util.Timeout(timeout)
@Setup(Level.Trial)
def setup(): Unit = {
pekko.actor.BenchmarkActors.requireRightNumberOfCores(threads)
system = ActorSystem(
TypedBenchmarkActors.echoActorsSupervisor(numMessagesPerActorPair, numActors, dispatcher, batchSize),
"TypedActorBenchmark",
ConfigFactory.parseString(s"""
pekko.actor {
default-mailbox.mailbox-capacity = 512
fjp-dispatcher {
executor = "fork-join-executor"
fork-join-executor {
parallelism-min = $threads
parallelism-factor = 1.0
parallelism-max = $threads
}
throughput = $tpt
mailbox-type = "$mailbox"
}
affinity-dispatcher {
executor = "affinity-pool-executor"
affinity-pool-executor {
parallelism-min = $threads
parallelism-factor = 1.0
parallelism-max = $threads
task-queue-size = 512
idle-cpu-level = 5
fair-work-distribution.threshold = 2048
}
throughput = $tpt
mailbox-type = "$mailbox"
}
}
"""))
}
@TearDown(Level.Trial)
def shutdown(): Unit = {
system.terminate()
Await.ready(system.whenTerminated, 15.seconds)
}
@Benchmark
@OperationsPerInvocation(totalMessages)
def echo(): Unit = {
Await.result(system.ask(Start(_)), timeout)
}
}

View file

@ -0,0 +1,197 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor.typed
import java.util.concurrent.CountDownLatch
import scala.concurrent.duration._
import org.apache.pekko
import pekko.Done
import pekko.actor.typed.scaladsl.ActorContext
import pekko.actor.typed.scaladsl.Behaviors
object TypedBenchmarkActors {
// to avoid benchmark to be dominated by allocations of message
// we pass the respondTo actor ref into the behavior
case object Message
private def echoBehavior(respondTo: ActorRef[Message.type]): Behavior[Message.type] = Behaviors.receive { (_, _) =>
respondTo ! Message
Behaviors.same
}
private def echoSender(
messagesPerPair: Int,
onDone: ActorRef[Done],
batchSize: Int,
childProps: Props): Behavior[Message.type] =
Behaviors.setup { ctx =>
val echo = ctx.spawn(echoBehavior(ctx.self), "echo", childProps)
var left = messagesPerPair / 2
var batch = 0
def sendBatch(): Boolean = {
if (left > 0) {
var i = 0
while (i < batchSize) {
echo ! Message
i += 1
}
left -= batchSize
batch = batchSize
true
} else
false
}
Behaviors.receiveMessage { _ =>
batch -= 1
if (batch <= 0 && !sendBatch()) {
onDone ! Done
Behaviors.stopped
} else {
Behaviors.same
}
}
}
case class Start(respondTo: ActorRef[Completed])
case class Completed(startNanoTime: Long)
def echoActorsSupervisor(
numMessagesPerActorPair: Int,
numActors: Int,
dispatcher: String,
batchSize: Int): Behavior[Start] =
Behaviors.receive { (ctx, msg) =>
msg match {
case Start(respondTo) =>
// note: no protection against accidentally running bench sessions in parallel
val sessionBehavior =
startEchoBenchSession(numMessagesPerActorPair, numActors, dispatcher, batchSize, respondTo)
ctx.spawnAnonymous(sessionBehavior)
Behaviors.same
}
}
private def startEchoBenchSession(
messagesPerPair: Int,
numActors: Int,
dispatcher: String,
batchSize: Int,
respondTo: ActorRef[Completed]): Behavior[Unit] = {
val numPairs = numActors / 2
Behaviors
.setup[Any] { ctx =>
val props = Props.empty.withDispatcherFromConfig("pekko.actor." + dispatcher)
val pairs = (1 to numPairs).map { _ =>
ctx.spawnAnonymous(echoSender(messagesPerPair, ctx.self.narrow[Done], batchSize, props), props)
}
val startNanoTime = System.nanoTime()
pairs.foreach(_ ! Message)
var interactionsLeft = numPairs
Behaviors.receiveMessagePartial {
case Done =>
interactionsLeft -= 1
if (interactionsLeft == 0) {
val totalNumMessages = numPairs * messagesPerPair
printProgress(totalNumMessages, numActors, startNanoTime)
respondTo ! Completed(startNanoTime)
Behaviors.stopped
} else {
Behaviors.same
}
}
}
.narrow[Unit]
}
sealed trait PingPongCommand
case class StartPingPong(
messagesPerPair: Int,
numActors: Int,
dispatcher: String,
throughPut: Int,
shutdownTimeout: Duration,
replyTo: ActorRef[PingPongStarted])
extends PingPongCommand
case class PingPongStarted(completedLatch: CountDownLatch, startNanoTime: Long, totalNumMessages: Int)
case object Stop extends PingPongCommand
def benchmarkPingPongSupervisor(): Behavior[PingPongCommand] = {
Behaviors.setup { ctx =>
Behaviors.receiveMessage {
case StartPingPong(numMessagesPerActorPair, numActors, dispatcher, throughput, _, replyTo) =>
val numPairs = numActors / 2
val totalNumMessages = numPairs * numMessagesPerActorPair
val (actors, latch) = startPingPongActorPairs(ctx, numMessagesPerActorPair, numPairs, dispatcher)
val startNanoTime = System.nanoTime()
replyTo ! PingPongStarted(latch, startNanoTime, totalNumMessages)
initiatePingPongForPairs(actors, inFlight = throughput * 2)
Behaviors.same
case Stop =>
ctx.children.foreach(ctx.stop _)
Behaviors.same
}
}
}
private def initiatePingPongForPairs(refs: Vector[(ActorRef[Message], ActorRef[Message])], inFlight: Int): Unit = {
for {
(ping, pong) <- refs
message = Message(pong) // just allocate once
_ <- 1 to inFlight
} ping ! message
}
private def startPingPongActorPairs(
ctx: ActorContext[_],
messagesPerPair: Int,
numPairs: Int,
dispatcher: String): (Vector[(ActorRef[Message], ActorRef[Message])], CountDownLatch) = {
val fullPathToDispatcher = "pekko.actor." + dispatcher
val latch = new CountDownLatch(numPairs * 2)
val pingPongBehavior = newPingPongBehavior(messagesPerPair, latch)
val pingPongProps = Props.empty.withDispatcherFromConfig(fullPathToDispatcher)
val actors = for {
_ <- (1 to numPairs).toVector
} yield {
val ping = ctx.spawnAnonymous(pingPongBehavior, pingPongProps)
val pong = ctx.spawnAnonymous(pingPongBehavior, pingPongProps)
(ping, pong)
}
(actors, latch)
}
case class Message(replyTo: ActorRef[Message])
private def newPingPongBehavior(messagesPerPair: Int, latch: CountDownLatch): Behavior[Message] =
Behaviors.setup { ctx =>
var left = messagesPerPair / 2
val pong = Message(ctx.self) // we re-use a single pong to avoid alloc on each msg
Behaviors.receiveMessage[Message] {
case Message(replyTo) =>
replyTo ! pong
if (left == 0) {
latch.countDown()
Behaviors.stopped // note that this will likely lead to dead letters
} else {
left -= 1
Behaviors.same
}
}
}
def printProgress(totalMessages: Long, numActors: Int, startNanoTime: Long) = {
val durationMicros = (System.nanoTime() - startNanoTime) / 1000
println(
f" $totalMessages messages by $numActors actors took ${durationMicros / 1000} ms, " +
f"${totalMessages.toDouble / durationMicros}%,.2f M msg/s")
}
}

View file

@ -0,0 +1,138 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor.typed
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.Future
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations.Benchmark
import org.openjdk.jmh.annotations.BenchmarkMode
import org.openjdk.jmh.annotations.Fork
import org.openjdk.jmh.annotations.Level
import org.openjdk.jmh.annotations.Measurement
import org.openjdk.jmh.annotations.Mode
import org.openjdk.jmh.annotations.OperationsPerInvocation
import org.openjdk.jmh.annotations.Param
import org.openjdk.jmh.annotations.Scope
import org.openjdk.jmh.annotations.Setup
import org.openjdk.jmh.annotations.State
import org.openjdk.jmh.annotations.TearDown
import org.openjdk.jmh.annotations.Threads
import org.openjdk.jmh.annotations.Warmup
import org.apache.pekko
import pekko.actor.typed.scaladsl.AskPattern._
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(1)
@Threads(1)
@Warmup(iterations = 10, time = 5, timeUnit = TimeUnit.SECONDS, batchSize = 1)
@Measurement(iterations = 10, time = 15, timeUnit = TimeUnit.SECONDS, batchSize = 1)
class TypedForkJoinActorBenchmark {
import TypedBenchmarkActors._
import TypedForkJoinActorBenchmark._
@Param(Array("50"))
var tpt = 0
@Param(Array(coresStr)) // coresStr, cores2xStr, cores4xStr
var threads = ""
@Param(
Array(
"pekko.dispatch.UnboundedMailbox",
"pekko.dispatch.SingleConsumerOnlyUnboundedMailbox",
"pekko.actor.ManyToOneArrayMailbox",
"pekko.actor.JCToolsMailbox"))
var mailbox = ""
implicit var system: ActorSystem[PingPongCommand] = _
@Setup(Level.Trial)
def setup(): Unit = {
pekko.actor.BenchmarkActors.requireRightNumberOfCores(cores)
system = ActorSystem(
TypedBenchmarkActors.benchmarkPingPongSupervisor(),
"TypedForkJoinActorBenchmark",
ConfigFactory.parseString(s"""
pekko.actor {
default-mailbox.mailbox-capacity = 512
fjp-dispatcher {
executor = "fork-join-executor"
fork-join-executor {
parallelism-min = $threads
parallelism-factor = 1.0
parallelism-max = $threads
}
throughput = $tpt
mailbox-type = "$mailbox"
}
}
"""))
}
@Benchmark
@OperationsPerInvocation(totalMessagesLessThanCores)
def pingPongLessActorsThanCores(): Unit =
runPingPongBench(messages, lessThanCoresActors, "fjp-dispatcher", tpt)
@Benchmark
@OperationsPerInvocation(totalMessagesSameAsCores)
def pingPongSameNumberOfActorsAsCores(): Unit =
runPingPongBench(messages, sameAsCoresActors, "fjp-dispatcher", tpt)
@Benchmark
@OperationsPerInvocation(totalMessagesMoreThanCores)
def pingPongMoreActorsThanCores(): Unit =
runPingPongBench(messages, moreThanCoresActors, "fjp-dispatcher", tpt)
def runPingPongBench(numMessages: Int, numActors: Int, dispatcher: String, tpt: Int): Unit = {
val response: Future[PingPongStarted] =
system.ask[PingPongStarted](ref => StartPingPong(numMessages, numActors, dispatcher, tpt, timeout, ref))(
timeout,
system.scheduler)
val started = Await.result(response, timeout)
started.completedLatch.await(timeout.toSeconds, TimeUnit.SECONDS)
printProgress(started.totalNumMessages, numActors, started.startNanoTime)
system ! Stop
}
@TearDown(Level.Trial)
def shutdown(): Unit = {
system.terminate()
Await.ready(system.whenTerminated, 15.seconds)
}
}
object TypedForkJoinActorBenchmark {
final val messages = 2000000 // messages per actor pair
val timeout = 30.seconds
// Constants because they are used in annotations
// update according to cpu
final val cores = 8
final val coresStr = "8"
final val cores2xStr = "16"
final val cores4xStr = "24"
final val twoActors = 2
final val moreThanCoresActors = cores * 2
final val lessThanCoresActors = cores / 2
final val sameAsCoresActors = cores
final val totalMessagesTwoActors = messages
final val totalMessagesMoreThanCores = (moreThanCoresActors * messages) / 2
final val totalMessagesLessThanCores = (lessThanCoresActors * messages) / 2
final val totalMessagesSameAsCores = (sameAsCoresActors * messages) / 2
}

View file

@ -0,0 +1,269 @@
/*
* Copyright (C) 2020-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.actor.typed.delivery
import java.util.UUID
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.util.Failure
import scala.util.Success
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.Done
import pekko.actor.typed.ActorRef
import pekko.actor.typed.ActorSystem
import pekko.actor.typed.Behavior
import pekko.actor.typed.delivery.ProducerController.MessageWithConfirmation
import pekko.actor.typed.receptionist.ServiceKey
import pekko.actor.typed.scaladsl.AskPattern._
import pekko.actor.typed.scaladsl.Behaviors
object Producer {
trait Command
case object Run extends Command
private case class WrappedRequestNext(r: ProducerController.RequestNext[Consumer.Command]) extends Command
private case object AskReply extends Command
private implicit val askTimeout: pekko.util.Timeout = 5.seconds
def apply(
numberOfMessages: Int,
useAsk: Boolean,
producerController: ActorRef[ProducerController.Command[Consumer.Command]]): Behavior[Command] = {
Behaviors.setup { context =>
val requestNextAdapter =
context.messageAdapter[ProducerController.RequestNext[Consumer.Command]](WrappedRequestNext(_))
Behaviors.receiveMessagePartial {
case WrappedRequestNext(next) =>
if (next.confirmedSeqNr >= numberOfMessages) {
context.log.info("Completed {} messages", numberOfMessages)
Behaviors.stopped
} else if (useAsk) {
context.ask[MessageWithConfirmation[Consumer.Command], ProducerController.SeqNr](
next.askNextTo,
askReplyTo => MessageWithConfirmation(Consumer.TheMessage, askReplyTo)) {
case Success(_) => AskReply
case Failure(e) => throw e
}
Behaviors.same
} else {
next.sendNextTo ! Consumer.TheMessage
Behaviors.same
}
case Run =>
context.log.info("Starting {} messages", numberOfMessages)
producerController ! ProducerController.Start(requestNextAdapter)
Behaviors.same
case AskReply =>
Behaviors.same
}
}
}
}
object Consumer {
trait Command
case object TheMessage extends Command
private case class WrappedDelivery(d: ConsumerController.Delivery[Command]) extends Command
def serviceKey(testName: String): ServiceKey[ConsumerController.Command[Command]] =
ServiceKey[ConsumerController.Command[Consumer.Command]](testName)
def apply(consumerController: ActorRef[ConsumerController.Command[Command]]): Behavior[Command] = {
Behaviors.setup { context =>
val traceEnabled = context.log.isTraceEnabled
val deliveryAdapter =
context.messageAdapter[ConsumerController.Delivery[Command]](WrappedDelivery(_))
consumerController ! ConsumerController.Start(deliveryAdapter)
Behaviors.receiveMessagePartial {
case WrappedDelivery(d @ ConsumerController.Delivery(_, confirmTo)) =>
if (traceEnabled)
context.log.trace("Processed {}", d.seqNr)
confirmTo ! ConsumerController.Confirmed
Behaviors.same
}
}
}
}
object WorkPullingProducer {
trait Command
case object Run extends Command
private case class WrappedRequestNext(r: WorkPullingProducerController.RequestNext[Consumer.Command]) extends Command
def apply(
numberOfMessages: Int,
producerController: ActorRef[WorkPullingProducerController.Command[Consumer.Command]]): Behavior[Command] = {
Behaviors.setup { context =>
val requestNextAdapter =
context.messageAdapter[WorkPullingProducerController.RequestNext[Consumer.Command]](WrappedRequestNext(_))
var remaining = numberOfMessages + context.system.settings.config
.getInt("pekko.reliable-delivery.consumer-controller.flow-control-window")
Behaviors.receiveMessagePartial {
case WrappedRequestNext(next) =>
remaining -= 1
if (remaining == 0) {
context.log.info("Completed {} messages", numberOfMessages)
Behaviors.stopped
} else {
next.sendNextTo ! Consumer.TheMessage
Behaviors.same
}
case Run =>
context.log.info("Starting {} messages", numberOfMessages)
producerController ! WorkPullingProducerController.Start(requestNextAdapter)
Behaviors.same
}
}
}
}
object Guardian {
trait Command
final case class RunPointToPoint(id: String, numberOfMessages: Int, useAsk: Boolean, replyTo: ActorRef[Done])
extends Command
final case class RunWorkPulling(id: String, numberOfMessages: Int, workers: Int, replyTo: ActorRef[Done])
extends Command
final case class ProducerTerminated(consumers: List[ActorRef[Consumer.Command]], replyTo: ActorRef[Done])
extends Command
def apply(): Behavior[Command] = {
Behaviors.setup { context =>
Behaviors.receiveMessage {
case RunPointToPoint(id, numberOfMessages, useAsk, replyTo) =>
// point-to-point
val consumerController =
context.spawn(ConsumerController[Consumer.Command](), s"consumerController-$id")
val consumers = List(context.spawn(Consumer(consumerController), s"consumer-$id"))
val producerController = context.spawn(
ProducerController[Consumer.Command](id, durableQueueBehavior = None),
s"producerController-$id")
val producer = context.spawn(Producer(numberOfMessages, useAsk, producerController), s"producer-$id")
consumerController ! ConsumerController.RegisterToProducerController(producerController)
context.watchWith(producer, ProducerTerminated(consumers, replyTo))
producer ! Producer.Run
Behaviors.same
case RunWorkPulling(id, numberOfMessages, workers, replyTo) =>
// workPulling
val sKey = Consumer.serviceKey(id)
val consumerController =
context.spawn(ConsumerController[Consumer.Command](sKey), s"consumerController-$id")
val consumers = (1 to workers).map { n =>
context.spawn(Consumer(consumerController), s"consumer-$n-$id")
}.toList
val producerController = context.spawn(
WorkPullingProducerController[Consumer.Command](id, sKey, durableQueueBehavior = None),
s"producerController-$id")
val producer = context.spawn(WorkPullingProducer(numberOfMessages, producerController), s"producer-$id")
context.watchWith(producer, ProducerTerminated(consumers, replyTo))
producer ! WorkPullingProducer.Run
Behaviors.same
case ProducerTerminated(consumers, replyTo) =>
consumers.foreach(context.stop)
replyTo ! Done
Behaviors.same
case msg =>
throw new RuntimeException(s"Unexpected message $msg")
}
}
}
}
object ReliableDeliveryBenchmark {
final val messagesPerOperation = 100000
final val timeout = 30.seconds
}
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(1)
@Threads(1)
@Warmup(iterations = 10, time = 5, timeUnit = TimeUnit.SECONDS, batchSize = 1)
@Measurement(iterations = 10, time = 10, timeUnit = TimeUnit.SECONDS, batchSize = 1)
class ReliableDeliveryBenchmark {
import ReliableDeliveryBenchmark._
@Param(Array("50"))
var window = 0
implicit var system: ActorSystem[Guardian.Command] = _
implicit val askTimeout: pekko.util.Timeout = pekko.util.Timeout(timeout)
@Setup(Level.Trial)
def setup(): Unit = {
system = ActorSystem(
Guardian(),
"ReliableDeliveryBenchmark",
ConfigFactory.parseString(s"""
pekko.loglevel = INFO
pekko.reliable-delivery {
consumer-controller.flow-control-window = $window
}
"""))
}
@TearDown(Level.Trial)
def shutdown(): Unit = {
system.terminate()
Await.ready(system.whenTerminated, 15.seconds)
}
@Benchmark
@OperationsPerInvocation(messagesPerOperation)
def pointToPoint(): Unit = {
Await.result(
system.ask(
Guardian.RunPointToPoint(s"point-to-point-${UUID.randomUUID()}", messagesPerOperation, useAsk = false, _)),
timeout)
}
@Benchmark
@OperationsPerInvocation(messagesPerOperation)
def pointToPointAsk(): Unit = {
Await.result(
system.ask(
Guardian.RunPointToPoint(s"point-to-point-${UUID.randomUUID()}", messagesPerOperation, useAsk = true, _)),
timeout)
}
@Benchmark
@OperationsPerInvocation(messagesPerOperation)
def workPulling1(): Unit = {
Await.result(
system.ask(Guardian.RunWorkPulling(s"work-pulling-${UUID.randomUUID()}", messagesPerOperation, workers = 1, _)),
timeout)
}
@Benchmark
@OperationsPerInvocation(messagesPerOperation)
def workPulling2(): Unit = {
Await.result(
system.ask(Guardian.RunWorkPulling(s"work-pulling-${UUID.randomUUID()}", messagesPerOperation, workers = 2, _)),
timeout)
}
}

View file

@ -0,0 +1,83 @@
/*
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.ddata
import java.util.concurrent.TimeUnit
import org.openjdk.jmh.annotations.{ Scope => JmhScope }
import org.openjdk.jmh.annotations.Benchmark
import org.openjdk.jmh.annotations.BenchmarkMode
import org.openjdk.jmh.annotations.Fork
import org.openjdk.jmh.annotations.Level
import org.openjdk.jmh.annotations.Measurement
import org.openjdk.jmh.annotations.Mode
import org.openjdk.jmh.annotations.OutputTimeUnit
import org.openjdk.jmh.annotations.Param
import org.openjdk.jmh.annotations.Setup
import org.openjdk.jmh.annotations.State
import org.openjdk.jmh.annotations.Warmup
import org.apache.pekko
import pekko.actor.Address
import pekko.cluster.UniqueAddress
@Fork(2)
@State(JmhScope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Warmup(iterations = 4)
@Measurement(iterations = 5)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
class ORSetMergeBenchmark {
@Param(Array("1", "10", "20", "100"))
var set1Size = 0
val nodeA = UniqueAddress(Address("akka", "Sys", "aaaa", 2552), 1L)
val nodeB = UniqueAddress(nodeA.address.copy(host = Some("bbbb")), 2L)
val nodeC = UniqueAddress(nodeA.address.copy(host = Some("cccc")), 3L)
val nodeD = UniqueAddress(nodeA.address.copy(host = Some("dddd")), 4L)
val nodeE = UniqueAddress(nodeA.address.copy(host = Some("eeee")), 5L)
val nodes = Vector(nodeA, nodeB, nodeC, nodeD, nodeE)
val nodesIndex = Iterator.from(0)
def nextNode(): UniqueAddress = nodes(nodesIndex.next() % nodes.size)
var set1: ORSet[String] = _
var addFromSameNode: ORSet[String] = _
var addFromOtherNode: ORSet[String] = _
var complex1: ORSet[String] = _
var complex2: ORSet[String] = _
var elem1: String = _
var elem2: String = _
@Setup(Level.Trial)
def setup(): Unit = {
set1 = (1 to set1Size).foldLeft(ORSet.empty[String])((s, n) => s.add(nextNode(), "elem" + n))
addFromSameNode = set1.add(nodeA, "elem" + set1Size + 1).merge(set1)
addFromOtherNode = set1.add(nodeB, "elem" + set1Size + 1).merge(set1)
complex1 = set1.add(nodeB, "a").add(nodeC, "b").remove(nodeD, "elem" + set1Size).merge(set1)
complex2 = set1.add(nodeA, "a").add(nodeA, "c").add(nodeB, "d").merge(set1)
elem1 = "elem" + (set1Size + 1)
elem2 = "elem" + (set1Size + 2)
}
@Benchmark
def mergeAddFromSameNode: ORSet[String] = {
// this is the scenario when updating and then merging with local value
// set2 produced by modify function
val set2 = set1.add(nodeA, elem1).add(nodeA, elem2)
// replicator merges with local value
set1.merge(set2)
}
@Benchmark
def mergeAddFromOtherNode: ORSet[String] = set1.merge(addFromOtherNode)
@Benchmark
def mergeAddFromBothNodes: ORSet[String] = addFromSameNode.merge(addFromOtherNode)
@Benchmark
def mergeComplex: ORSet[String] = complex1.merge(complex2)
}

View file

@ -0,0 +1,76 @@
/*
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.ddata
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations.{ Scope => JmhScope }
import org.openjdk.jmh.annotations.Benchmark
import org.openjdk.jmh.annotations.BenchmarkMode
import org.openjdk.jmh.annotations.Fork
import org.openjdk.jmh.annotations.Measurement
import org.openjdk.jmh.annotations.Mode
import org.openjdk.jmh.annotations.OutputTimeUnit
import org.openjdk.jmh.annotations.State
import org.openjdk.jmh.annotations.TearDown
import org.openjdk.jmh.annotations.Warmup
import org.apache.pekko
import pekko.actor.ActorRef
import pekko.actor.ActorSystem
import pekko.actor.Props
import pekko.cluster.Cluster
import pekko.serialization.SerializationExtension
import pekko.serialization.Serializers
@Fork(2)
@State(JmhScope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Warmup(iterations = 4)
@Measurement(iterations = 5)
@OutputTimeUnit(TimeUnit.SECONDS)
class ORSetSerializationBenchmark {
private val config = ConfigFactory.parseString("""
pekko.actor.provider=cluster
pekko.remote.classic.netty.tcp.port=0
pekko.remote.artery.canonical.port = 0
""")
private val system1 = ActorSystem("ORSetSerializationBenchmark", config)
private val system2 = ActorSystem("ORSetSerializationBenchmark", config)
private val ref1 = (1 to 10).map(n => system1.actorOf(Props.empty, s"ref1-$n"))
private val ref2 = (1 to 10).map(n => system2.actorOf(Props.empty, s"ref2-$n"))
private val orSet = {
val selfUniqueAddress1 = SelfUniqueAddress(Cluster(system1).selfUniqueAddress)
val selfUniqueAddress2 = SelfUniqueAddress(Cluster(system2).selfUniqueAddress)
val set1 = ref1.foldLeft(ORSet.empty[ActorRef]) { case (acc, r) => acc.add(selfUniqueAddress1, r) }
val set2 = ref2.foldLeft(ORSet.empty[ActorRef]) { case (acc, r) => acc.add(selfUniqueAddress2, r) }
set1.merge(set2)
}
private val serialization = SerializationExtension(system1)
private val serializerId = serialization.findSerializerFor(orSet).identifier
private val manifest = Serializers.manifestFor(serialization.findSerializerFor(orSet), orSet)
@TearDown
def shutdown(): Unit = {
Await.result(system1.terminate(), 5.seconds)
Await.result(system2.terminate(), 5.seconds)
}
@Benchmark
def serializeRoundtrip: ORSet[ActorRef] = {
val bytes = serialization.serialize(orSet).get
serialization.deserialize(bytes, serializerId, manifest).get.asInstanceOf[ORSet[ActorRef]]
}
}

View file

@ -0,0 +1,80 @@
/*
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.ddata
import java.util.concurrent.TimeUnit
import org.openjdk.jmh.annotations.{ Scope => JmhScope }
import org.openjdk.jmh.annotations.Benchmark
import org.openjdk.jmh.annotations.BenchmarkMode
import org.openjdk.jmh.annotations.Fork
import org.openjdk.jmh.annotations.Level
import org.openjdk.jmh.annotations.Measurement
import org.openjdk.jmh.annotations.Mode
import org.openjdk.jmh.annotations.OutputTimeUnit
import org.openjdk.jmh.annotations.Param
import org.openjdk.jmh.annotations.Setup
import org.openjdk.jmh.annotations.State
import org.openjdk.jmh.annotations.Warmup
import org.apache.pekko
import pekko.actor.Address
import pekko.cluster.UniqueAddress
@Fork(2)
@State(JmhScope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Warmup(iterations = 4)
@Measurement(iterations = 5)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
class VersionVectorBenchmark {
@Param(Array("1", "2", "5"))
var size = 0
val nodeA = UniqueAddress(Address("akka", "Sys", "aaaa", 2552), 1L)
val nodeB = UniqueAddress(nodeA.address.copy(host = Some("bbbb")), 2L)
val nodeC = UniqueAddress(nodeA.address.copy(host = Some("cccc")), 3L)
val nodeD = UniqueAddress(nodeA.address.copy(host = Some("dddd")), 4L)
val nodeE = UniqueAddress(nodeA.address.copy(host = Some("eeee")), 5L)
val nodes = Vector(nodeA, nodeB, nodeC, nodeD, nodeE)
val nodesIndex = Iterator.from(0)
def nextNode(): UniqueAddress = nodes(nodesIndex.next() % nodes.size)
var vv1: VersionVector = _
var vv2: VersionVector = _
var vv3: VersionVector = _
var dot1: VersionVector = _
@Setup(Level.Trial)
def setup(): Unit = {
vv1 = (1 to size).foldLeft(VersionVector.empty)((vv, _) => vv + nextNode())
vv2 = vv1 + nextNode()
vv3 = vv1 + nextNode()
dot1 = VersionVector(nodeA, vv1.versionAt(nodeA))
}
@Benchmark
def increment: VersionVector = vv1 + nodeA
@Benchmark
def compareSame1: Boolean = vv1 == dot1
@Benchmark
def compareSame2: Boolean = vv2 == dot1
@Benchmark
def compareGreaterThan1: Boolean = vv1 > dot1
@Benchmark
def compareGreaterThan2: Boolean = vv2 > dot1
@Benchmark
def merge: VersionVector = vv1.merge(vv2)
@Benchmark
def mergeConflicting: VersionVector = vv2.merge(vv3)
}

View file

@ -0,0 +1,28 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.dispatch
import java.util.concurrent.TimeUnit
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(2)
@Warmup(iterations = 10)
@Measurement(iterations = 20)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
class CachingConfigBenchmark {
val deepKey = "pekko.actor.deep.settings.something"
val deepConfigString = s"""$deepKey = something"""
val deepConfig = ConfigFactory.parseString(deepConfigString)
val deepCaching = new CachingConfig(deepConfig)
@Benchmark def deep_config = deepConfig.hasPath(deepKey)
@Benchmark def deep_caching = deepCaching.hasPath(deepKey)
}

View file

@ -0,0 +1,77 @@
/*
* Copyright (C) 2016-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.dispatch
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.actor._
import pekko.testkit.TestProbe
object NodeQueueBenchmark {
final val burst = 100000
case object Stop
}
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(2)
@Warmup(iterations = 5)
@Measurement(iterations = 10)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
class NodeQueueBenchmark {
import NodeQueueBenchmark._
val config = ConfigFactory.parseString("""
dispatcher {
executor = "thread-pool-executor"
throughput = 1000
thread-pool-executor {
fixed-pool-size = 1
}
}
mailbox {
mailbox-type = "org.apache.pekko.dispatch.SingleConsumerOnlyUnboundedMailbox"
mailbox-capacity = 1000000
}
""").withFallback(ConfigFactory.load())
implicit val sys: ActorSystem = ActorSystem("ANQ", config)
val ref = sys.actorOf(Props(new Actor {
def receive = {
case Stop => sender() ! Stop
case _ =>
}
}).withDispatcher("dispatcher").withMailbox("mailbox"), "receiver")
@TearDown
def teardown(): Unit = Await.result(sys.terminate(), 5.seconds)
@TearDown(Level.Invocation)
def waitInBetween(): Unit = {
val probe = TestProbe()
probe.send(ref, Stop)
probe.expectMsg(Stop)
System.gc()
System.gc()
System.gc()
}
@Benchmark
@OperationsPerInvocation(burst)
def send(): Unit = {
var todo = burst
while (todo > 0) {
ref ! "hello"
todo -= 1
}
}
}

View file

@ -0,0 +1,55 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.event
import java.util.concurrent.TimeUnit
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.event.Logging.LogLevel
@Fork(3)
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Warmup(iterations = 10)
@Measurement(iterations = 20, timeUnit = TimeUnit.MILLISECONDS)
class LogLevelAccessBenchmark {
/*
volatile logLevel, guard on loggers
20 readers, 2 writers
a.e.LogLevelAccessBenchmark.g thrpt 60 1862566.204 37860.541 ops/ms
a.e.LogLevelAccessBenchmark.g:readLogLevel thrpt 60 1860031.729 37834.335 ops/ms
a.e.LogLevelAccessBenchmark.g:writeLogLevel_1 thrpt 60 1289.452 45.403 ops/ms
a.e.LogLevelAccessBenchmark.g:writeLogLevel_2 thrpt 60 1245.023 51.071 ops/ms
*/
val NoopBus = new LoggingBus {
override def subscribe(subscriber: Subscriber, to: Classifier): Boolean = true
override def publish(event: Event): Unit = ()
override def unsubscribe(subscriber: Subscriber, from: Classifier): Boolean = true
override def unsubscribe(subscriber: Subscriber): Unit = ()
}
var log: BusLogging = pekko.event.Logging(NoopBus, "").asInstanceOf[BusLogging]
@Benchmark
@GroupThreads(20)
@Group("g")
def readLogLevel(): LogLevel =
log.bus.logLevel
@Benchmark
@Group("g")
def setLogLevel_1(): Unit =
log.bus.setLogLevel(Logging.ErrorLevel)
@Benchmark
@Group("g")
def setLogLevel_2(): Unit =
log.bus.setLogLevel(Logging.DebugLevel)
}

View file

@ -0,0 +1,16 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.persistence
import org.apache.pekko.actor.Actor
/** only as a "the best we could possibly get" baseline, does not persist anything */
class BaselineActor(respondAfter: Int) extends Actor {
override def receive = {
case n: Int => if (n == respondAfter) sender() ! n
}
}
final case class Evt(i: Int)

View file

@ -0,0 +1,115 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.persistence
import java.io.File
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import org.apache.commons.io.FileUtils
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.actor._
import pekko.persistence.journal.AsyncWriteTarget._
import pekko.persistence.journal.leveldb.{ SharedLeveldbJournal, SharedLeveldbStore }
import pekko.testkit.TestProbe
import scala.annotation.nowarn
/*
# OS: OSX 10.9.3
# CPU: Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz
# Date: Mon Jul 23 11:07:42 CEST 2014
This bench emulates what we provide with "Processor batching".
As expected, batching writes is better than writing 1 by 1.
The important thing though is that there didn't appear to be any "write latency spikes" throughout this bench.
[info] Benchmark Mode Samples Score Score error Units
[info] a.p.LevelDbBatchingBenchmark.write_1 avgt 20 0.799 0.011 ms/op
[info] a.p.LevelDbBatchingBenchmark.writeBatch_10 avgt 20 0.117 0.001 ms/op
[info] a.p.LevelDbBatchingBenchmark.writeBatch_100 avgt 20 0.050 0.000 ms/op
[info] a.p.LevelDbBatchingBenchmark.writeBatch_200 avgt 20 0.041 0.001 ms/op
*/
@Fork(1)
@Threads(10)
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.AverageTime))
class LevelDbBatchingBenchmark {
var sys: ActorSystem = _
var probe: TestProbe = _
var store: ActorRef = _
val batch_1 = List.fill(1) { AtomicWrite(PersistentRepr("data", 12, "pa")) }
val batch_10 = List.fill(10) { AtomicWrite(PersistentRepr("data", 12, "pa")) }
val batch_100 = List.fill(100) { AtomicWrite(PersistentRepr("data", 12, "pa")) }
val batch_200 = List.fill(200) { AtomicWrite(PersistentRepr("data", 12, "pa")) }
@Setup(Level.Trial)
@nowarn("msg=deprecated")
def setup(): Unit = {
sys = ActorSystem("sys")
deleteStorage(sys)
SharedLeveldbJournal.setStore(store, sys)
probe = TestProbe()(sys)
store = sys.actorOf(Props[SharedLeveldbStore](), "store")
}
@TearDown(Level.Trial)
def tearDown(): Unit = {
store ! PoisonPill
Thread.sleep(500)
sys.terminate()
Await.ready(sys.whenTerminated, 10.seconds)
}
@Benchmark
@Measurement(timeUnit = TimeUnit.MICROSECONDS)
@OperationsPerInvocation(1)
def write_1(): Unit = {
probe.send(store, WriteMessages(batch_1))
probe.expectMsgType[Any]
}
@Benchmark
@Measurement(timeUnit = TimeUnit.MICROSECONDS)
@OperationsPerInvocation(10)
def writeBatch_10(): Unit = {
probe.send(store, WriteMessages(batch_10))
probe.expectMsgType[Any]
}
@Benchmark
@Measurement(timeUnit = TimeUnit.MICROSECONDS)
@OperationsPerInvocation(100)
def writeBatch_100(): Unit = {
probe.send(store, WriteMessages(batch_100))
probe.expectMsgType[Any]
}
@Benchmark
@Measurement(timeUnit = TimeUnit.MICROSECONDS)
@OperationsPerInvocation(200)
def writeBatch_200(): Unit = {
probe.send(store, WriteMessages(batch_200))
probe.expectMsgType[Any]
}
// TOOLS
private def deleteStorage(sys: ActorSystem): Unit = {
val storageLocations =
List(
"pekko.persistence.journal.leveldb.dir",
"pekko.persistence.journal.leveldb-shared.store.dir",
"pekko.persistence.snapshot-store.local.dir").map(s => new File(sys.settings.config.getString(s)))
storageLocations.foreach(FileUtils.deleteDirectory)
}
}

View file

@ -0,0 +1,120 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.persistence
import java.io.File
import scala.concurrent.Await
import scala.concurrent.duration._
import org.apache.commons.io.FileUtils
import org.openjdk.jmh.annotations._
import org.openjdk.jmh.annotations.Scope
import org.apache.pekko
import pekko.actor._
import pekko.testkit.TestProbe
/*
# OS: OSX 10.9.3
# CPU: Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz
# Date: Mon Jun 9 13:22:42 CEST 2014
[info] Benchmark Mode Samples Mean Mean error Units
[info] a.p.PersistentActorDeferBenchmark.tell_persistAsync_defer_persistAsync_reply thrpt 10 6.858 0.515 ops/ms
[info] a.p.PersistentActorDeferBenchmark.tell_persistAsync_defer_persistAsync_replyASAP thrpt 10 20.256 2.941 ops/ms
[info] a.p.PersistentActorDeferBenchmark.tell_processor_Persistent_reply thrpt 10 6.531 0.114 ops/ms
[info] a.p.PersistentActorDeferBenchmark.tell_processor_Persistent_replyASAP thrpt 10 26.000 0.694 ops/ms
*/
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
class PersistentActorDeferBenchmark {
val config = PersistenceSpec.config("leveldb", "benchmark")
lazy val storageLocations =
List(
"pekko.persistence.journal.leveldb.dir",
"pekko.persistence.journal.leveldb-shared.store.dir",
"pekko.persistence.snapshot-store.local.dir").map(s => new File(system.settings.config.getString(s)))
var system: ActorSystem = _
var probe: TestProbe = _
var persistAsync_defer: ActorRef = _
var persistAsync_defer_replyASAP: ActorRef = _
val data10k = (1 to 10000).toArray
@Setup
def setup(): Unit = {
system = ActorSystem("test", config)
probe = TestProbe()(system)
storageLocations.foreach(FileUtils.deleteDirectory)
persistAsync_defer = system.actorOf(Props(classOf[`persistAsync, defer`], data10k.last), "a-1")
persistAsync_defer_replyASAP =
system.actorOf(Props(classOf[`persistAsync, defer, respond ASAP`], data10k.last), "a-2")
}
@TearDown
def shutdown(): Unit = {
system.terminate()
Await.ready(system.whenTerminated, 15.seconds)
storageLocations.foreach(FileUtils.deleteDirectory)
}
@Benchmark
@OperationsPerInvocation(10000)
def tell_persistAsync_defer_persistAsync_reply(): Unit = {
for (i <- data10k) persistAsync_defer.tell(i, probe.ref)
probe.expectMsg(data10k.last)
}
@Benchmark
@OperationsPerInvocation(10000)
def tell_persistAsync_defer_persistAsync_replyASAP(): Unit = {
for (i <- data10k) persistAsync_defer_replyASAP.tell(i, probe.ref)
probe.expectMsg(data10k.last)
}
}
class `persistAsync, defer`(respondAfter: Int) extends PersistentActor {
override def persistenceId: String = self.path.name
override def receiveCommand = {
case n: Int =>
persistAsync(Evt(n)) { _ =>
}
deferAsync(Evt(n)) { e =>
if (e.i == respondAfter) sender() ! e.i
}
}
override def receiveRecover = {
case _ => // do nothing
}
}
class `persistAsync, defer, respond ASAP`(respondAfter: Int) extends PersistentActor {
override def persistenceId: String = self.path.name
override def receiveCommand = {
case n: Int =>
persistAsync(Evt(n)) { _ =>
}
deferAsync(Evt(n)) { _ =>
}
if (n == respondAfter) sender() ! n
}
override def receiveRecover = {
case _ => // do nothing
}
}

View file

@ -0,0 +1,166 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.persistence
import java.io.File
import scala.concurrent.Await
import scala.concurrent.duration._
import org.apache.commons.io.FileUtils
import org.openjdk.jmh.annotations._
import org.openjdk.jmh.annotations.Scope
import org.apache.pekko
import pekko.actor._
import pekko.testkit.TestProbe
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
class PersistentActorThroughputBenchmark {
val config = PersistenceSpec.config("leveldb", "benchmark")
lazy val storageLocations =
List(
"pekko.persistence.journal.leveldb.dir",
"pekko.persistence.journal.leveldb-shared.store.dir",
"pekko.persistence.snapshot-store.local.dir").map(s => new File(system.settings.config.getString(s)))
var system: ActorSystem = _
var probe: TestProbe = _
var actor: ActorRef = _
var persistPersistentActor: ActorRef = _
var persistAsync1PersistentActor: ActorRef = _
var noPersistPersistentActor: ActorRef = _
var persistAsyncQuickReplyPersistentActor: ActorRef = _
val data10k = (1 to 10000).toArray
@Setup
def setup(): Unit = {
system = ActorSystem("test", config)
probe = TestProbe()(system)
storageLocations.foreach(FileUtils.deleteDirectory)
actor = system.actorOf(Props(classOf[BaselineActor], data10k.last), "a-1")
noPersistPersistentActor = system.actorOf(Props(classOf[NoPersistPersistentActor], data10k.last), "nop-1")
persistPersistentActor = system.actorOf(Props(classOf[PersistPersistentActor], data10k.last), "ep-1")
persistAsync1PersistentActor = system.actorOf(Props(classOf[PersistAsyncPersistentActor], data10k.last), "epa-1")
persistAsyncQuickReplyPersistentActor =
system.actorOf(Props(classOf[PersistAsyncQuickReplyPersistentActor], data10k.last), "epa-2")
}
@TearDown
def shutdown(): Unit = {
system.terminate()
Await.ready(system.whenTerminated, 15.seconds)
storageLocations.foreach(FileUtils.deleteDirectory)
}
@Benchmark
@OperationsPerInvocation(10000)
def actor_normalActor_reply_baseline(): Unit = {
for (i <- data10k) actor.tell(i, probe.ref)
probe.expectMsg(data10k.last)
}
@Benchmark
@OperationsPerInvocation(10000)
def persistentActor_persist_reply(): Unit = {
for (i <- data10k) persistPersistentActor.tell(i, probe.ref)
probe.expectMsg(Evt(data10k.last))
}
@Benchmark
@OperationsPerInvocation(10000)
def persistentActor_persistAsync_reply(): Unit = {
for (i <- data10k) persistAsync1PersistentActor.tell(i, probe.ref)
probe.expectMsg(Evt(data10k.last))
}
@Benchmark
@OperationsPerInvocation(10000)
def persistentActor_noPersist_reply(): Unit = {
for (i <- data10k) noPersistPersistentActor.tell(i, probe.ref)
probe.expectMsg(Evt(data10k.last))
}
@Benchmark
@OperationsPerInvocation(10000)
def persistentActor_persistAsync_replyRightOnCommandReceive(): Unit = {
for (i <- data10k) persistAsyncQuickReplyPersistentActor.tell(i, probe.ref)
probe.expectMsg(Evt(data10k.last))
}
}
class NoPersistPersistentActor(respondAfter: Int) extends PersistentActor {
override def persistenceId: String = self.path.name
override def receiveCommand = {
case n: Int => if (n == respondAfter) sender() ! Evt(n)
}
override def receiveRecover = {
case _ => // do nothing
}
}
class PersistPersistentActor(respondAfter: Int) extends PersistentActor {
override def persistenceId: String = self.path.name
override def receiveCommand = {
case n: Int =>
persist(Evt(n)) { e =>
if (e.i == respondAfter) sender() ! e
}
}
override def receiveRecover = {
case _ => // do nothing
}
}
class PersistAsyncPersistentActor(respondAfter: Int) extends PersistentActor {
override def persistenceId: String = self.path.name
override def receiveCommand = {
case n: Int =>
persistAsync(Evt(n)) { e =>
if (e.i == respondAfter) sender() ! e
}
}
override def receiveRecover = {
case _ => // do nothing
}
}
class PersistAsyncQuickReplyPersistentActor(respondAfter: Int) extends PersistentActor {
override def persistenceId: String = self.path.name
override def receiveCommand = {
case n: Int =>
val e = Evt(n)
if (n == respondAfter) sender() ! e
persistAsync(e)(identity)
}
override def receiveRecover = {
case _ => // do nothing
}
}

View file

@ -0,0 +1,237 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.persistence
import java.io.File
import scala.concurrent.Await
import scala.concurrent.duration._
import org.apache.commons.io.FileUtils
import org.openjdk.jmh.annotations._
import org.openjdk.jmh.annotations.Scope
import org.apache.pekko
import pekko.actor._
import pekko.testkit.TestProbe
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
class PersistentActorWithAtLeastOnceDeliveryBenchmark {
val config = PersistenceSpec.config("leveldb", "benchmark")
lazy val storageLocations =
List(
"pekko.persistence.journal.leveldb.dir",
"pekko.persistence.journal.leveldb-shared.store.dir",
"pekko.persistence.snapshot-store.local.dir").map(s => new File(system.settings.config.getString(s)))
var system: ActorSystem = _
var probe: TestProbe = _
var actor: ActorRef = _
var persistPersistentActorWithAtLeastOnceDelivery: ActorRef = _
var persistAsyncPersistentActorWithAtLeastOnceDelivery: ActorRef = _
var noPersistPersistentActorWithAtLeastOnceDelivery: ActorRef = _
var destinationActor: ActorRef = _
val dataCount = 10000
@Setup
def setup(): Unit = {
system = ActorSystem("PersistentActorWithAtLeastOnceDeliveryBenchmark", config)
probe = TestProbe()(system)
storageLocations.foreach(FileUtils.deleteDirectory)
destinationActor = system.actorOf(Props[DestinationActor](), "destination")
noPersistPersistentActorWithAtLeastOnceDelivery = system.actorOf(
Props(classOf[NoPersistPersistentActorWithAtLeastOnceDelivery], dataCount, probe.ref, destinationActor.path),
"nop-1")
persistPersistentActorWithAtLeastOnceDelivery = system.actorOf(
Props(classOf[PersistPersistentActorWithAtLeastOnceDelivery], dataCount, probe.ref, destinationActor.path),
"ep-1")
persistAsyncPersistentActorWithAtLeastOnceDelivery = system.actorOf(
Props(classOf[PersistAsyncPersistentActorWithAtLeastOnceDelivery], dataCount, probe.ref, destinationActor.path),
"epa-1")
}
@TearDown
def shutdown(): Unit = {
system.terminate()
Await.ready(system.whenTerminated, 15.seconds)
storageLocations.foreach(FileUtils.deleteDirectory)
}
@Benchmark
@OperationsPerInvocation(10000)
def persistentActor_persistAsync_with_AtLeastOnceDelivery(): Unit = {
for (i <- 1 to dataCount)
persistAsyncPersistentActorWithAtLeastOnceDelivery.tell(i, probe.ref)
probe.expectMsg(20.seconds, Evt(dataCount))
}
@Benchmark
@OperationsPerInvocation(10000)
def persistentActor_persist_with_AtLeastOnceDelivery(): Unit = {
for (i <- 1 to dataCount)
persistPersistentActorWithAtLeastOnceDelivery.tell(i, probe.ref)
probe.expectMsg(2.minutes, Evt(dataCount))
}
@Benchmark
@OperationsPerInvocation(10000)
def persistentActor_noPersist_with_AtLeastOnceDelivery(): Unit = {
for (i <- 1 to dataCount)
noPersistPersistentActorWithAtLeastOnceDelivery.tell(i, probe.ref)
probe.expectMsg(20.seconds, Evt(dataCount))
}
}
class NoPersistPersistentActorWithAtLeastOnceDelivery(
respondAfter: Int,
val upStream: ActorRef,
val downStream: ActorPath)
extends PersistentActor
with AtLeastOnceDelivery {
override def redeliverInterval = 100.milliseconds
override def persistenceId: String = self.path.name
override def receiveCommand = {
case n: Int =>
deliver(downStream)(deliveryId => Msg(deliveryId, n))
if (n == respondAfter)
// switch to wait all message confirmed
context.become(waitConfirm)
case Confirm(deliveryId) =>
confirmDelivery(deliveryId)
case _ => // do nothing
}
override def receiveRecover = {
case _ => // do nothing
}
val waitConfirm: Actor.Receive = {
case Confirm(deliveryId) =>
confirmDelivery(deliveryId)
if (numberOfUnconfirmed == 0) {
upStream ! Evt(respondAfter)
context.unbecome()
}
case _ => // do nothing
}
}
class PersistPersistentActorWithAtLeastOnceDelivery(
respondAfter: Int,
val upStream: ActorRef,
val downStream: ActorPath)
extends PersistentActor
with AtLeastOnceDelivery {
override def redeliverInterval = 100.milliseconds
override def persistenceId: String = self.path.name
override def receiveCommand = {
case n: Int =>
persist(MsgSent(n)) { _ =>
deliver(downStream)(deliveryId => Msg(deliveryId, n))
if (n == respondAfter)
// switch to wait all message confirmed
context.become(waitConfirm)
}
case Confirm(deliveryId) =>
confirmDelivery(deliveryId)
case _ => // do nothing
}
override def receiveRecover = {
case _ => // do nothing
}
val waitConfirm: Actor.Receive = {
case Confirm(deliveryId) =>
confirmDelivery(deliveryId)
if (numberOfUnconfirmed == 0) {
upStream ! Evt(respondAfter)
context.unbecome()
}
case _ => // do nothing
}
}
class PersistAsyncPersistentActorWithAtLeastOnceDelivery(
respondAfter: Int,
val upStream: ActorRef,
val downStream: ActorPath)
extends PersistentActor
with AtLeastOnceDelivery {
override def redeliverInterval = 100.milliseconds
override def persistenceId: String = self.path.name
override def receiveCommand = {
case n: Int =>
persistAsync(MsgSent(n)) { _ =>
deliver(downStream)(deliveryId => Msg(deliveryId, n))
if (n == respondAfter)
// switch to wait all message confirmed
context.become(waitConfirm)
}
case Confirm(deliveryId) =>
confirmDelivery(deliveryId)
case _ => // do nothing
}
override def receiveRecover = {
case _ => // do nothing
}
val waitConfirm: Actor.Receive = {
case Confirm(deliveryId) =>
confirmDelivery(deliveryId)
if (numberOfUnconfirmed == 0) {
upStream ! Evt(respondAfter)
context.unbecome()
}
case _ => // do nothing
}
}
case class Msg(deliveryId: Long, n: Int)
case class Confirm(deliveryId: Long)
sealed trait Event
case class MsgSent(n: Int) extends Event
case class MsgConfirmed(deliveryId: Long) extends Event
class DestinationActor extends Actor {
var seqNr = 0L
override def receive = {
case n: Int =>
sender() ! Confirm(n)
case Msg(deliveryId, _) =>
seqNr += 1
if (seqNr % 11 == 0) {
// drop it
} else {
sender() ! Confirm(deliveryId)
}
case _ => // do nothing
}
}

View file

@ -0,0 +1,65 @@
/*
* Copyright (C) 2016-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.remote.artery
import org.apache.pekko
import pekko.stream.Attributes
import pekko.stream.Outlet
import pekko.stream.SourceShape
import pekko.stream.stage.GraphStage
import pekko.stream.stage.GraphStageLogic
import pekko.stream.stage.OutHandler
/**
* Emits integers from 1 to the given `elementCount`. The `java.lang.Integer`
* objects are allocated in the constructor of the operator, so it should be created
* before the benchmark is started.
*/
class BenchTestSource(elementCount: Int) extends GraphStage[SourceShape[java.lang.Integer]] {
private val elements = new Array[java.lang.Integer](elementCount)
(1 to elementCount).foreach(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)
}
}

View file

@ -0,0 +1,338 @@
/*
* Copyright (C) 2016-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.remote.artery
import java.io.IOException
import java.nio.ByteBuffer
import java.nio.ByteOrder
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
import scala.annotation.tailrec
import scala.concurrent.Await
import scala.concurrent.Future
import scala.concurrent.duration._
import scala.annotation.nowarn
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.Done
import pekko.NotUsed
import pekko.actor._
import pekko.remote._
import pekko.remote.artery.Decoder.InboundCompressionAccess
import pekko.remote.artery.compress._
import pekko.serialization.BaseSerializer
import pekko.serialization.ByteBufferSerializer
import pekko.serialization.SerializationExtension
import pekko.stream.SystemMaterializer
import pekko.stream.scaladsl._
import pekko.util.OptionVal
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(2)
@Warmup(iterations = 4)
@Measurement(iterations = 5)
class CodecBenchmark {
import CodecBenchmark._
@nowarn("msg=immutable val") // JMH updates this via reflection
@Param(Array(Standard, RemoteInstrument))
private var configType: String = _
implicit var system: ActorSystem = _
var systemB: ActorSystem = _
private val envelopePool = new EnvelopeBufferPool(1024 * 1024, 128)
private val inboundEnvelopePool = ReusableInboundEnvelope.createObjectPool(capacity = 16)
private val outboundEnvelopePool = ReusableOutboundEnvelope.createObjectPool(capacity = 16)
val headerIn = HeaderBuilder.in(NoInboundCompressions)
val envelopeTemplateBuffer = ByteBuffer.allocate(1024 * 1024).order(ByteOrder.LITTLE_ENDIAN)
var uniqueLocalAddress: UniqueAddress = _
val payload = DummyMessageInstance
private val inboundContext: InboundContext = new InboundContext {
override def localAddress: UniqueAddress = uniqueLocalAddress
override def association(uid: Long): OptionVal[OutboundContext] = OptionVal.None
// the following methods are not used by in this test
override def sendControl(to: Address, message: ControlMessage): Unit = ???
override def association(remoteAddress: Address): OutboundContext = ???
override def completeHandshake(peer: UniqueAddress): Future[Done] = ???
override lazy val settings: ArterySettings =
ArterySettings(ConfigFactory.load().getConfig("pekko.remote.artery"))
override def publishDropped(inbound: InboundEnvelope, reason: String): Unit = ()
}
@nowarn("msg=never used") private var remoteRefB: RemoteActorRef = _
@nowarn("msg=never used") private var resolvedRef: InternalActorRef = _
@nowarn("msg=never used") private var senderStringA: String = _
@nowarn("msg=never used") private var recipientStringB: String = _
private var encodeGraph: Flow[String, Unit, NotUsed] = _
private var decodeGraph: Flow[String, Unit, NotUsed] = _
private var encodeDecodeGraph: Flow[String, Unit, NotUsed] = _
@Setup(Level.Trial)
def setupTrial(): Unit = {
val commonConfig = ConfigFactory.parseString(s"""
pekko {
loglevel = WARNING
actor.provider = remote
remote.artery.enabled = on
remote.artery.canonical.hostname = localhost
remote.artery.canonical.port = 0
actor.serializers.codec-benchmark = "${classOf[DummyMessageSerializer].getName}"
actor.serialization-identifiers { "${classOf[DummyMessageSerializer].getName}" = 4711 }
actor.serialization-bindings {"${classOf[DummyMessage].getName}" = codec-benchmark }
}
""")
val config = configType match {
case RemoteInstrument =>
ConfigFactory
.parseString(
s"""pekko.remote.artery.advanced.instruments = [ "${classOf[DummyRemoteInstrument].getName}" ]""")
.withFallback(commonConfig)
case _ =>
commonConfig
}
system = ActorSystem("CodecBenchmark", config)
systemB = ActorSystem("systemB", system.settings.config)
// eager init of materializer
SystemMaterializer(system).materializer
uniqueLocalAddress = UniqueAddress(
system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress,
AddressUidExtension(system).longAddressUid)
val actorOnSystemA = system.actorOf(Props.empty, "a")
senderStringA = actorOnSystemA.path.toSerializationFormatWithAddress(uniqueLocalAddress.address)
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 remoteInstruments: RemoteInstruments = if (configType == RemoteInstrument) {
new RemoteInstruments(system.asInstanceOf[ExtendedActorSystem], system.log, Vector(new DummyRemoteInstrument()))
} else null
val envelope = new EnvelopeBuffer(envelopeTemplateBuffer)
val outboundEnvelope = OutboundEnvelope(OptionVal.None, payload, OptionVal.None)
headerIn.setVersion(ArteryTransport.HighestVersion)
headerIn.setUid(42)
headerIn.setSenderActorRef(actorOnSystemA)
headerIn.setRecipientActorRef(remoteRefB)
headerIn.setManifest("")
headerIn.setRemoteInstruments(remoteInstruments)
MessageSerializer.serializeForArtery(SerializationExtension(system), outboundEnvelope, headerIn, envelope)
envelope.byteBuffer.flip()
// Now build up the graphs
val encoder: Flow[OutboundEnvelope, EnvelopeBuffer, Encoder.OutboundCompressionAccess] =
Flow.fromGraph(
new Encoder(
uniqueLocalAddress,
system.asInstanceOf[ExtendedActorSystem],
outboundEnvelopePool,
envelopePool,
streamId = 1,
debugLogSend = false,
version = ArteryTransport.HighestVersion))
val encoderInput: Flow[String, OutboundEnvelope, NotUsed] =
Flow[String].map(_ => outboundEnvelopePool.acquire().init(OptionVal.None, payload, OptionVal.Some(remoteRefB)))
val compressions = new InboundCompressionsImpl(system, inboundContext, inboundContext.settings.Advanced.Compression)
val decoder: Flow[EnvelopeBuffer, InboundEnvelope, InboundCompressionAccess] =
Flow.fromGraph(
new Decoder(
inboundContext,
system.asInstanceOf[ExtendedActorSystem],
uniqueLocalAddress,
inboundContext.settings,
compressions,
inboundEnvelopePool))
val deserializer: Flow[InboundEnvelope, InboundEnvelope, NotUsed] =
Flow.fromGraph(new Deserializer(inboundContext, system.asInstanceOf[ExtendedActorSystem], envelopePool))
val decoderInput: Flow[String, EnvelopeBuffer, NotUsed] = Flow[String].map { _ =>
val envelope = envelopePool.acquire()
envelopeTemplateBuffer.rewind()
envelope.byteBuffer.put(envelopeTemplateBuffer)
envelope.byteBuffer.flip()
envelope
}
encodeGraph = encoderInput.via(encoder).map(envelope => envelopePool.release(envelope))
decodeGraph = decoderInput.via(decoder).via(deserializer).map {
case env: ReusableInboundEnvelope => inboundEnvelopePool.release(env)
case _ =>
}
encodeDecodeGraph = encoderInput.via(encoder).via(decoder).via(deserializer).map {
case env: ReusableInboundEnvelope => inboundEnvelopePool.release(env)
case _ =>
}
}
@TearDown(Level.Trial)
def tearDownTrial(): Unit = {
Await.result(system.terminate(), 5.seconds)
Await.result(systemB.terminate(), 5.seconds)
}
@Setup(Level.Iteration)
def setupIteration(): Unit = {
System.gc()
}
@TearDown(Level.Iteration)
def tearDownIteration(): Unit = {}
@Benchmark
@OperationsPerInvocation(OperationsPerInvocation)
def reference(): Unit = {
val latch = new CountDownLatch(1)
val N = OperationsPerInvocation
Source.fromGraph(new BenchTestSourceSameElement(N, "elem")).runWith(new LatchSink(N, latch))
if (!latch.await(30, TimeUnit.SECONDS))
throw new RuntimeException("Latch didn't complete in time")
}
@Benchmark
@OperationsPerInvocation(OperationsPerInvocation)
def encode(): Unit = {
val latch = new CountDownLatch(1)
val N = OperationsPerInvocation
Source.fromGraph(new BenchTestSourceSameElement(N, "elem")).via(encodeGraph).runWith(new LatchSink(N, latch))
if (!latch.await(30, TimeUnit.SECONDS))
throw new RuntimeException("Latch didn't complete in time")
}
@Benchmark
@OperationsPerInvocation(OperationsPerInvocation)
def decode(): Unit = {
val latch = new CountDownLatch(1)
val N = OperationsPerInvocation
Source.fromGraph(new BenchTestSourceSameElement(N, "elem")).via(decodeGraph).runWith(new LatchSink(N, latch))
if (!latch.await(30, TimeUnit.SECONDS))
throw new RuntimeException("Latch didn't complete in time")
}
@Benchmark
@OperationsPerInvocation(OperationsPerInvocation)
def both(): Unit = {
val latch = new CountDownLatch(1)
val N = OperationsPerInvocation
Source.fromGraph(new BenchTestSourceSameElement(N, "elem")).via(encodeDecodeGraph).runWith(new LatchSink(N, latch))
if (!latch.await(30, TimeUnit.SECONDS))
throw new RuntimeException("Latch didn't complete in time")
}
}
object CodecBenchmark {
// Configurations
final val Standard = "Standard"
final val RemoteInstrument = "RemoteInstrument"
// How many iterations between materializations
final val OperationsPerInvocation = 1000000
// DummyMessage and serailizer that doesn't consume bytes during serialization/deserialization
val DummyMessageInstance = new DummyMessage
class DummyMessage
class DummyMessageSerializer(val system: ExtendedActorSystem) extends BaseSerializer with ByteBufferSerializer {
private val TheMagicConstant: Byte = 47
private val Preserialized = {
val buf = ByteBuffer.allocate(100)
buf.put(TheMagicConstant)
buf.flip()
buf.array()
}
override def includeManifest: Boolean = false
override def toBinary(o: AnyRef, buf: ByteBuffer): Unit = buf.put(TheMagicConstant)
override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = {
val b = buf.get()
if (b == TheMagicConstant)
DummyMessageInstance
else
throw new IOException(s"DummyMessage deserialization error. Expected $TheMagicConstant got $b")
}
override def toBinary(o: AnyRef): Array[Byte] = Preserialized
override def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef =
fromBinary(ByteBuffer.wrap(bytes), "NoManifestForYou")
}
// DummyRemoteInstrument that doesn't allocate unnecessary bytes during serialization/deserialization
class DummyRemoteInstrument extends RemoteInstrument {
private val Metadata = "slevin".getBytes
override def identifier: Byte = 7 // Lucky number slevin
override def remoteWriteMetadata(
recipient: ActorRef,
message: Object,
sender: ActorRef,
buffer: ByteBuffer): Unit = {
buffer.putInt(Metadata.length)
buffer.put(Metadata)
}
override def remoteReadMetadata(
recipient: ActorRef,
message: Object,
sender: ActorRef,
buffer: ByteBuffer): Unit = {
val length = Metadata.length
val metaLength = buffer.getInt
@tailrec
def compare(pos: Int): Boolean = {
if (pos == length) true
else if (Metadata(pos) == buffer.get()) compare(pos + 1)
else false
}
if (metaLength != length || !compare(0))
throw new IOException(s"DummyInstrument deserialization error. Expected ${Metadata.toString}")
}
override def remoteMessageSent(
recipient: ActorRef,
message: Object,
sender: ActorRef,
size: Int,
time: Long): Unit = ()
override def remoteMessageReceived(
recipient: ActorRef,
message: Object,
sender: ActorRef,
size: Int,
time: Long): Unit = ()
}
}

View file

@ -0,0 +1,70 @@
/*
* Copyright (C) 2016-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.remote.artery
import java.util.concurrent.CountDownLatch
import java.util.concurrent.CyclicBarrier
import org.apache.pekko
import pekko.stream.Attributes
import pekko.stream.Inlet
import pekko.stream.SinkShape
import pekko.stream.stage.GraphStage
import pekko.stream.stage.GraphStageLogic
import pekko.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 onUpstreamFailure(ex: Throwable): Unit = {
println(ex.getMessage)
ex.printStackTrace()
}
override def onPush(): Unit = {
n += 1
if (n == countDownAfter)
latch.countDown()
grab(in)
pull(in)
}
setHandler(in, this)
}
}
class BarrierSink(countDownAfter: Int, latch: CountDownLatch, barrierAfter: Int, barrier: CyclicBarrier)
extends GraphStage[SinkShape[Any]] {
val in: Inlet[Any] = Inlet("BarrierSink")
override val shape: SinkShape[Any] = SinkShape(in)
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
new GraphStageLogic(shape) with InHandler {
var n = 0
override def preStart(): Unit = pull(in)
override def onPush(): Unit = {
n += 1
grab(in)
if (n == countDownAfter)
latch.countDown()
else if (n % barrierAfter == 0)
barrier.await()
pull(in)
}
setHandler(in, this)
}
}

View file

@ -0,0 +1,104 @@
/*
* Copyright (C) 2016-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.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._
import org.apache.pekko.util.Unsafe
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(2)
@Warmup(iterations = 5)
@Measurement(iterations = 10)
class LiteralEncodingBenchmark {
private val UsAscii = Charset.forName("US-ASCII")
private val str = "akka://SomeSystem@host12:1234/user/foo"
private val buffer = ByteBuffer.allocate(128).order(ByteOrder.LITTLE_ENDIAN)
private val literalChars = Array.ofDim[Char](64)
private val literalBytes = Array.ofDim[Byte](64)
@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()
Unsafe.copyUSAsciiStrToBytes(str, literalBytes)
buffer.put(literalBytes, 0, length)
buffer.flip()
// read
buffer.get(literalBytes, 0, length)
var i = 0
while (i < length) {
// UsAscii
literalChars(i) = literalBytes(i).asInstanceOf[Char]
i += 1
}
String.valueOf(literalChars, 0, length)
}
}

View file

@ -0,0 +1,139 @@
/*
* Copyright (C) 2016-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.remote.artery
import java.util.concurrent.CountDownLatch
import java.util.concurrent.CyclicBarrier
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import org.agrona.concurrent.ManyToOneConcurrentArrayQueue
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.actor.ActorSystem
import pekko.stream.KillSwitches
import pekko.stream.OverflowStrategy
import pekko.stream.SystemMaterializer
import pekko.stream.scaladsl._
@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 = ActorSystem("SendQueueBenchmark", config)
@Setup
def setup(): Unit = {
// eager init of materializer
SystemMaterializer(system).materializer
}
@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)
val (queue, killSwitch) = source
.viaMat(KillSwitches.single)(Keep.both)
.toMat(new BarrierSink(N, latch, burstSize, barrier))(Keep.left)
.run()
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(PartialFunction.empty, PartialFunction.empty, 1024, OverflowStrategy.dropBuffer)
val (ref, killSwitch) = source
.viaMat(KillSwitches.single)(Keep.both)
.toMat(new BarrierSink(N, latch, burstSize, barrier))(Keep.left)
.run()
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()
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()
}
}

View file

@ -0,0 +1,51 @@
/*
* Copyright (C) 2016-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.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
}
}
}

View file

@ -0,0 +1,27 @@
/*
* Copyright (C) 2016-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.remote.artery.compress
import java.util.concurrent.ThreadLocalRandom
import org.openjdk.jmh.annotations._
@Fork(1)
@State(Scope.Benchmark)
class InvertCompressionTableBenchmark {
/*
TODO: Possibly specialise the inversion, it's not in hot path so not doing it for now
a.r.artery.compress.CompressionTableBenchmark.invert_comp_to_decomp_1024 N/A thrpt 20 5828.963 ± 281.631 ops/s
a.r.artery.compress.CompressionTableBenchmark.invert_comp_to_decomp_256 N/A thrpt 20 29040.889 ± 345.425 ops/s
*/
def randomName = ThreadLocalRandom.current().nextInt(1000).toString
val compTable_256 = CompressionTable(17L, 2, Map(Vector.fill[String](256)(randomName).zipWithIndex: _*))
val compTable_1024 = CompressionTable(17L, 3, Map(Vector.fill[String](1024)(randomName).zipWithIndex: _*))
@Benchmark def invert_comp_to_decomp_256 = compTable_256.invert
@Benchmark def invert_comp_to_decomp_1024 = compTable_1024.invert
}

View file

@ -0,0 +1,105 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.remote.artery.compress
import org.apache.pekko
import pekko.actor.Actor
import pekko.actor.ActorRef
import pekko.actor.ActorSystem
import pekko.actor.ExtendedActorSystem
import pekko.actor.Props
import pekko.pattern.PromiseActorRef
import pekko.remote.artery.SerializationFormatCache
import pekko.serialization.Serialization
import org.openjdk.jmh.annotations.Benchmark
import org.openjdk.jmh.annotations.Fork
import org.openjdk.jmh.annotations.Level
import org.openjdk.jmh.annotations.OperationsPerInvocation
import org.openjdk.jmh.annotations.Param
import org.openjdk.jmh.annotations.Scope
import org.openjdk.jmh.annotations.Setup
import org.openjdk.jmh.annotations.State
import org.openjdk.jmh.annotations.TearDown
import org.openjdk.jmh.infra.Blackhole
import scala.annotation.nowarn
import scala.concurrent.Promise
/**
* Actually more like specific benchmarks for the few concrete LRU cache usages
*/
@Fork(1)
@State(Scope.Benchmark)
@nowarn
class SerializationFormatCacheBenchmark {
// note 1 means only top level no temporary at all
@Param(Array("1", "2", "5", "10"))
private var everyNToToplevel = 0
// a few "normal" top level actors communicating
@Param(Array("100"))
private var uniqueTopLevelRefs = 0
// we want to simulate one per request-response, but create upfront, so very high number
@Param(Array("100000"))
private var uniqueTemporaryRefs = 0
private var system: ActorSystem = _
// hardocoded capacity of 1024
// note that this is not quite realistic, with a single cache,
// in practice there are N caches, one in each outgoing artery lane
private var cache: SerializationFormatCache = _
private var temporaryActorRefs: Array[ActorRef] = _
private var topLevelActorRefs: Array[ActorRef] = _
object Parent {
def props(childCount: Int, childProps: Props) = Props(new Parent(childCount, childProps))
}
class Parent(childCount: Int, childProps: Props) extends Actor {
val children = (0 to childCount).map(_ => context.actorOf(childProps))
def receive = PartialFunction.empty
}
@Setup
def init(): Unit = {
system = ActorSystem("SerializationFormatCacheBenchmark")
temporaryActorRefs = Array.tabulate(uniqueTemporaryRefs)(n =>
new PromiseActorRef(
system.asInstanceOf[ExtendedActorSystem].provider,
Promise(),
"Any",
// request path is encoded in this string
s"_user_region_shard${n % 100}_entitypretendid${n}"))
topLevelActorRefs = Array.tabulate(uniqueTopLevelRefs)(n => system.actorOf(Props.empty, s"actor_$n"))
}
// new empty cache instance each iteration to have more control over cached contents
@Setup(Level.Iteration)
def perRunSetup(): Unit = {
cache = new SerializationFormatCache
}
@TearDown
def shutdown(): Unit = {
system.terminate()
}
@Benchmark
@OperationsPerInvocation(2000)
def useCache(blackhole: Blackhole): Unit = {
// serialization requires this
Serialization.withTransportInformation(system.asInstanceOf[ExtendedActorSystem]) { () =>
var i: Int = 0
while (i < 2000) {
val actorRef =
if (i % everyNToToplevel == 0) topLevelActorRefs(i % uniqueTopLevelRefs)
else temporaryActorRefs(i % uniqueTemporaryRefs)
blackhole.consume(cache.getOrCompute(actorRef))
i += 1
}
}
}
}

View file

@ -0,0 +1,112 @@
/*
* Copyright (C) 2016-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.remote.compress
import java.util.Random
import org.openjdk.jmh.annotations._
import org.openjdk.jmh.infra.Blackhole
import org.apache.pekko.remote.artery.compress.TopHeavyHitters
/**
* On Macbook pro:
* [info] Benchmark (n) Mode Cnt Score Error Units
* [info] HeavyHittersBenchmark.updateHitter 8192 thrpt 40 357 405.512 ± 3329.008 ops/s
* [info] HeavyHittersBenchmark.updateNotHitter 8192 thrpt 40 259 032 711.743 ± 7199514.142 ops/s
* [info] HeavyHittersBenchmark.updateRandomHitter 8192 thrpt 40 2 105 102.088 ± 18214.624 ops/s
*
* ===
* on our benchmarking box:
* ubuntu@ip-172-31-43-199:~/akka-ktoso$ lscpu
* Architecture: x86_64
* CPU op-mode(s): 32-bit, 64-bit
* Byte Order: Little Endian
* CPU(s): 2
* Thread(s) per core: 2
* CPU MHz: 2494.068
* Hypervisor vendor: Xen
* Virtualization type: full
* L1d cache: 32K
* L1i cache: 32K
* L2 cache: 256K
* L3 cache: 25600K
*
* ubuntu@ip-172-31-43-199:~/akka-ktoso$ cpuid | grep nm
* (simple synth) = Intel Core i9-4000 / Xeon E5-1600/E5-2600 v2 (Ivy Bridge-EP C1/M1/S1), 22nm
*
* [info] Benchmark (n) Mode Cnt Score Error Units
* [info] HeavyHittersBenchmark.updateHitter 8192 thrpt 40 309 512.584 ± 153.248 ops/s
* [info] HeavyHittersBenchmark.updateNotHitter 8192 thrpt 40 248 170 545.577 ± 1244986.765 ops/s
* [info] HeavyHittersBenchmark.updateRandomHitter 8192 thrpt 40 1 207 521.674 ± 912.676 ops/s
*/
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(2)
class HeavyHittersBenchmark {
// @Param(Array("512", "8192"))
@Param(Array("8192"))
var n: Int = 0
private var topN: TopHeavyHitters[String] = _
val rand = new Random(1001021)
val preallocatedNums: Array[Long] = Array.ofDim(8192)
val preallocatedStrings: Array[String] = Array.ofDim(8192)
@Setup
def init(): Unit = {
topN = new TopHeavyHitters(n)
var i = 0
while (i < n) {
topN.update(i.toString, i)
preallocatedNums(i) = rand.nextLong()
preallocatedStrings(i) = i.toString
i += 1
}
}
@Benchmark
@OperationsPerInvocation(8192)
def updateNotHitter(blackhole: Blackhole): Unit = {
var i = 0
while (i < 8192) {
blackhole.consume(topN.update("NOT", 1)) // definitely not a heavy hitter
i += 1
}
}
@Benchmark
@OperationsPerInvocation(8192)
def updateExistingHitter(blackhole: Blackhole): Unit = {
var i: Int = 0
while (i < 8192) {
blackhole.consume(topN.update(preallocatedStrings(i % 16), Long.MaxValue)) // definitely a heavy hitter
i += 1
}
}
@Benchmark
def updateNewHitter(blackhole: Blackhole): Unit = {
var i = 0
while (i < 8192) {
blackhole.consume(topN.update(preallocatedStrings(i), Long.MaxValue))
i += 1
}
}
@Benchmark
@OperationsPerInvocation(8192)
def updateRandomHitter(blackhole: Blackhole): Unit = {
var i = 0
while (i < 8192) {
blackhole.consume(topN.update(preallocatedStrings(i), preallocatedNums(i))) // maybe a heavy hitter
i += 1
}
}
}

View file

@ -0,0 +1,288 @@
/*
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.serialization.jackson
import java.time.{ Duration => JDuration }
import java.time.Instant
import java.time.LocalDateTime
import java.util
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.annotation.nowarn
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.actor._
import pekko.serialization.Serialization
import pekko.serialization.SerializationExtension
import pekko.serialization.SerializerWithStringManifest
object JacksonSerializationBench {
trait TestMessage
final case class Small(name: String, num: Int) extends TestMessage
final case class Medium(
field1: String,
field2: String,
field3: String,
num1: Int,
num2: Int,
num3: Int,
flag1: Boolean,
flag2: Boolean,
duration: FiniteDuration,
date: LocalDateTime,
instant: Instant,
nested1: Small,
nested2: Small,
nested3: Small)
extends TestMessage
final case class Large(
nested1: Medium,
nested2: Medium,
nested3: Medium,
vector: Vector[Medium],
map: Map[String, Medium])
extends TestMessage
final class TimeMessage(val duration: FiniteDuration, val date: LocalDateTime, val instant: Instant)
extends TestMessage
}
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.SECONDS)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(2)
@Warmup(iterations = 4)
@Measurement(iterations = 5)
class JacksonSerializationBench {
import JacksonSerializationBench._
val smallMsg1 = Small("abc", 17)
val smallMsg2 = Small("def", 18)
val smallMsg3 = Small("ghi", 19)
val mediumMsg1 = Medium(
"abc",
"def",
"ghi",
1,
2,
3,
false,
true,
5.seconds,
LocalDateTime.of(2019, 4, 29, 23, 15, 3, 12345),
Instant.now(),
smallMsg1,
smallMsg2,
smallMsg3)
val mediumMsg2 = Medium(
"ABC",
"DEF",
"GHI",
10,
20,
30,
true,
false,
5.millis,
LocalDateTime.of(2019, 4, 29, 23, 15, 4, 12345),
Instant.now(),
smallMsg1,
smallMsg2,
smallMsg3)
val mediumMsg3 = Medium(
"abcABC",
"defDEF",
"ghiGHI",
100,
200,
300,
true,
true,
200.millis,
LocalDateTime.of(2019, 4, 29, 23, 15, 5, 12345),
Instant.now(),
smallMsg1,
smallMsg2,
smallMsg3)
val largeMsg = Large(
mediumMsg1,
mediumMsg2,
mediumMsg3,
Vector(mediumMsg1, mediumMsg2, mediumMsg3),
Map("a" -> mediumMsg1, "b" -> mediumMsg2, "c" -> mediumMsg3))
val timeMsg = new TimeMessage(5.seconds, LocalDateTime.of(2019, 4, 29, 23, 15, 3, 12345), Instant.now())
import JavaMessages._
val jSmallMsg1 = new JSmall("abc", 17)
val jSmallMsg2 = new JSmall("def", 18)
val jSmallMsg3 = new JSmall("ghi", 19)
val jMediumMsg1 = new JMedium(
"abc",
"def",
"ghi",
1,
2,
3,
false,
true,
JDuration.ofSeconds(5),
LocalDateTime.of(2019, 4, 29, 23, 15, 3, 12345),
Instant.now(),
jSmallMsg1,
jSmallMsg2,
jSmallMsg3)
val jMediumMsg2 = new JMedium(
"ABC",
"DEF",
"GHI",
10,
20,
30,
true,
false,
JDuration.ofMillis(5),
LocalDateTime.of(2019, 4, 29, 23, 15, 4, 12345),
Instant.now(),
jSmallMsg1,
jSmallMsg2,
jSmallMsg3)
val jMediumMsg3 = new JMedium(
"abcABC",
"defDEF",
"ghiGHI",
100,
200,
300,
true,
true,
JDuration.ofMillis(200),
LocalDateTime.of(2019, 4, 29, 23, 15, 5, 12345),
Instant.now(),
jSmallMsg1,
jSmallMsg2,
jSmallMsg3)
val jMap = new util.HashMap[String, JMedium]()
jMap.put("a", jMediumMsg1)
jMap.put("b", jMediumMsg2)
jMap.put("c", jMediumMsg3)
val jLargeMsg = new JLarge(
jMediumMsg1,
jMediumMsg2,
jMediumMsg3,
java.util.Arrays.asList(jMediumMsg1, jMediumMsg2, jMediumMsg3),
jMap)
var system: ActorSystem = _
var serialization: Serialization = _
@nowarn("msg=immutable val") // JMH updates this via reflection
@Param(Array("jackson-json", "jackson-cbor")) // "java"
private var serializerName: String = _
@nowarn("msg=immutable val")
@Param(Array("off", "gzip", "lz4"))
private var compression: String = _
@Setup(Level.Trial)
def setupTrial(): Unit = {
val config = ConfigFactory.parseString(s"""
pekko {
loglevel = WARNING
actor {
serialization-bindings {
"${classOf[TestMessage].getName}" = $serializerName
"${classOf[JTestMessage].getName}" = $serializerName
}
}
serialization.jackson {
serialization-features {
#WRITE_DATES_AS_TIMESTAMPS = off
}
}
}
pekko.serialization.jackson.jackson-json.compression {
algorithm = $compression
compress-larger-than = 100 b
}
""")
system = ActorSystem("JacksonSerializationBench", config)
serialization = SerializationExtension(system)
}
@TearDown(Level.Trial)
def tearDownTrial(): Unit = {
Await.result(system.terminate(), 5.seconds)
}
private var size = 0L
private def serializeDeserialize[T <: AnyRef](msg: T): T = {
serialization.findSerializerFor(msg) match {
case serializer: SerializerWithStringManifest =>
val blob = serializer.toBinary(msg)
if (size != blob.length) {
size = blob.length
println(
s"# Size is $size of ${msg.getClass.getName} with " +
s"${system.settings.config.getString("pekko.serialization.jackson.jackson-json.compression.algorithm")}")
}
serializer.fromBinary(blob, serializer.manifest(msg)).asInstanceOf[T]
case serializer =>
val blob = serializer.toBinary(msg)
if (serializer.includeManifest)
serializer.fromBinary(blob, Some(msg.getClass)).asInstanceOf[T]
else
serializer.fromBinary(blob, None).asInstanceOf[T]
}
}
@Benchmark
def small(): Small = {
serializeDeserialize(smallMsg1)
}
@Benchmark
def medium(): Medium = {
serializeDeserialize(mediumMsg1)
}
@Benchmark
def large(): Large = {
serializeDeserialize(largeMsg)
}
@Benchmark
def jSmall(): JSmall = {
serializeDeserialize(jSmallMsg1)
}
@Benchmark
def jMedium(): JMedium = {
serializeDeserialize(jMediumMsg1)
}
@Benchmark
def jLarge(): JLarge = {
serializeDeserialize(jLargeMsg)
}
@Benchmark
def timeMessage(): TimeMessage = {
serializeDeserialize(timeMsg)
}
}

View file

@ -0,0 +1,96 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.stream
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.NotUsed
import pekko.actor.Actor
import pekko.actor.ActorRef
import pekko.actor.ActorSystem
import pekko.actor.Props
import pekko.remote.artery.BenchTestSource
import pekko.remote.artery.LatchSink
import pekko.stream.scaladsl._
import pekko.stream.testkit.scaladsl.StreamTestKit
import pekko.util.Timeout
object AskBenchmark {
final val OperationsPerInvocation = 100000
}
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.SECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class AskBenchmark {
import AskBenchmark._
val config = ConfigFactory.parseString("""
pekko.actor.default-dispatcher {
executor = "fork-join-executor"
fork-join-executor {
parallelism-factor = 1
}
}
""")
implicit val system: ActorSystem = ActorSystem("MapAsyncBenchmark", config)
import system.dispatcher
var testSource: Source[java.lang.Integer, NotUsed] = _
var actor: ActorRef = _
implicit val timeout: Timeout = Timeout(10.seconds)
@Param(Array("1", "4"))
var parallelism = 0
@Param(Array("false", "true"))
var spawn = false
@Setup
def setup(): Unit = {
testSource = Source.fromGraph(new BenchTestSource(OperationsPerInvocation))
actor = system.actorOf(Props(new Actor {
override def receive = {
case element => sender() ! element
}
}))
// eager init of materializer
SystemMaterializer(system).materializer
}
@TearDown
def shutdown(): Unit = {
Await.result(system.terminate(), 5.seconds)
}
@Benchmark
@OperationsPerInvocation(OperationsPerInvocation)
def mapAsync(): Unit = {
val latch = new CountDownLatch(1)
testSource.ask[Int](parallelism)(actor).runWith(new LatchSink(OperationsPerInvocation, latch))
awaitLatch(latch)
}
private def awaitLatch(latch: CountDownLatch): Unit = {
if (!latch.await(30, TimeUnit.SECONDS)) {
StreamTestKit.printDebugDump(SystemMaterializer(system).materializer.supervisor)
throw new RuntimeException("Latch didn't complete in time")
}
}
}

View file

@ -0,0 +1,44 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.stream
import java.util.concurrent.TimeUnit
import scala.concurrent._
import scala.concurrent.duration._
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.actor.ActorSystem
import pekko.stream.scaladsl._
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class EmptySourceBenchmark {
implicit val system: ActorSystem = ActorSystem("EmptySourceBenchmark")
@TearDown
def shutdown(): Unit = {
Await.result(system.terminate(), 5.seconds)
}
val setup = Source.empty[String].toMat(Sink.ignore)(Keep.right)
@Benchmark def empty(): Unit =
Await.result(setup.run(), Duration.Inf)
/*
(not serious benchmark, just ballpark check: run on macbook 15, late 2013)
While it was a PublisherSource:
[info] EmptySourceBenchmark.empty thrpt 10 11.219 ± 6.498 ops/ms
Rewrite to GraphStage:
[info] EmptySourceBenchmark.empty thrpt 10 17.556 ± 2.865 ops/ms
*/
}

View file

@ -0,0 +1,110 @@
/*
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.stream
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.NotUsed
import pekko.actor.ActorSystem
import pekko.remote.artery.BenchTestSource
import pekko.remote.artery.LatchSink
import pekko.stream.impl.fusing.GraphStages
import pekko.stream.scaladsl._
import pekko.stream.testkit.scaladsl.StreamTestKit
object FlatMapConcatBenchmark {
final val OperationsPerInvocation = 100000
}
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.SECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class FlatMapConcatBenchmark {
import FlatMapConcatBenchmark._
private val config = ConfigFactory.parseString("""
pekko.actor.default-dispatcher {
executor = "fork-join-executor"
fork-join-executor {
parallelism-factor = 1
}
}
""")
private implicit val system: ActorSystem = ActorSystem("FlatMapConcatBenchmark", config)
var testSource: Source[java.lang.Integer, NotUsed] = _
@Setup
def setup(): Unit = {
// eager init of materializer
SystemMaterializer(system).materializer
testSource = Source.fromGraph(new BenchTestSource(OperationsPerInvocation))
}
@TearDown
def shutdown(): Unit = {
Await.result(system.terminate(), 5.seconds)
}
@Benchmark
@OperationsPerInvocation(OperationsPerInvocation)
def sourceDotSingle(): Unit = {
val latch = new CountDownLatch(1)
testSource.flatMapConcat(Source.single).runWith(new LatchSink(OperationsPerInvocation, latch))
awaitLatch(latch)
}
@Benchmark
@OperationsPerInvocation(OperationsPerInvocation)
def internalSingleSource(): Unit = {
val latch = new CountDownLatch(1)
testSource
.flatMapConcat(elem => new GraphStages.SingleSource(elem))
.runWith(new LatchSink(OperationsPerInvocation, latch))
awaitLatch(latch)
}
@Benchmark
@OperationsPerInvocation(OperationsPerInvocation)
def oneElementList(): Unit = {
val latch = new CountDownLatch(1)
testSource.flatMapConcat(n => Source(n :: Nil)).runWith(new LatchSink(OperationsPerInvocation, latch))
awaitLatch(latch)
}
@Benchmark
@OperationsPerInvocation(OperationsPerInvocation)
def mapBaseline(): Unit = {
val latch = new CountDownLatch(1)
testSource.map(elem => elem).runWith(new LatchSink(OperationsPerInvocation, latch))
awaitLatch(latch)
}
private def awaitLatch(latch: CountDownLatch): Unit = {
if (!latch.await(30, TimeUnit.SECONDS)) {
implicit val ec = system.dispatcher
StreamTestKit.printDebugDump(SystemMaterializer(system).materializer.supervisor)
throw new RuntimeException("Latch didn't complete in time")
}
}
}

View file

@ -0,0 +1,61 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.stream
import java.util.concurrent.TimeUnit
import scala.concurrent._
import scala.concurrent.duration._
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.{ Done, NotUsed }
import pekko.actor.ActorSystem
import pekko.remote.artery.BenchTestSourceSameElement
import pekko.stream.scaladsl._
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class FlatMapMergeBenchmark {
implicit val system: ActorSystem = ActorSystem("FlatMapMergeBenchmark")
val NumberOfElements = 100000
@Param(Array("0", "1", "10"))
var NumberOfStreams = 0
var graph: RunnableGraph[Future[Done]] = _
def createSource(count: Int): Graph[SourceShape[java.lang.Integer], NotUsed] =
new BenchTestSourceSameElement(count, 1)
@Setup
def setup(): Unit = {
val source = NumberOfStreams match {
// Base line: process NumberOfElements-many elements from a single source without using flatMapMerge
case 0 => createSource(NumberOfElements)
// Stream merging: process NumberOfElements-many elements from n sources, each producing (NumberOfElements/n)-many elements
case n =>
val subSource = createSource(NumberOfElements / n)
Source.repeat(()).take(n).flatMapMerge(n, _ => subSource)
}
graph = Source.fromGraph(source).toMat(Sink.ignore)(Keep.right)
// eager init of materializer
SystemMaterializer(system).materializer
}
@TearDown
def shutdown(): Unit = {
Await.result(system.terminate(), 5.seconds)
}
@Benchmark
@OperationsPerInvocation(100000) // Note: needs to match NumberOfElements.
def flat_map_merge_100k_elements(): Unit = {
Await.result(graph.run(), Duration.Inf)
}
}

View file

@ -0,0 +1,111 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.stream
import java.util.concurrent.Semaphore
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.util.Success
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.NotUsed
import pekko.actor.ActorSystem
import pekko.remote.artery.BenchTestSource
import pekko.stream.impl.fusing.GraphStages
import pekko.stream.scaladsl._
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.SECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class FlowMapBenchmark {
val config = ConfigFactory.parseString("""
pekko {
log-config-on-start = off
log-dead-letters-during-shutdown = off
loglevel = "WARNING"
actor.default-dispatcher {
#executor = "thread-pool-executor"
throughput = 1024
}
actor.default-mailbox {
mailbox-type = "org.apache.pekko.dispatch.SingleConsumerOnlyUnboundedMailbox"
}
test {
timefactor = 1.0
filter-leeway = 3s
single-expect-default = 3s
default-timeout = 5s
calling-thread-dispatcher {
type = org.apache.pekko.testkit.CallingThreadDispatcherConfigurator
}
}
}""".stripMargin).withFallback(ConfigFactory.load())
implicit val system: ActorSystem = ActorSystem("test", config)
@Param(Array("true", "false"))
var UseGraphStageIdentity = false
final val successMarker = Success(1)
final val successFailure = Success(new Exception)
// safe to be benchmark scoped because the flows we construct in this bench are stateless
var flow: Source[java.lang.Integer, NotUsed] = _
@Param(Array("8", "32", "128"))
var initialInputBufferSize = 0
@Param(Array("1", "5", "10"))
var numberOfMapOps = 0
@Setup
def setup(): Unit = {
flow = mkMaps(Source.fromGraph(new BenchTestSource(100000)), numberOfMapOps) {
if (UseGraphStageIdentity)
GraphStages.identity[java.lang.Integer]
else
Flow[java.lang.Integer].map(identity)
}
// eager init of materializer
SystemMaterializer(system).materializer
}
@TearDown
def shutdown(): Unit = {
Await.result(system.terminate(), 5.seconds)
}
@Benchmark
@OperationsPerInvocation(100000)
def flow_map_100k_elements(): Unit = {
val lock = new Semaphore(1) // todo rethink what is the most lightweight way to await for a streams completion
lock.acquire()
flow
.toMat(Sink.onComplete(_ => lock.release()))(Keep.right)
.withAttributes(Attributes.inputBuffer(initialInputBufferSize, initialInputBufferSize))
.run()
lock.acquire()
}
// source setup
private def mkMaps[O, Mat](source: Source[O, Mat], count: Int)(flow: => Graph[FlowShape[O, O], _]): Source[O, Mat] = {
var f = source
for (_ <- 1 to count)
f = f.via(flow)
f
}
}

View file

@ -0,0 +1,93 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.stream
import java.util.concurrent.Semaphore
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.util.Random
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.NotUsed
import pekko.actor.ActorSystem
import pekko.remote.artery.BenchTestSourceSameElement
import pekko.stream.scaladsl.Framing
import pekko.stream.scaladsl.Sink
import pekko.stream.scaladsl.Source
import pekko.util.ByteString
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.SECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class FramingBenchmark {
val config: Config = ConfigFactory.parseString("""
pekko {
log-config-on-start = off
log-dead-letters-during-shutdown = off
stdout-loglevel = "OFF"
loglevel = "OFF"
actor.default-dispatcher {
#executor = "thread-pool-executor"
throughput = 1024
}
actor.default-mailbox {
mailbox-type = "org.apache.pekko.dispatch.SingleConsumerOnlyUnboundedMailbox"
}
test {
timefactor = 1.0
filter-leeway = 3s
single-expect-default = 3s
default-timeout = 5s
calling-thread-dispatcher {
type = org.apache.pekko.testkit.CallingThreadDispatcherConfigurator
}
}
}""".stripMargin).withFallback(ConfigFactory.load())
implicit val system: ActorSystem = ActorSystem("test", config)
// Safe to be benchmark scoped because the flows we construct in this bench are stateless
var flow: Source[ByteString, NotUsed] = _
@Param(Array("32", "64", "128", "256", "512", "1024"))
var messageSize = 0
@Param(Array("1", "8", "16", "32", "64", "128"))
var framePerSeq = 0
@Setup
def setup(): Unit = {
SystemMaterializer(system).materializer
val frame = List.range(0, messageSize, 1).map(_ => Random.nextPrintableChar()).mkString + "\n"
val messageChunk = ByteString(List.range(0, framePerSeq, 1).map(_ => frame).mkString)
Source
.fromGraph(new BenchTestSourceSameElement(100000, messageChunk))
.via(Framing.delimiter(ByteString("\n"), Int.MaxValue))
}
@TearDown
def shutdown(): Unit = {
Await.result(system.terminate(), 5.seconds)
}
@Benchmark
@OperationsPerInvocation(100000)
def framing(): Unit = {
val lock = new Semaphore(1)
lock.acquire()
flow.runWith(Sink.onComplete(_ => lock.release()))
lock.acquire()
}
}

View file

@ -0,0 +1,300 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.stream
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
import org.openjdk.jmh.annotations.OperationsPerInvocation
import org.apache.pekko
import pekko.NotUsed
import pekko.actor.ActorSystem
import pekko.stream.scaladsl._
import pekko.stream.stage._
object FusedGraphsBenchmark {
val ElementCount = 100 * 1000
@volatile var blackhole: org.openjdk.jmh.infra.Blackhole = _
}
// Just to avoid allocations and still have a way to do some work in stages. The value itself does not matter
// so no issues with sharing (the result does not make any sense, but hey)
class MutableElement(var value: Int)
class TestSource(elems: Array[MutableElement]) extends GraphStage[SourceShape[MutableElement]] {
val out = Outlet[MutableElement]("TestSource.out")
override val shape = SourceShape(out)
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
new GraphStageLogic(shape) with OutHandler {
private[this] var left = FusedGraphsBenchmark.ElementCount - 1
override def onPull(): Unit = {
if (left >= 0) {
push(out, elems(left))
left -= 1
} else completeStage()
}
setHandler(out, this)
}
}
class JitSafeCompletionLatch extends GraphStageWithMaterializedValue[SinkShape[MutableElement], CountDownLatch] {
val in = Inlet[MutableElement]("JitSafeCompletionLatch.in")
override val shape = SinkShape(in)
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, CountDownLatch) = {
val latch = new CountDownLatch(1)
val logic = new GraphStageLogic(shape) with InHandler {
private[this] var sum = 0
override def preStart(): Unit = pull(in)
override def onPush(): Unit = {
sum += grab(in).value
pull(in)
}
override def onUpstreamFinish(): Unit = {
// Do not ignore work along the chain
FusedGraphsBenchmark.blackhole.consume(sum)
latch.countDown()
completeStage()
}
setHandler(in, this)
}
(logic, latch)
}
}
class IdentityStage extends GraphStage[FlowShape[MutableElement, MutableElement]] {
val in = Inlet[MutableElement]("Identity.in")
val out = Outlet[MutableElement]("Identity.out")
override val shape = FlowShape(in, out)
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
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)
}
}
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class FusedGraphsBenchmark {
import FusedGraphsBenchmark._
implicit val system: ActorSystem = ActorSystem(
"test",
ConfigFactory.parseString(s"""
pekko.stream.materializer.sync-processing-limit = ${Int.MaxValue}
"""))
var testElements: Array[MutableElement] = _
var singleIdentity: RunnableGraph[CountDownLatch] = _
var chainOfIdentities: RunnableGraph[CountDownLatch] = _
var singleMap: RunnableGraph[CountDownLatch] = _
var chainOfMaps: RunnableGraph[CountDownLatch] = _
var repeatTakeMapAndFold: RunnableGraph[CountDownLatch] = _
var singleBuffer: RunnableGraph[CountDownLatch] = _
var chainOfBuffers: RunnableGraph[CountDownLatch] = _
var broadcastZip: RunnableGraph[CountDownLatch] = _
var balanceMerge: RunnableGraph[CountDownLatch] = _
var broadcastZipBalanceMerge: RunnableGraph[CountDownLatch] = _
@Setup
def setup(): Unit = {
// eager init of materializer
SystemMaterializer(system).materializer
testElements = Array.fill(ElementCount)(new MutableElement(0))
val addFunc = (x: MutableElement) => { x.value += 1; x }
val testSource = Source.fromGraph(new TestSource(testElements))
val testSink = Sink.fromGraph(new JitSafeCompletionLatch)
def fuse(r: RunnableGraph[CountDownLatch]): RunnableGraph[CountDownLatch] = {
RunnableGraph.fromGraph(r)
}
val identityStage = new IdentityStage
singleIdentity = fuse(testSource.via(identityStage).toMat(testSink)(Keep.right))
chainOfIdentities = fuse(
testSource
.via(identityStage)
.via(identityStage)
.via(identityStage)
.via(identityStage)
.via(identityStage)
.via(identityStage)
.via(identityStage)
.via(identityStage)
.via(identityStage)
.via(identityStage)
.toMat(testSink)(Keep.right))
singleMap = fuse(testSource.map(addFunc).toMat(testSink)(Keep.right))
chainOfMaps = fuse(
testSource
.map(addFunc)
.map(addFunc)
.map(addFunc)
.map(addFunc)
.map(addFunc)
.map(addFunc)
.map(addFunc)
.map(addFunc)
.map(addFunc)
.map(addFunc)
.toMat(testSink)(Keep.right))
repeatTakeMapAndFold = fuse(
Source
.repeat(new MutableElement(0))
.take(ElementCount)
.map(addFunc)
.map(addFunc)
.fold(new MutableElement(0))((acc, x) => { acc.value += x.value; acc })
.toMat(testSink)(Keep.right))
singleBuffer = fuse(testSource.buffer(10, OverflowStrategy.backpressure).toMat(testSink)(Keep.right))
chainOfBuffers = fuse(
testSource
.buffer(10, OverflowStrategy.backpressure)
.buffer(10, OverflowStrategy.backpressure)
.buffer(10, OverflowStrategy.backpressure)
.buffer(10, OverflowStrategy.backpressure)
.buffer(10, OverflowStrategy.backpressure)
.buffer(10, OverflowStrategy.backpressure)
.buffer(10, OverflowStrategy.backpressure)
.buffer(10, OverflowStrategy.backpressure)
.buffer(10, OverflowStrategy.backpressure)
.buffer(10, OverflowStrategy.backpressure)
.toMat(testSink)(Keep.right))
val broadcastZipFlow: Flow[MutableElement, MutableElement, NotUsed] = Flow.fromGraph(GraphDSL.create() {
implicit b =>
import GraphDSL.Implicits._
val bcast = b.add(Broadcast[MutableElement](2))
val zip = b.add(Zip[MutableElement, MutableElement]())
bcast ~> zip.in0
bcast ~> zip.in1
FlowShape(bcast.in, zip.out.map(_._1).outlet)
})
val balanceMergeFlow: Flow[MutableElement, MutableElement, NotUsed] = Flow.fromGraph(GraphDSL.create() {
implicit b =>
import GraphDSL.Implicits._
val balance = b.add(Balance[MutableElement](2))
val merge = b.add(Merge[MutableElement](2))
balance ~> merge
balance ~> merge
FlowShape(balance.in, merge.out)
})
broadcastZip = fuse(testSource.via(broadcastZipFlow).toMat(testSink)(Keep.right))
balanceMerge = fuse(testSource.via(balanceMergeFlow).toMat(testSink)(Keep.right))
broadcastZipBalanceMerge = fuse(testSource.via(broadcastZipFlow).via(balanceMergeFlow).toMat(testSink)(Keep.right))
}
@Benchmark
@OperationsPerInvocation(100 * 1000)
def single_identity(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
singleIdentity.run().await()
}
@Benchmark
@OperationsPerInvocation(100 * 1000)
def chain_of_identities(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
chainOfIdentities.run().await()
}
@Benchmark
@OperationsPerInvocation(100 * 1000)
def single_map(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
singleMap.run().await()
}
@Benchmark
@OperationsPerInvocation(100 * 1000)
def chain_of_maps(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
chainOfMaps.run().await()
}
@Benchmark
@OperationsPerInvocation(100 * 1000)
def repeat_take_map_and_fold(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
repeatTakeMapAndFold.run().await()
}
@Benchmark
@OperationsPerInvocation(100 * 1000)
def single_buffer(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
singleBuffer.run().await()
}
@Benchmark
@OperationsPerInvocation(100 * 1000)
def chain_of_buffers(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
chainOfBuffers.run().await()
}
@Benchmark
@OperationsPerInvocation(100 * 1000)
def broadcast_zip(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
broadcastZip.run().await()
}
@Benchmark
@OperationsPerInvocation(100 * 1000)
def balance_merge(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
balanceMerge.run().await()
}
@Benchmark
@OperationsPerInvocation(100 * 1000)
def broadcast_zip_balance_merge(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
FusedGraphsBenchmark.blackhole = blackhole
broadcastZipBalanceMerge.run().await()
}
@TearDown
def shutdown(): Unit = {
Await.result(system.terminate(), 5.seconds)
}
}

View file

@ -0,0 +1,39 @@
/*
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.stream
import java.util.concurrent.TimeUnit
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.NotUsed
import pekko.stream.scaladsl.RunnableGraph
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.SECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class GraphBuilderBenchmark {
@Param(Array("1", "10", "100", "1000"))
var complexity = 0
@Benchmark
def flow_with_map(): RunnableGraph[NotUsed] =
MaterializationBenchmark.flowWithMapBuilder(complexity)
@Benchmark
def graph_with_junctions_gradual(): RunnableGraph[NotUsed] =
MaterializationBenchmark.graphWithJunctionsGradualBuilder(complexity)
@Benchmark
def graph_with_junctions_immediate(): RunnableGraph[NotUsed] =
MaterializationBenchmark.graphWithJunctionsImmediateBuilder(complexity)
@Benchmark
def graph_with_imported_flow(): RunnableGraph[NotUsed] =
MaterializationBenchmark.graphWithImportedFlowBuilder(complexity)
}

View file

@ -0,0 +1,99 @@
/*
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.stream
import java.util.concurrent.TimeUnit
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.event._
import pekko.stream.impl.fusing.GraphInterpreterSpecKit
import pekko.stream.impl.fusing.GraphInterpreter.{ DownstreamBoundaryStageLogic, UpstreamBoundaryStageLogic }
import pekko.stream.impl.fusing.GraphStages
import pekko.stream.stage._
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class InterpreterBenchmark {
import InterpreterBenchmark._
// manual, and not via @Param, because we want @OperationsPerInvocation on our tests
final val data100k: Vector[Int] = (1 to 100000).toVector
@Param(Array("1", "5", "10"))
var numberOfIds: Int = 0
@Benchmark
@OperationsPerInvocation(100000)
def graph_interpreter_100k_elements(): Unit = {
new GraphInterpreterSpecKit {
new TestSetup {
val identities = Vector.fill(numberOfIds)(GraphStages.identity[Int])
val source = new GraphDataSource("source", data100k)
val sink = new GraphDataSink[Int]("sink", data100k.size)
val b = builder(identities: _*).connect(source, identities.head.in).connect(identities.last.out, sink)
// FIXME: This should not be here, this is pure setup overhead
for (i <- 0 until identities.size - 1) {
b.connect(identities(i).out, identities(i + 1).in)
}
b.init()
sink.requestOne()
interpreter.execute(Int.MaxValue)
}
}
}
}
object InterpreterBenchmark {
case class GraphDataSource[T](override val toString: String, data: Vector[T]) extends UpstreamBoundaryStageLogic[T] {
var idx: Int = 0
override val out: pekko.stream.Outlet[T] = Outlet[T]("out")
out.id = 0
setHandler(
out,
new OutHandler {
override def onPull(): Unit = {
if (idx < data.size) {
push(out, data(idx))
idx += 1
} else {
completeStage()
}
}
override def onDownstreamFinish(cause: Throwable): Unit = cancelStage(cause)
})
}
case class GraphDataSink[T](override val toString: String, var expected: Int)
extends DownstreamBoundaryStageLogic[T] {
override val in: pekko.stream.Inlet[T] = Inlet[T]("in")
in.id = 0
setHandler(in,
new InHandler {
override def onPush(): Unit = {
expected -= 1
if (expected > 0) pull(in)
// Otherwise do nothing, it will exit the interpreter
}
})
def requestOne(): Unit = pull(in)
}
val NoopBus = new LoggingBus {
override def subscribe(subscriber: Subscriber, to: Classifier): Boolean = true
override def publish(event: Event): Unit = ()
override def unsubscribe(subscriber: Subscriber, from: Classifier): Boolean = true
override def unsubscribe(subscriber: Subscriber): Unit = ()
}
}

View file

@ -0,0 +1,60 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.stream
import java.util.concurrent.TimeUnit
import scala.concurrent._
import scala.concurrent.duration._
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.actor.ActorSystem
import pekko.stream.scaladsl._
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.SECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class InvokeWithFeedbackBenchmark {
implicit val system: ActorSystem = ActorSystem("InvokeWithFeedbackBenchmark")
var sourceQueue: SourceQueueWithComplete[Int] = _
var sinkQueue: SinkQueueWithCancel[Int] = _
val waitForResult = 100.millis
@Setup
def setup(): Unit = {
// these are currently the only two built in stages using invokeWithFeedback
val (in, out) =
Source
.queue[Int](bufferSize = 1, overflowStrategy = OverflowStrategy.backpressure)
.toMat(Sink.queue[Int]())(Keep.both)
.run()
sourceQueue = in
sinkQueue = out
}
@OperationsPerInvocation(100000)
@Benchmark
def pass_through_100k_elements(): Unit = {
(0 to 100000).foreach { n =>
val f = sinkQueue.pull()
Await.result(sourceQueue.offer(n), waitForResult)
Await.result(f, waitForResult)
}
}
@TearDown
def tearDown(): Unit = {
sourceQueue.complete()
// no way to observe sink completion from the outside
Await.result(system.terminate(), 5.seconds)
}
}

View file

@ -0,0 +1,59 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.stream
import java.util.concurrent.TimeUnit
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.stream.impl.JsonObjectParser
import pekko.util.ByteString
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.SECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class JsonFramingBenchmark {
val json =
ByteString("""{"fname":"Frank","name":"Smith","age":42,"id":1337,"boardMember":false}""")
val json5 =
ByteString("""|{"fname":"Frank","name":"Smith","age":42,"id":1337,"boardMember":false},
|{"fname":"Bob","name":"Smith","age":42,"id":1337,"boardMember":false},
|{"fname":"Bob","name":"Smith","age":42,"id":1337,"boardMember":false},
|{"fname":"Bob","name":"Smith","age":42,"id":1337,"boardMember":false},
|{"fname":"Hank","name":"Smith","age":42,"id":1337,"boardMember":false}""".stripMargin)
val jsonLong =
ByteString(
s"""{"fname":"Frank","name":"Smith","age":42,"id":1337,"boardMember":false,"description":"${"a" * 1000000}"}""")
val bracket = new JsonObjectParser
@Benchmark
def counting_1: ByteString = {
bracket.offer(json)
bracket.poll().get
}
@Benchmark
@OperationsPerInvocation(5)
def counting_offer_5: ByteString = {
bracket.offer(json5)
bracket.poll().get
bracket.poll().get
bracket.poll().get
bracket.poll().get
bracket.poll().get
}
@Benchmark
def counting_long_document: ByteString = {
bracket.offer(jsonLong)
bracket.poll().get
}
}

View file

@ -0,0 +1,98 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.stream
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.Future
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.NotUsed
import pekko.actor.ActorSystem
import pekko.remote.artery.BenchTestSource
import pekko.remote.artery.LatchSink
import pekko.stream.scaladsl._
import pekko.stream.testkit.scaladsl.StreamTestKit
object MapAsyncBenchmark {
final val OperationsPerInvocation = 100000
}
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.SECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class MapAsyncBenchmark {
import MapAsyncBenchmark._
val config = ConfigFactory.parseString("""
pekko.actor.default-dispatcher {
executor = "fork-join-executor"
fork-join-executor {
parallelism-factor = 1
}
}
""")
implicit val system: ActorSystem = ActorSystem("MapAsyncBenchmark", config)
import system.dispatcher
var testSource: Source[java.lang.Integer, NotUsed] = _
@Param(Array("1", "4"))
var parallelism = 0
@Param(Array("false", "true"))
var spawn = false
@Setup
def setup(): Unit = {
// eager init of materializer
SystemMaterializer(system).materializer
testSource = Source.fromGraph(new BenchTestSource(OperationsPerInvocation))
}
@TearDown
def shutdown(): Unit = {
Await.result(system.terminate(), 5.seconds)
}
@Benchmark
@OperationsPerInvocation(OperationsPerInvocation)
def mapAsync(): Unit = {
val latch = new CountDownLatch(1)
testSource
.mapAsync(parallelism)(elem => if (spawn) Future(elem) else Future.successful(elem))
.runWith(new LatchSink(OperationsPerInvocation, latch))
awaitLatch(latch)
}
@Benchmark
@OperationsPerInvocation(OperationsPerInvocation)
def mapAsyncUnordered(): Unit = {
val latch = new CountDownLatch(1)
testSource
.mapAsyncUnordered(parallelism)(elem => if (spawn) Future(elem) else Future.successful(elem))
.runWith(new LatchSink(OperationsPerInvocation, latch))
awaitLatch(latch)
}
private def awaitLatch(latch: CountDownLatch): Unit = {
if (!latch.await(30, TimeUnit.SECONDS)) {
StreamTestKit.printDebugDump(SystemMaterializer(system).materializer.supervisor)
throw new RuntimeException("Latch didn't complete in time")
}
}
}

View file

@ -0,0 +1,144 @@
/*
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.stream
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.Future
import scala.concurrent.duration._
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.Done
import pekko.NotUsed
import pekko.actor.ActorSystem
import pekko.stream.scaladsl._
object MaterializationBenchmark {
val flowWithMapBuilder = (numOfOperators: Int) => {
var source = Source.single(())
for (_ <- 1 to numOfOperators) {
source = source.map(identity)
}
source.to(Sink.ignore)
}
val graphWithJunctionsGradualBuilder = (numOfJunctions: Int) =>
RunnableGraph.fromGraph(GraphDSL.create() { implicit b =>
import GraphDSL.Implicits._
val broadcast = b.add(Broadcast[Unit](numOfJunctions))
var outlet = broadcast.out(0)
for (i <- 1 until numOfJunctions) {
val merge = b.add(Merge[Unit](2))
outlet ~> merge
broadcast.out(i) ~> merge
outlet = merge.out
}
Source.single(()) ~> broadcast
outlet ~> Sink.ignore
ClosedShape
})
val graphWithJunctionsImmediateBuilder = (numOfJunctions: Int) =>
RunnableGraph.fromGraph(GraphDSL.create() { implicit b =>
import GraphDSL.Implicits._
val broadcast = b.add(Broadcast[Unit](numOfJunctions))
val merge = b.add(Merge[Unit](numOfJunctions))
for (_ <- 0 until numOfJunctions) {
broadcast ~> merge
}
Source.single(()) ~> broadcast
merge ~> Sink.ignore
ClosedShape
})
val graphWithImportedFlowBuilder = (numOfFlows: Int) =>
RunnableGraph.fromGraph(GraphDSL.createGraph(Source.single(())) { implicit b => source =>
import GraphDSL.Implicits._
val flow = Flow[Unit].map(identity)
var out: Outlet[Unit] = source.out
for (_ <- 0 until numOfFlows) {
val flowShape = b.add(flow)
out ~> flowShape
out = flowShape.outlet
}
out ~> Sink.ignore
ClosedShape
})
final val subStreamCount = 10000
val subStreamBuilder: Int => RunnableGraph[Future[Unit]] = numOfOperators => {
val subFlow = {
var flow = Flow[Unit]
for (_ <- 1 to numOfOperators) {
flow = flow.map(identity)
}
flow
}
Source.repeat(Source.single(())).take(subStreamCount).flatMapConcat(_.via(subFlow)).toMat(Sink.last)(Keep.right)
}
}
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.SECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class MaterializationBenchmark {
import MaterializationBenchmark._
implicit val system: ActorSystem = ActorSystem("MaterializationBenchmark")
var flowWithMap: RunnableGraph[NotUsed] = _
var graphWithJunctionsGradual: RunnableGraph[NotUsed] = _
var graphWithJunctionsImmediate: RunnableGraph[NotUsed] = _
var graphWithImportedFlow: RunnableGraph[NotUsed] = _
var subStream: RunnableGraph[Future[Unit]] = _
@Param(Array("1", "10", "100"))
var complexity = 0
@Setup
def setup(): Unit = {
flowWithMap = flowWithMapBuilder(complexity)
graphWithJunctionsGradual = graphWithJunctionsGradualBuilder(complexity)
graphWithJunctionsImmediate = graphWithJunctionsImmediateBuilder(complexity)
graphWithImportedFlow = graphWithImportedFlowBuilder(complexity)
subStream = subStreamBuilder(complexity)
}
@TearDown
def shutdown(): Unit = {
Await.result(system.terminate(), 5.seconds)
}
@Benchmark
def flow_with_map(): NotUsed = flowWithMap.run()
@Benchmark
def graph_with_junctions_gradual(): NotUsed = graphWithJunctionsGradual.run()
@Benchmark
def graph_with_junctions_immediate(): NotUsed = graphWithJunctionsImmediate.run()
@Benchmark
def graph_with_imported_flow(): NotUsed = graphWithImportedFlow.run()
@Benchmark
@OperationsPerInvocation(subStreamCount)
def sub_stream(): Done = {
Await.result(subStream.run(), 5.seconds)
Done
}
}

View file

@ -0,0 +1,361 @@
/*
* Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.stream
import java.util
import java.util.concurrent.TimeUnit
import org.apache.pekko.stream.impl.MaterializerSession
import org.apache.pekko.stream.impl.NewLayout._
import org.apache.pekko.stream.impl.StreamLayout.{ AtomicModule, Module }
import org.openjdk.jmh.annotations._
import org.reactivestreams.{ Publisher, Subscriber, Subscription }
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class NewLayoutBenchmark {
// TODO: This benchmark is heavily copy-pasta. This is a temporary benchmark as these two implementations
// will never exist at the same time. This needs to be turned into a better one once the design
// settles.
// --- These test classes do not use the optimized linear builder, for testing the composite builder instead
class CompositeTestSource extends AtomicModule {
val out = Outlet[Any]("testSourceC.out")
override val shape: Shape = SourceShape(out)
override val attributes: Attributes = Attributes.name("testSource")
val traversalBuilder = TraversalBuilder.atomic(this)
override def withAttributes(attributes: Attributes): Module = ???
override def carbonCopy: Module = ???
override def replaceShape(s: Shape): Module = ???
override def toString = "TestSource"
}
class CompositeTestSink extends AtomicModule {
val in = Inlet[Any]("testSinkC.in")
override val shape: Shape = SinkShape(in)
override val attributes: Attributes = Attributes.name("testSink")
val traversalBuilder = TraversalBuilder.atomic(this)
override def withAttributes(attributes: Attributes): Module = ???
override def carbonCopy: Module = ???
override def replaceShape(s: Shape): Module = ???
override def toString = "TestSink"
}
class CompositeTestFlow(tag: String) extends AtomicModule {
val in = Inlet[Any](s"testFlowC$tag.in")
val out = Outlet[Any](s"testFlowC$tag.out")
override val shape: Shape = FlowShape(in, out)
override val attributes: Attributes = Attributes.name(s"testFlow$tag")
val traversalBuilder = TraversalBuilder.atomic(this)
override def withAttributes(attributes: Attributes): Module = ???
override def carbonCopy: Module = ???
override def replaceShape(s: Shape): Module = ???
override def toString = s"TestFlow$tag"
}
// --- These test classes DO use the optimized linear builder, for testing the composite builder instead
class LinearTestSource extends AtomicModule {
val out = Outlet[Any]("testSource.out")
override val shape: Shape = SourceShape(out)
override val attributes: Attributes = Attributes.name("testSource")
val traversalBuilder = TraversalBuilder.linear(this)
override def withAttributes(attributes: Attributes): Module = ???
override def carbonCopy: Module = ???
override def replaceShape(s: Shape): Module = ???
override def toString = "TestSource"
}
class LinearTestSink extends AtomicModule {
val in = Inlet[Any]("testSink.in")
override val shape: Shape = SinkShape(in)
override val attributes: Attributes = Attributes.name("testSink")
val traversalBuilder = TraversalBuilder.linear(this)
override def withAttributes(attributes: Attributes): Module = ???
override def carbonCopy: Module = ???
override def replaceShape(s: Shape): Module = ???
override def toString = "TestSink"
}
class LinearTestFlow(tag: String) extends AtomicModule {
val in = Inlet[Any](s"testFlow$tag.in")
val out = Outlet[Any](s"testFlow$tag.out")
override val shape: Shape = FlowShape(in, out)
override val attributes: Attributes = Attributes.name(s"testFlow$tag")
val traversalBuilder = TraversalBuilder.linear(this)
override def withAttributes(attributes: Attributes): Module = ???
override def carbonCopy: Module = ???
override def replaceShape(s: Shape): Module = ???
override def toString = s"TestFlow$tag"
}
class MaterializationResult(
val connections: Int,
val inlets: Array[InPort],
val outlets: Array[OutPort]
) {
override def toString = {
outlets.iterator.zip(inlets.iterator).mkString("connections: ", ", ", "")
}
}
class OldSource extends AtomicModule {
val out = Outlet[Any]("out")
override val shape = SourceShape(out)
override def replaceShape(s: Shape): Module = ???
override def carbonCopy: Module = ???
override def attributes: Attributes = Attributes.none
override def withAttributes(attributes: Attributes): Module = this
}
class OldSink extends AtomicModule {
val in = Inlet[Any]("in")
override val shape = SinkShape(in)
override def replaceShape(s: Shape): Module = ???
override def carbonCopy: Module = ???
override def attributes: Attributes = Attributes.none
override def withAttributes(attributes: Attributes): Module = this
}
class OldFlow extends AtomicModule {
val in = Inlet[Any]("in")
val out = Outlet[Any]("out")
override val shape = FlowShape(in, out)
override def replaceShape(s: Shape): Module = ???
override def carbonCopy: Module = ???
override def attributes: Attributes = Attributes.none
override def withAttributes(attributes: Attributes): Module = this
}
val linearSource = new LinearTestSource
val linearSink = new LinearTestSink
val linearFlow = new LinearTestFlow("linearFlow")
val compositeSource = new CompositeTestSource
val compositeSink = new CompositeTestSink
val compositeFlow = new CompositeTestFlow("linearFlow")
val oldSource = new OldSource
val oldSink = new OldSink
val oldFlow = new OldFlow
def testMaterializeNew(b: TraversalBuilder): MaterializationResult = {
require(b.isTraversalComplete, "Traversal builder must be complete")
val connections = b.inSlots
val inlets = Array.ofDim[InPort](connections)
val outlets = Array.ofDim[OutPort](connections)
// Track next assignable number for input ports
var inOffs = 0
var current: Traversal = b.traversal.get
val traversalStack = new util.ArrayList[Traversal](16)
traversalStack.add(current)
// Due to how Concat works, we need a stack. This probably can be optimized for the most common cases.
while (!traversalStack.isEmpty) {
current = traversalStack.remove(traversalStack.size() - 1)
while (current ne EmptyTraversal) {
current match {
case MaterializeAtomic(mod, outToSlot) =>
var i = 0
val inletsIter = mod.shape.inlets.iterator
while (inletsIter.hasNext) {
val in = inletsIter.next()
inlets(inOffs + i) = in
i += 1
}
val outletsIter = mod.shape.outlets.iterator
while (outletsIter.hasNext) {
val out = outletsIter.next()
outlets(inOffs + outToSlot(out.id)) = out
}
inOffs += mod.shape.inlets.size
current = current.next
// And that's it ;)
case Concat(first, next) =>
traversalStack.add(next)
current = first
case _ =>
current = current.next
}
}
}
new MaterializationResult(connections, inlets, outlets)
}
case class TestPublisher(owner: Module, port: OutPort) extends Publisher[Any] with Subscription {
var downstreamModule: Module = _
var downstreamPort: InPort = _
override def subscribe(s: Subscriber[_ >: Any]): Unit = s match {
case TestSubscriber(o, p) =>
downstreamModule = o
downstreamPort = p
s.onSubscribe(this)
}
override def request(n: Long): Unit = ()
override def cancel(): Unit = ()
}
case class TestSubscriber(owner: Module, port: InPort) extends Subscriber[Any] {
var upstreamModule: Module = _
var upstreamPort: OutPort = _
override def onSubscribe(s: Subscription): Unit = s match {
case TestPublisher(o, p) =>
upstreamModule = o
upstreamPort = p
}
override def onError(t: Throwable): Unit = ()
override def onComplete(): Unit = ()
override def onNext(t: Any): Unit = ()
}
class FlatTestMaterializer(_module: Module) extends MaterializerSession(_module, Attributes()) {
private var i = 0
var publishers = Array.ofDim[TestPublisher](1024)
var subscribers = Array.ofDim[TestSubscriber](1024)
override protected def materializeAtomic(atomic: AtomicModule, effectiveAttributes: Attributes,
matVal: java.util.Map[Module, Any]): Unit = {
for (inPort <- atomic.inPorts) {
val subscriber = TestSubscriber(atomic, inPort)
subscribers(i) = subscriber
i += 1
assignPort(inPort, subscriber)
}
for (outPort <- atomic.outPorts) {
val publisher = TestPublisher(atomic, outPort)
publishers(i) = publisher
i += 1
assignPort(outPort, publisher)
}
}
}
def testMaterializeOld(m: Module, blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
val mat = new FlatTestMaterializer(m)
mat.materialize()
blackhole.consume(mat.publishers)
}
@Benchmark
def source_and_sink_new_linear(): TraversalBuilder = {
linearSource.traversalBuilder.append(linearSink.traversalBuilder, linearSink.shape)
}
@Benchmark
def source_and_sink_new_composite(): TraversalBuilder = {
compositeSource.traversalBuilder
.add(compositeSink.traversalBuilder, compositeSink.shape)
.wire(compositeSource.out, compositeSink.in)
}
@Benchmark
def source_and_sink_old(): Module = {
oldSource
.compose(oldSink)
.wire(oldSource.out, oldSink.in)
}
@Benchmark
def source_flow_and_sink_new_linear(): TraversalBuilder = {
linearSource.traversalBuilder
.append(linearFlow.traversalBuilder, linearFlow.shape)
.append(linearSink.traversalBuilder, linearSink.shape)
}
@Benchmark
def source_flow_and_sink_new_composite(): TraversalBuilder = {
compositeSource.traversalBuilder
.add(compositeFlow.traversalBuilder, compositeFlow.shape)
.wire(compositeSource.out, compositeFlow.in)
.add(compositeSink.traversalBuilder, compositeSink.shape)
.wire(compositeFlow.out, compositeSink.in)
}
@Benchmark
def source_flow_and_sink_old(): Module = {
oldSource
.compose(oldFlow)
.wire(oldSource.out, oldFlow.in)
.compose(oldSink)
.wire(oldFlow.out, oldSink.in)
}
val sourceSinkLinear = linearSource.traversalBuilder.append(linearSink.traversalBuilder, linearSink.shape)
val sourceSinkComposite = compositeSource.traversalBuilder
.add(compositeSink.traversalBuilder, compositeSink.shape)
.wire(compositeSource.out, compositeSink.in)
val sourceSinkOld = oldSource
.compose(oldSink)
.wire(oldSource.out, oldSink.in)
val sourceFlowSinkLinear = linearSource.traversalBuilder
.append(linearFlow.traversalBuilder, linearFlow.shape)
.append(linearSink.traversalBuilder, linearSink.shape)
val sourceFlowSinkComposite = compositeSource.traversalBuilder
.add(compositeFlow.traversalBuilder, compositeFlow.shape)
.wire(compositeSource.out, compositeFlow.in)
.add(compositeSink.traversalBuilder, compositeSink.shape)
.wire(compositeFlow.out, compositeSink.in)
val sourceFlowSinkOld = oldSource
.compose(oldFlow)
.wire(oldSource.out, oldFlow.in)
.compose(oldSink)
.wire(oldFlow.out, oldSink.in)
@Benchmark
def mat_source_and_sink_new_linear(): MaterializationResult = {
testMaterializeNew(sourceSinkLinear)
}
@Benchmark
def mat_source_and_sink_new_composite(): MaterializationResult = {
testMaterializeNew(sourceSinkComposite)
}
@Benchmark
def mat_source_and_sink_old(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
testMaterializeOld(sourceSinkOld, blackhole: org.openjdk.jmh.infra.Blackhole)
}
@Benchmark
def mat_source_flow_and_sink_new_linear(): MaterializationResult = {
testMaterializeNew(sourceFlowSinkLinear)
}
@Benchmark
def mat_source_flow_and_sink_new_composite(): MaterializationResult = {
testMaterializeNew(sourceFlowSinkComposite)
}
@Benchmark
def mat_source_flow_and_sink_old(blackhole: org.openjdk.jmh.infra.Blackhole): Unit = {
testMaterializeOld(sourceFlowSinkOld, blackhole: org.openjdk.jmh.infra.Blackhole)
}
}

View file

@ -0,0 +1,112 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.stream
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.NotUsed
import pekko.actor.ActorSystem
import pekko.remote.artery.BenchTestSource
import pekko.remote.artery.FixedSizePartitionHub
import pekko.remote.artery.LatchSink
import pekko.stream.scaladsl._
import pekko.stream.scaladsl.PartitionHub
import pekko.stream.testkit.scaladsl.StreamTestKit
object PartitionHubBenchmark {
final val OperationsPerInvocation = 100000
}
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.SECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class PartitionHubBenchmark {
import PartitionHubBenchmark._
val config = ConfigFactory.parseString("""
pekko.actor.default-dispatcher {
executor = "fork-join-executor"
fork-join-executor {
parallelism-factor = 1
}
}
""")
implicit val system: ActorSystem = ActorSystem("PartitionHubBenchmark", config)
@Param(Array("2", "5", "10", "20", "30"))
var NumberOfStreams = 0
@Param(Array("256"))
var BufferSize = 0
var testSource: Source[java.lang.Integer, NotUsed] = _
@Setup
def setup(): Unit = {
// eager init of materializer
SystemMaterializer(system).materializer
testSource = Source.fromGraph(new BenchTestSource(OperationsPerInvocation))
}
@TearDown
def shutdown(): Unit = {
Await.result(system.terminate(), 5.seconds)
}
@Benchmark
@OperationsPerInvocation(OperationsPerInvocation)
def partition(): Unit = {
val N = OperationsPerInvocation
val latch = new CountDownLatch(NumberOfStreams)
val source = testSource.runWith(
PartitionHub.sink[java.lang.Integer](
(_, elem) => elem.intValue % NumberOfStreams,
startAfterNrOfConsumers = NumberOfStreams,
bufferSize = BufferSize))
for (_ <- 0 until NumberOfStreams)
source.runWith(new LatchSink(N / NumberOfStreams, latch))
if (!latch.await(30, TimeUnit.SECONDS)) {
dumpMaterializer()
throw new RuntimeException("Latch didn't complete in time")
}
}
// @Benchmark
// @OperationsPerInvocation(OperationsPerInvocation)
def arteryLanes(): Unit = {
val N = OperationsPerInvocation
val latch = new CountDownLatch(NumberOfStreams)
val source = testSource.runWith(
Sink.fromGraph(
new FixedSizePartitionHub(_.intValue % NumberOfStreams, lanes = NumberOfStreams, bufferSize = BufferSize)))
for (_ <- 0 until NumberOfStreams)
source.runWith(new LatchSink(N / NumberOfStreams, latch))
if (!latch.await(30, TimeUnit.SECONDS)) {
dumpMaterializer()
throw new RuntimeException("Latch didn't complete in time")
}
}
private def dumpMaterializer(): Unit = {
implicit val ec = system.dispatcher
StreamTestKit.printDebugDump(SystemMaterializer(system).materializer.supervisor)
}
}

View file

@ -0,0 +1,70 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.stream
import java.util.concurrent.Semaphore
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.util.Success
import com.typesafe.config.ConfigFactory
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.actor.ActorSystem
import pekko.remote.artery.BenchTestSource
import pekko.stream.scaladsl._
/*
Just a brief reference run (3.1 GHz Intel Core i7, MacBook Pro late 2017):
[info] SourceRefBenchmark.source_ref_100k_elements thrpt 10 724650.336 ± 233643.256 ops/s
*/
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.SECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class SourceRefBenchmark {
val config = ConfigFactory.parseString("""
pekko {
log-config-on-start = off
log-dead-letters-during-shutdown = off
loglevel = "WARNING"
}""".stripMargin).withFallback(ConfigFactory.load())
implicit val system: ActorSystem = ActorSystem("test", config)
final val successMarker = Success(1)
final val successFailure = Success(new Exception)
// safe to be benchmark scoped because the flows we construct in this bench are stateless
var sourceRef: SourceRef[java.lang.Integer] = _
// @Param(Array("16", "32", "128"))
// var initialInputBufferSize = 0
@Setup(Level.Invocation)
def setup(): Unit = {
sourceRef = Source.fromGraph(new BenchTestSource(100000)).runWith(StreamRefs.sourceRef())
}
@TearDown
def shutdown(): Unit = {
Await.result(system.terminate(), 5.seconds)
}
@Benchmark
@OperationsPerInvocation(100000)
def source_ref_100k_elements(): Unit = {
val lock = new Semaphore(1) // todo rethink what is the most lightweight way to await for a streams completion
lock.acquire()
sourceRef.source.runWith(Sink.onComplete(_ => lock.release()))
lock.acquire()
}
}

View file

@ -0,0 +1,55 @@
/*
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.stream.impl
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.duration._
import org.openjdk.jmh.annotations._
import org.openjdk.jmh.annotations.TearDown
import org.apache.pekko
import pekko.actor.ActorSystem
import pekko.stream.scaladsl.Keep
import pekko.stream.scaladsl.Sink
import pekko.stream.scaladsl.StreamConverters
object OutputStreamSourceStageBenchmark {
final val WritesPerBench = 10000
}
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
@BenchmarkMode(Array(Mode.Throughput))
class OutputStreamSourceStageBenchmark {
import OutputStreamSourceStageBenchmark.WritesPerBench
implicit val system: ActorSystem = ActorSystem("OutputStreamSourceStageBenchmark")
private val bytes: Array[Byte] = Array.emptyByteArray
@Benchmark
@OperationsPerInvocation(WritesPerBench)
def consumeWrites(): Unit = {
val (os, done) = StreamConverters.asOutputStream().toMat(Sink.ignore)(Keep.both).run()
new Thread(new Runnable {
def run(): Unit = {
var counter = 0
while (counter > WritesPerBench) {
os.write(bytes)
counter += 1
}
os.close()
}
}).start()
Await.result(done, 30.seconds)
}
@TearDown
def shutdown(): Unit = {
Await.result(system.terminate(), 5.seconds)
}
}

View file

@ -0,0 +1,111 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.stream.io
import java.nio.file.Files
import java.nio.file.Path
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.Future
import scala.concurrent.Promise
import scala.concurrent.duration._
import org.openjdk.jmh.annotations._
import org.apache.pekko
import pekko.Done
import pekko.NotUsed
import pekko.actor.ActorSystem
import pekko.stream.Attributes
import pekko.stream.IOResult
import pekko.stream.scaladsl._
import pekko.util.ByteString
/**
* Benchmark (bufSize) Mode Cnt Score Error Units
* FileSourcesBenchmark.fileChannel 2048 avgt 100 1140.192 ± 55.184 ms/op
*/
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
@BenchmarkMode(Array(Mode.AverageTime))
class FileSourcesBenchmark {
implicit val system: ActorSystem = ActorSystem("file-sources-benchmark")
val file: Path = {
val line = ByteString("x" * 2048 + "\n")
val f = Files.createTempFile(getClass.getName, ".bench.tmp")
val ft = Source
.fromIterator(() => Iterator.continually(line))
.take(10 * 39062) // adjust as needed
.runWith(FileIO.toPath(f))
Await.result(ft, 30.seconds)
f
}
@Param(Array("2048"))
var bufSize = 0
var fileChannelSource: Source[ByteString, Future[IOResult]] = _
var fileInputStreamSource: Source[ByteString, Future[IOResult]] = _
var ioSourceLinesIterator: Source[ByteString, NotUsed] = _
@Setup
def setup(): Unit = {
fileChannelSource = FileIO.fromPath(file, bufSize)
fileInputStreamSource = StreamConverters.fromInputStream(() => Files.newInputStream(file), bufSize)
ioSourceLinesIterator =
Source.fromIterator(() => scala.io.Source.fromFile(file.toFile).getLines()).map(ByteString(_))
}
@TearDown
def teardown(): Unit = {
Files.delete(file)
}
@TearDown
def shutdown(): Unit = {
Await.result(system.terminate(), Duration.Inf)
}
@Benchmark
def fileChannel(): Unit = {
val h = fileChannelSource.to(Sink.ignore).run()
Await.result(h, 30.seconds)
}
@Benchmark
def fileChannel_noReadAhead(): Unit = {
val h = fileChannelSource.withAttributes(Attributes.inputBuffer(1, 1)).to(Sink.ignore).run()
Await.result(h, 30.seconds)
}
@Benchmark
def inputStream(): Unit = {
val h = fileInputStreamSource.to(Sink.ignore).run()
Await.result(h, 30.seconds)
}
/*
* The previous status quo was very slow:
* Benchmark Mode Cnt Score Error Units
* FileSourcesBenchmark.naive_ioSourceLinesIterator avgt 20 7067.944 ± 1341.847 ms/op
*/
@Benchmark
def naive_ioSourceLinesIterator(): Unit = {
val p = Promise[Done]()
ioSourceLinesIterator.to(Sink.onComplete(p.complete(_))).run()
Await.result(p.future, 30.seconds)
}
}

View file

@ -0,0 +1,96 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.stream.io
import java.nio.file.Files
import java.nio.file.Path
import java.util.concurrent.TimeUnit
import scala.concurrent.Await
import scala.concurrent.Future
import scala.concurrent.duration._
import org.openjdk.jmh.annotations._
import org.openjdk.jmh.annotations.BenchmarkMode
import org.openjdk.jmh.annotations.Scope
import org.openjdk.jmh.annotations.State
import org.apache.pekko
import pekko.actor.ActorSystem
import pekko.stream.IOResult
import pekko.stream.scaladsl._
import pekko.util.ByteString
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.AverageTime))
@Fork(1)
@Threads(1)
@Warmup(iterations = 5, timeUnit = TimeUnit.SECONDS, batchSize = 1)
@Measurement(iterations = 10, timeUnit = TimeUnit.SECONDS, batchSize = 1)
class FileSourcesScaleBenchmark {
/**
* Benchmark (bufSize) Mode Cnt Score Error Units
* FileSourcesScaleBenchmark.flatMapMerge 2048 avgt 10 1.587 ± 0.118 s/op
* FileSourcesScaleBenchmark.mapAsync 2048 avgt 10 0.899 ± 0.103 s/op
*/
implicit val system: ActorSystem = ActorSystem("file-sources-benchmark")
val FILES_NUMBER = 40
val files: Seq[Path] = {
val line = ByteString("x" * 2048 + "\n")
(1 to FILES_NUMBER).map(i => {
val f = Files.createTempFile(getClass.getName, s"$i.bench.tmp")
val ft = Source
.fromIterator(() => Iterator.continually(line))
.take(20000) // adjust as needed
.runWith(FileIO.toPath(f))
Await.result(ft, 300.seconds)
f
})
}
@Param(Array("2048"))
var bufSize = 0
var fileChannelSource: Seq[Source[ByteString, Future[IOResult]]] = _
@Setup
def setup(): Unit = {
fileChannelSource = files.map(FileIO.fromPath(_, bufSize))
}
@TearDown
def teardown(): Unit = {
files.foreach(Files.delete)
}
@TearDown
def shutdown(): Unit = {
Await.result(system.terminate(), Duration.Inf)
}
@Benchmark
def flatMapMerge(): Unit = {
val h = Source
.fromIterator(() => files.iterator)
.flatMapMerge(FILES_NUMBER, path => FileIO.fromPath(path, bufSize))
.runWith(Sink.ignore)
Await.result(h, 300.seconds)
}
@Benchmark
def mapAsync(): Unit = {
val h = Source
.fromIterator(() => files.iterator)
.mapAsync(FILES_NUMBER)(path => FileIO.fromPath(path, bufSize).runWith(Sink.ignore))
.runWith(Sink.ignore)
Await.result(h, 300.seconds)
}
}

View file

@ -0,0 +1,105 @@
/*
* Copyright (C) 2021-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.util
import java.util.concurrent.TimeUnit
import org.openjdk.jmh.annotations._
import org.openjdk.jmh.infra.Blackhole
@State(Scope.Benchmark)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(2)
@Warmup(iterations = 4)
@Measurement(iterations = 10)
class ByteString_append_Benchmark {
private val bs = ByteString(Array.ofDim[Byte](10))
@Benchmark
@OperationsPerInvocation(10000)
def appendThree(bh: Blackhole): Unit = {
var result = ByteString.empty
var i = 0
while (i < 10000) {
result = result ++ bs
if (i % 3 == 0) {
bh.consume(result)
result = ByteString.empty
}
i += 1
}
bh.consume(result)
}
@Benchmark
@OperationsPerInvocation(10000)
def appendMany(bh: Blackhole): Unit = {
var result = ByteString.empty
var i = 0
while (i < 10000) {
result = result ++ bs
i += 1
}
bh.consume(result)
}
@Benchmark
@OperationsPerInvocation(10000)
def builderOne(bh: Blackhole): Unit = {
val builder = ByteString.newBuilder
var i = 0
while (i < 10000) {
builder ++= bs
bh.consume(builder.result())
builder.clear()
i += 1
}
}
@Benchmark
@OperationsPerInvocation(10000)
def builderThree(bh: Blackhole): Unit = {
val builder = ByteString.newBuilder
var i = 0
while (i < 10000) {
builder ++= bs
if (i % 3 == 0) {
bh.consume(builder.result())
builder.clear()
}
i += 1
}
bh.consume(builder.result())
}
@Benchmark
@OperationsPerInvocation(10000)
def builderFive(bh: Blackhole): Unit = {
val builder = ByteString.newBuilder
var i = 0
while (i < 10000) {
builder ++= bs
if (i % 5 == 0) {
bh.consume(builder.result())
builder.clear()
}
i += 1
}
bh.consume(builder.result())
}
@Benchmark
@OperationsPerInvocation(10000)
def builderMany(bh: Blackhole): Unit = {
val builder = ByteString.newBuilder
var i = 0
while (i < 10000) {
builder ++= bs
i += 1
}
bh.consume(builder.result())
}
}

View file

@ -0,0 +1,44 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.util
import java.util.concurrent.TimeUnit
import org.openjdk.jmh.annotations._
import org.apache.pekko.util.ByteString.{ ByteString1, ByteStrings }
@State(Scope.Benchmark)
@Measurement(timeUnit = TimeUnit.MILLISECONDS)
class ByteString_apply_Benchmark {
val bss = ByteStrings(Vector.fill(1024)(ByteString1(Array(0.toByte))))
/*
akka-bench-jmh/jmh:run -f 1 -wi 3 -i 3 .*ByteString_apply_Benchmark.*
2.12 original
ByteString_apply_Benchmark.bss_apply_best_case thrpt 3 204261596.303 ± 94507102.894 ops/s
ByteString_apply_Benchmark.bss_apply_worst_case thrpt 3 170359.149 ± 102901.206 ops/s
2.12 optimized
ByteString_apply_Benchmark.bss_apply_best_case thrpt 3 206985005.270 ± 7855543.098 ops/s
ByteString_apply_Benchmark.bss_apply_worst_case thrpt 3 437929.845 ± 27264.190 ops/s
2.13 original
ByteString_apply_Benchmark.bss_apply_best_case thrpt 3 206854021.793 ± 81500220.451 ops/s
ByteString_apply_Benchmark.bss_apply_worst_case thrpt 3 237125.194 ± 128394.832 ops/s
2.13 optimized
ByteString_apply_Benchmark.bss_apply_best_case thrpt 3 209266780.913 ± 6821134.296 ops/s
ByteString_apply_Benchmark.bss_apply_worst_case thrpt 3 430348.094 ± 24412.915 ops/s
*/
@Benchmark
def bss_apply_best_case: Byte = bss(0)
@Benchmark
def bss_apply_worst_case: Byte = bss(1023)
}

View file

@ -0,0 +1,93 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.util
import java.nio.ByteBuffer
import java.util.concurrent.TimeUnit
import org.openjdk.jmh.annotations._
import org.apache.pekko.util.ByteString.{ ByteString1C, ByteStrings }
@State(Scope.Benchmark)
@Measurement(timeUnit = TimeUnit.MILLISECONDS)
class ByteString_copyToBuffer_Benchmark {
val _bs_mini = ByteString(Array.ofDim[Byte](128 * 4))
val _bs_small = ByteString(Array.ofDim[Byte](1024 * 1))
val _bs_large = ByteString(Array.ofDim[Byte](1024 * 4))
val bs_mini = ByteString(Array.ofDim[Byte](128 * 4 * 4))
val bs_small = ByteString(Array.ofDim[Byte](1024 * 1 * 4))
val bs_large = ByteString(Array.ofDim[Byte](1024 * 4 * 4))
val bss_mini = ByteStrings(Vector.fill(4)(bs_mini.asInstanceOf[ByteString1C].toByteString1), 4 * bs_mini.length)
val bss_small = ByteStrings(Vector.fill(4)(bs_small.asInstanceOf[ByteString1C].toByteString1), 4 * bs_small.length)
val bss_large = ByteStrings(Vector.fill(4)(bs_large.asInstanceOf[ByteString1C].toByteString1), 4 * bs_large.length)
val bss_pc_large = bss_large.compact
val buf = ByteBuffer.allocate(1024 * 4 * 4)
/*
BEFORE
[info] Benchmark Mode Cnt Score Error Units
[info] ByteStringBenchmark.bs_large_copyToBuffer thrpt 40 142 163 289.866 ± 21751578.294 ops/s
[info] ByteStringBenchmark.bss_large_copyToBuffer thrpt 40 1 489 195.631 ± 209165.487 ops/s << that's the interesting case, we needlessly fold and allocate tons of Stream etc
[info] ByteStringBenchmark.bss_large_pc_copyToBuffer thrpt 40 184 466 756.364 ± 9169108.378 ops/s // "can't beat that"
[info] ....[Thread state: RUNNABLE]........................................................................
[info] 35.9% 35.9% scala.collection.Iterator$class.toStream
[info] 20.2% 20.2% scala.collection.immutable.Stream.foldLeft
[info] 11.6% 11.6% scala.collection.immutable.Stream$StreamBuilder.<init>
[info] 10.9% 10.9% org.apache.pekko.util.ByteIterator.<init>
[info] 6.1% 6.1% scala.collection.mutable.ListBuffer.<init>
[info] 5.2% 5.2% org.apache.pekko.util.ByteString.copyToBuffer
[info] 5.2% 5.2% scala.collection.AbstractTraversable.<init>
[info] 2.2% 2.2% scala.collection.immutable.VectorIterator.initFrom
[info] 1.2% 1.2% org.apache.pekko.util.generated.ByteStringBenchmark_bss_large_copyToBuffer.bss_large_copyToBuffer_thrpt_jmhStub
[info] 0.3% 0.3% org.apache.pekko.util.ByteIterator$MultiByteArrayIterator.copyToBuffer
[info] 1.2% 1.2% <other>
AFTER specializing impls
[info] ....[Thread state: RUNNABLE]........................................................................
[info] 99.5% 99.6% org.apache.pekko.util.generated.ByteStringBenchmark_bss_large_copyToBuffer_jmhTest.bss_large_copyToBuffer_thrpt_jmhStub
[info] 0.1% 0.1% java.util.concurrent.CountDownLatch.countDown
[info] 0.1% 0.1% sun.reflect.NativeMethodAccessorImpl.invoke0
[info] 0.1% 0.1% sun.misc.Unsafe.putObject
[info] 0.1% 0.1% org.openjdk.jmh.infra.IterationParamsL2.getBatchSize
[info] 0.1% 0.1% java.lang.Thread.currentThread
[info] 0.1% 0.1% sun.misc.Unsafe.compareAndSwapInt
[info] 0.1% 0.1% sun.reflect.AccessorGenerator.internalize
[info] Benchmark Mode Cnt Score Error Units
[info] ByteStringBenchmark.bs_large_copyToBuffer thrpt 40 177 328 585.473 ± 7742067.648 ops/s
[info] ByteStringBenchmark.bss_large_copyToBuffer thrpt 40 113 535 003.488 ± 3899763.124 ops/s // previous bad case now very good (was 2M/s)
[info] ByteStringBenchmark.bss_large_pc_copyToBuffer thrpt 40 203 590 896.493 ± 7582752.024 ops/s // "can't beat that"
*/
@Benchmark
def bs_large_copyToBuffer(): Int = {
buf.flip()
bs_large.copyToBuffer(buf)
}
@Benchmark
def bss_large_copyToBuffer(): Int = {
buf.flip()
bss_large.copyToBuffer(buf)
}
/** Pre-compacted */
@Benchmark
def bss_large_pc_copyToBuffer(): Int = {
buf.flip()
bss_pc_large.copyToBuffer(buf)
}
}

View file

@ -0,0 +1,66 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.util
import java.nio.charset.Charset
import java.util.concurrent.TimeUnit
import org.openjdk.jmh.annotations._
import org.apache.pekko.util.ByteString.{ ByteString1C, ByteStrings }
@State(Scope.Benchmark)
@Measurement(timeUnit = TimeUnit.MILLISECONDS)
class ByteString_decode_Benchmark {
val _bs_large = ByteString(Array.ofDim[Byte](1024 * 4))
val bs_large = ByteString(Array.ofDim[Byte](1024 * 4 * 4))
val bss_large = ByteStrings(Vector.fill(4)(bs_large.asInstanceOf[ByteString1C].toByteString1), 4 * bs_large.length)
val bc_large = bss_large.compact // compacted
val utf8String = "utf-8"
val utf8 = Charset.forName(utf8String)
/*
Using Charset helps a bit, but nothing impressive:
[info] ByteString_decode_Benchmark.bc_large_decodeString_stringCharset_utf8 thrpt 20 21 612.293 ± 825.099 ops/s
=>
[info] ByteString_decode_Benchmark.bc_large_decodeString_charsetCharset_utf8 thrpt 20 22 473.372 ± 851.597 ops/s
[info] ByteString_decode_Benchmark.bs_large_decodeString_stringCharset_utf8 thrpt 20 84 443.674 ± 3723.987 ops/s
=>
[info] ByteString_decode_Benchmark.bs_large_decodeString_charsetCharset_utf8 thrpt 20 93 865.033 ± 2052.476 ops/s
[info] ByteString_decode_Benchmark.bss_large_decodeString_stringCharset_utf8 thrpt 20 14 886.553 ± 326.752 ops/s
=>
[info] ByteString_decode_Benchmark.bss_large_decodeString_charsetCharset_utf8 thrpt 20 16 031.670 ± 474.565 ops/s
*/
@Benchmark
def bc_large_decodeString_stringCharset_utf8: String =
bc_large.decodeString(utf8String)
@Benchmark
def bs_large_decodeString_stringCharset_utf8: String =
bs_large.decodeString(utf8String)
@Benchmark
def bss_large_decodeString_stringCharset_utf8: String =
bss_large.decodeString(utf8String)
@Benchmark
def bc_large_decodeString_charsetCharset_utf8: String =
bc_large.decodeString(utf8)
@Benchmark
def bs_large_decodeString_charsetCharset_utf8: String =
bs_large.decodeString(utf8)
@Benchmark
def bss_large_decodeString_charsetCharset_utf8: String =
bss_large.decodeString(utf8)
}

View file

@ -0,0 +1,72 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.util
import java.util.concurrent.TimeUnit
import scala.util.Random
import org.openjdk.jmh.annotations._
import org.apache.pekko.util.ByteString.{ ByteString1, ByteStrings }
@State(Scope.Benchmark)
@Measurement(timeUnit = TimeUnit.MILLISECONDS)
class ByteString_dropRight_Benchmark {
val str = List.fill[Byte](4)(0).mkString
val numVec = 1024
val bss = ByteStrings(Vector.fill(numVec)(ByteString1.fromString(str)))
val rand = new Random()
val len = str.size * numVec
val n_greater_or_eq_to_len = len + rand.nextInt(Int.MaxValue - len)
val n_neg = rand.nextInt(Int.MaxValue) * -1
val n_avg = len / 2
val n_best = 1
val n_worst = len - 1
/*
--------------------------------- BASELINE -----------------------------------------------------------------------
commit 0f2da7b26b5c4af35be87d2bd4a1a2392365df15
[info] Benchmark Mode Cnt Score Error Units
[info] ByteString_dropRight_Benchmark.bss_avg thrpt 40 25626.311 ± 1395.662 ops/s
[info] ByteString_dropRight_Benchmark.bss_best thrpt 40 8667558.031 ± 200233.008 ops/s
[info] ByteString_dropRight_Benchmark.bss_greater_or_eq_to_len thrpt 40 12658.684 ± 376.730 ops/s
[info] ByteString_dropRight_Benchmark.bss_negative thrpt 40 1214680926.895 ± 10661843.507 ops/s
[info] ByteString_dropRight_Benchmark.bss_worst thrpt 40 13087.245 ± 246.911 ops/s
--------------------------------- AFTER --------------------------------------------------------------------------
------ TODAY
[info] Benchmark Mode Cnt Score Error Units
[info] ByteString_dropRight_Benchmark.bss_avg thrpt 40 528969.025 ± 6039.001 ops/s
[info] ByteString_dropRight_Benchmark.bss_best thrpt 40 7925951.396 ± 249279.950 ops/s
[info] ByteString_dropRight_Benchmark.bss_greater_or_eq_to_len thrpt 40 893475724.604 ± 9836471.105 ops/s
[info] ByteString_dropRight_Benchmark.bss_negative thrpt 40 1182275022.613 ± 9710755.955 ops/s
[info] ByteString_dropRight_Benchmark.bss_worst thrpt 40 244599.957 ± 3276.140 ops/s
*/
@Benchmark
def bss_negative(): ByteString =
bss.dropRight(n_neg)
@Benchmark
def bss_greater_or_eq_to_len(): ByteString =
bss.dropRight(n_greater_or_eq_to_len)
@Benchmark
def bss_avg(): ByteString =
bss.dropRight(n_avg)
@Benchmark
def bss_best(): ByteString =
bss.dropRight(n_best)
@Benchmark
def bss_worst(): ByteString =
bss.dropRight(n_worst)
}

View file

@ -0,0 +1,157 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.util
import java.util.concurrent.TimeUnit
import org.openjdk.jmh.annotations._
import org.apache.pekko.util.ByteString.{ ByteString1C, ByteStrings }
@State(Scope.Benchmark)
@Measurement(timeUnit = TimeUnit.MILLISECONDS)
class ByteString_dropSliceTake_Benchmark {
val _bs_mini = ByteString(Array.ofDim[Byte](128 * 4))
val _bs_small = ByteString(Array.ofDim[Byte](1024 * 1))
val _bs_large = ByteString(Array.ofDim[Byte](1024 * 4))
val bs_mini = ByteString(Array.ofDim[Byte](128 * 4 * 4))
val bs_small = ByteString(Array.ofDim[Byte](1024 * 1 * 4))
val bs_large = ByteString(Array.ofDim[Byte](1024 * 4 * 4))
val bss_mini = ByteStrings(Vector.fill(4)(bs_mini.asInstanceOf[ByteString1C].toByteString1), 4 * bs_mini.length)
val bss_small = ByteStrings(Vector.fill(4)(bs_small.asInstanceOf[ByteString1C].toByteString1), 4 * bs_small.length)
val bss_large = ByteStrings(Vector.fill(4)(bs_large.asInstanceOf[ByteString1C].toByteString1), 4 * bs_large.length)
val bss_pc_large = bss_large.compact
/*
--------------------------------- BASELINE --------------------------------------------------------------------
[info] Benchmark Mode Cnt Score Error Units
[info] ByteString_dropSliceTake_Benchmark.bs_large_dropRight_100 thrpt 20 111 122 621.983 ± 6172679.160 ops/s
[info] ByteString_dropSliceTake_Benchmark.bs_large_dropRight_256 thrpt 20 110 238 003.870 ± 4042572.908 ops/s
[info] ByteString_dropSliceTake_Benchmark.bs_large_dropRight_2000 thrpt 20 106 435 449.123 ± 2972282.531 ops/s
[info] ByteString_dropSliceTake_Benchmark.bss_large_dropRight_100 thrpt 20 1 155 292.430 ± 23096.219 ops/s
[info] ByteString_dropSliceTake_Benchmark.bss_large_dropRight_256 thrpt 20 1 191 713.229 ± 15910.426 ops/s
[info] ByteString_dropSliceTake_Benchmark.bss_large_dropRight_2000 thrpt 20 1 201 342.579 ± 21119.392 ops/s
[info] ByteString_dropSliceTake_Benchmark.bs_large_drop_100 thrpt 20 108 252 561.824 ± 3841392.346 ops/s
[info] ByteString_dropSliceTake_Benchmark.bs_large_drop_256 thrpt 20 112 515 936.237 ± 5651549.124 ops/s
[info] ByteString_dropSliceTake_Benchmark.bs_large_drop_2000 thrpt 20 110 851 553.706 ± 3327510.108 ops/s
[info] ByteString_dropSliceTake_Benchmark.bss_large_drop_18 thrpt 20 983 544.541 ± 46299.808 ops/s
[info] ByteString_dropSliceTake_Benchmark.bss_large_drop_100 thrpt 20 875 345.433 ± 44760.533 ops/s
[info] ByteString_dropSliceTake_Benchmark.bss_large_drop_256 thrpt 20 864 182.258 ± 111172.303 ops/s
[info] ByteString_dropSliceTake_Benchmark.bss_large_drop_2000 thrpt 20 997 459.151 ± 33627.993 ops/s
[info] ByteString_dropSliceTake_Benchmark.bs_large_slice_80_80 thrpt 20 112 299 538.691 ± 7259114.294 ops/s
[info] ByteString_dropSliceTake_Benchmark.bs_large_slice_129_129 thrpt 20 105 640 836.625 ± 9112709.942 ops/s
[info] ByteString_dropSliceTake_Benchmark.bss_large_slice_80_80 thrpt 20 10 868 202.262 ± 526537.133 ops/s
[info] ByteString_dropSliceTake_Benchmark.bss_large_slice_129_129 thrpt 20 9 429 199.802 ± 1321542.453 ops/s
--------------------------------- AFTER -----------------------------------------------------------------------
------ TODAY
[info] Benchmark Mode Cnt Score Error Units
[info] ByteString_dropSliceTake_Benchmark.bs_large_dropRight_100 thrpt 20 126 091 961.654 ± 2813125.268 ops/s
[info] ByteString_dropSliceTake_Benchmark.bs_large_dropRight_256 thrpt 20 118 393 394.350 ± 2934782.759 ops/s
[info] ByteString_dropSliceTake_Benchmark.bs_large_dropRight_2000 thrpt 20 119 183 386.004 ± 4445324.298 ops/s
[info] ByteString_dropSliceTake_Benchmark.bss_large_dropRight_100 thrpt 20 8 813 065.392 ± 234570.880 ops/s
[info] ByteString_dropSliceTake_Benchmark.bss_large_dropRight_256 thrpt 20 9 039 585.934 ± 297168.301 ops/s
[info] ByteString_dropSliceTake_Benchmark.bss_large_dropRight_2000 thrpt 20 9 629 458.168 ± 124846.904 ops/s
[info] ByteString_dropSliceTake_Benchmark.bs_large_drop_100 thrpt 20 111 666 137.955 ± 4846727.674 ops/s
[info] ByteString_dropSliceTake_Benchmark.bs_large_drop_256 thrpt 20 114 405 514.622 ± 4985750.805 ops/s
[info] ByteString_dropSliceTake_Benchmark.bs_large_drop_2000 thrpt 20 114 364 716.297 ± 2512280.603 ops/s
[info] ByteString_dropSliceTake_Benchmark.bss_large_drop_18 thrpt 20 10 040 457.962 ± 527850.116 ops/s
[info] ByteString_dropSliceTake_Benchmark.bss_large_drop_100 thrpt 20 9 184 934.769 ± 549140.840 ops/s
[info] ByteString_dropSliceTake_Benchmark.bss_large_drop_256 thrpt 20 10 887 437.121 ± 195606.240 ops/s
[info] ByteString_dropSliceTake_Benchmark.bss_large_drop_2000 thrpt 20 10 725 300.292 ± 403470.413 ops/s
[info] ByteString_dropSliceTake_Benchmark.bs_large_slice_80_80 thrpt 20 233 017 314.148 ± 7070246.826 ops/s
[info] ByteString_dropSliceTake_Benchmark.bs_large_slice_129_129 thrpt 20 275 245 086.247 ± 4969752.048 ops/s
[info] ByteString_dropSliceTake_Benchmark.bss_large_slice_80_80 thrpt 20 264 963 420.976 ± 4259289.143 ops/s
[info] ByteString_dropSliceTake_Benchmark.bss_large_slice_129_129 thrpt 20 265 477 577.022 ± 4623974.283 ops/s
*/
// 18 == "http://example.com", a typical url length
@Benchmark
def bs_large_drop_0: ByteString =
bs_large.drop(0)
@Benchmark
def bss_large_drop_0: ByteString =
bss_large.drop(0)
@Benchmark
def bs_large_drop_18: ByteString =
bs_large.drop(18)
@Benchmark
def bss_large_drop_18: ByteString =
bss_large.drop(18)
@Benchmark
def bs_large_drop_100: ByteString =
bs_large.drop(100)
@Benchmark
def bss_large_drop_100: ByteString =
bss_large.drop(100)
@Benchmark
def bs_large_drop_256: ByteString =
bs_large.drop(256)
@Benchmark
def bss_large_drop_256: ByteString =
bss_large.drop(256)
@Benchmark
def bs_large_drop_2000: ByteString =
bs_large.drop(2000)
@Benchmark
def bss_large_drop_2000: ByteString =
bss_large.drop(2000)
/* these force 2 array drops, and 1 element drop inside the 2nd to first/last; can be considered as "bad case" */
@Benchmark
def bs_large_slice_129_129: ByteString =
bs_large.slice(129, 129)
@Benchmark
def bss_large_slice_129_129: ByteString =
bss_large.slice(129, 129)
/* these only move the indexes, don't drop any arrays "happy case" */
@Benchmark
def bs_large_slice_80_80: ByteString =
bs_large.slice(80, 80)
@Benchmark
def bss_large_slice_80_80: ByteString =
bss_large.slice(80, 80)
// drop right ---
@Benchmark
def bs_large_dropRight_100: ByteString =
bs_large.dropRight(100)
@Benchmark
def bss_large_dropRight_100: ByteString =
bss_large.dropRight(100)
@Benchmark
def bs_large_dropRight_256: ByteString =
bs_large.dropRight(256)
@Benchmark
def bss_large_dropRight_256: ByteString =
bss_large.dropRight(256)
@Benchmark
def bs_large_dropRight_2000: ByteString =
bs_large.dropRight(2000)
@Benchmark
def bss_large_dropRight_2000: ByteString =
bss_large.dropRight(2000)
}

View file

@ -0,0 +1,71 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.util
import java.util.concurrent.TimeUnit
import scala.util.Random
import org.openjdk.jmh.annotations._
import org.apache.pekko.util.ByteString.{ ByteString1, ByteStrings }
@State(Scope.Benchmark)
@Measurement(timeUnit = TimeUnit.MILLISECONDS)
class ByteString_drop_Benchmark {
val str = List.fill[Byte](4)(0).mkString
val numVec = 1024
val bss = ByteStrings(Vector.fill(numVec)(ByteString1.fromString(str)))
val rand = new Random()
val len = str.size * numVec
val n_greater_or_eq_to_len = len + rand.nextInt(Int.MaxValue - len)
val n_neg = rand.nextInt(Int.MaxValue) * -1
val n_avg = len / 2
val n_best = 1
val n_worst = len - 1
/*
--------------------------------- BASELINE ------------------------------------------------------------------
[info] Benchmark Mode Cnt Score Error Units
[info] ByteString_drop_Benchmark.bss_avg thrpt 40 544841.222 ± 12917.565 ops/s
[info] ByteString_drop_Benchmark.bss_best thrpt 40 10141204.609 ± 415441.925 ops/s
[info] ByteString_drop_Benchmark.bss_greater_or_eq_to_len thrpt 40 902173327.723 ± 9921650.983 ops/s
[info] ByteString_drop_Benchmark.bss_negative thrpt 40 1179430602.793 ± 12193702.247 ops/s
[info] ByteString_drop_Benchmark.bss_worst thrpt 40 297489.038 ± 5534.801 ops/s
*/
@Benchmark
def bss_negative(): Unit = {
@volatile var m: ByteString = null
m = bss.drop(n_neg)
}
@Benchmark
def bss_greater_or_eq_to_len(): Unit = {
@volatile var m: ByteString = null
m = bss.drop(n_greater_or_eq_to_len)
}
@Benchmark
def bss_avg(): Unit = {
@volatile var m: ByteString = null
m = bss.drop(n_avg)
}
@Benchmark
def bss_best(): Unit = {
@volatile var m: ByteString = null
m = bss.drop(n_best)
}
@Benchmark
def bss_worst(): Unit = {
@volatile var m: ByteString = null
m = bss.drop(n_worst)
}
}

View file

@ -0,0 +1,31 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.util
import java.util.concurrent.TimeUnit
import org.openjdk.jmh.annotations._
import org.openjdk.jmh.infra.Blackhole
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.AverageTime))
@OutputTimeUnit(TimeUnit.NANOSECONDS)
@Warmup(iterations = 5, time = 1, timeUnit = TimeUnit.SECONDS)
@Measurement(iterations = 10, time = 1, timeUnit = TimeUnit.SECONDS)
class ByteString_grouped_Benchmark {
private val bsLarge = ByteString(Array.ofDim[Byte](1000 * 1000))
/*
> akka-bench-jmh/jmh:run -f1 .*ByteString_grouped_Benchmark
[info] Benchmark Mode Cnt Score Error Units
[info] ByteString_grouped_Benchmark.grouped avgt 10 59386.328 ± 1466.045 ns/op
*/
@Benchmark
def grouped(bh: Blackhole): Unit = {
bh.consume(bsLarge.grouped(1000).foreach(bh.consume))
}
}

View file

@ -0,0 +1,46 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.util
import java.util.concurrent.TimeUnit
import org.openjdk.jmh.annotations._
@State(Scope.Benchmark)
@Measurement(timeUnit = TimeUnit.MILLISECONDS)
class ByteString_indexOf_Benchmark {
val start = ByteString("abcdefg") ++ ByteString("hijklmno") ++ ByteString("pqrstuv")
val bss = start ++ start ++ start ++ start ++ start ++ ByteString("xyz")
val bs = bss.compact // compacted
/*
original
ByteString_indexOf_Benchmark.bs1_indexOf_from thrpt 20 999335.124 ± 234047.176 ops/s
ByteString_indexOf_Benchmark.bss_indexOf_from_best_case thrpt 20 42735542.833 ± 1082874.815 ops/s
ByteString_indexOf_Benchmark.bss_indexOf_from_far_index_case thrpt 20 4941422.104 ± 109132.224 ops/s
ByteString_indexOf_Benchmark.bss_indexOf_from_worst_case thrpt 20 328123.207 ± 16550.271 ops/s
optimized
ByteString_indexOf_Benchmark.bs1_indexOf_from thrpt 20 339488707.553 ± 9680274.621 ops/s
ByteString_indexOf_Benchmark.bss_indexOf_from_best_case thrpt 20 126385479.889 ± 3644024.423 ops/s
ByteString_indexOf_Benchmark.bss_indexOf_from_far_index_case thrpt 20 14282036.963 ± 529652.214 ops/s
ByteString_indexOf_Benchmark.bss_indexOf_from_worst_case thrpt 20 7815676.051 ± 323031.073 ops/s
*/
@Benchmark
def bss_indexOf_from_worst_case: Int = bss.indexOf('z', 1)
@Benchmark
def bss_indexOf_from_far_index_case: Int = bss.indexOf('z', 109)
@Benchmark
def bss_indexOf_from_best_case: Int = bss.indexOf('a', 0)
@Benchmark
def bs1_indexOf_from: Int = bs.indexOf('ö', 5)
}

View file

@ -0,0 +1,77 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.util
import java.util.concurrent.TimeUnit
import scala.util.Random
import org.openjdk.jmh.annotations.{ Benchmark, Measurement, Scope, State }
import org.apache.pekko.util.ByteString.{ ByteString1, ByteStrings }
@State(Scope.Benchmark)
@Measurement(timeUnit = TimeUnit.MILLISECONDS)
class ByteString_take_Benchmark {
val str = List.fill[Byte](4)(0).mkString
val numVec = 1024
val bss = ByteStrings(Vector.fill(numVec)(ByteString1.fromString(str)))
val rand = new Random()
val len = str.size * numVec
val n_greater_or_eq_to_len = len + rand.nextInt(Int.MaxValue - len)
val n_neg = rand.nextInt(Int.MaxValue) * -1
val n_avg = len / 2
val n_best = 1
val n_worst = len - 1
/*
--------------------------------- BASELINE -------------------------------------------------------------------
commit 0f2da7b26b5c4af35be87d2bd4a1a2392365df15
[info] Benchmark Mode Cnt Score Error Units
[info] ByteString_take_Benchmark.bss_avg thrpt 40 28710.870 ± 437.608 ops/s
[info] ByteString_take_Benchmark.bss_best thrpt 40 20987018.075 ± 443608.693 ops/s
[info] ByteString_take_Benchmark.bss_greater_or_eq_to_len thrpt 40 894573619.213 ± 4360367.026 ops/s
[info] ByteString_take_Benchmark.bss_negative thrpt 40 1164398934.041 ± 15083443.165 ops/s
[info] ByteString_take_Benchmark.bss_worst thrpt 40 11936.857 ± 373.828 ops/s
--------------------------------- AFTER ----------------------------------------------------------------------
------ TODAY
[info] Benchmark Mode Cnt Score Error Units
[info] ByteString_take_Benchmark.bss_avg thrpt 40 539211.297 ± 9073.181 ops/s
[info] ByteString_take_Benchmark.bss_best thrpt 40 197237882.251 ± 2714956.732 ops/s
[info] ByteString_take_Benchmark.bss_greater_or_eq_to_len thrpt 40 866558812.838 ± 15343155.818 ops/s
[info] ByteString_take_Benchmark.bss_negative thrpt 40 1114723770.487 ± 30945339.512 ops/s
[info] ByteString_take_Benchmark.bss_worst thrpt 40 233870.585 ± 7326.227 ops/s
*/
@Benchmark
def bss_negative(): ByteString = {
bss.take(n_neg)
}
@Benchmark
def bss_greater_or_eq_to_len(): ByteString = {
bss.take(n_greater_or_eq_to_len)
}
@Benchmark
def bss_avg(): ByteString = {
bss.take(n_avg)
}
@Benchmark
def bss_best(): ByteString = {
bss.take(n_best)
}
@Benchmark
def bss_worst(): ByteString = {
bss.take(n_worst)
}
}

View file

@ -0,0 +1,73 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.util
import java.util.concurrent.TimeUnit
import org.openjdk.jmh.annotations._
import org.openjdk.jmh.infra.Blackhole
@State(Scope.Benchmark)
@Measurement(timeUnit = TimeUnit.MILLISECONDS)
class ByteString_toArray_Benchmark {
var bs: ByteString = _
var composed: ByteString = _
@Param(Array("10", "100", "1000"))
var kb = 0
/*
akka-bench-jmh/jmh:run -f 1 -wi 5 -i 5 .*ByteString_toArray_Benchmark.*
Benchmark (kb) Mode Cnt Score Error Units
2.12
composed_bs_to_array 10 thrpt 5 5625.395 ± 145.999 ops/s
composed_bs_to_array 100 thrpt 5 464.893 ± 33.579 ops/s
composed_bs_to_array 1000 thrpt 5 45.482 ± 4.217 ops/s
single_bs_to_array 10 thrpt 5 1450077.491 ± 27964.993 ops/s
single_bs_to_array 100 thrpt 5 72201.806 ± 637.800 ops/s
single_bs_to_array 1000 thrpt 5 5491.724 ± 178.696 ops/s
2.13 before (second) fix
composed_bs_to_array 10 thrpt 5 128.706 ± 4.590 ops/s
composed_bs_to_array 100 thrpt 5 13.147 ± 0.435 ops/s
composed_bs_to_array 1000 thrpt 5 1.255 ± 0.057 ops/s
single_bs_to_array 10 thrpt 5 1336977.040 ± 719295.197 ops/s
single_bs_to_array 100 thrpt 5 70202.111 ± 522.435 ops/s
single_bs_to_array 1000 thrpt 5 5444.186 ± 224.677 ops/s
2.13 with (second) fix
composed_bs_to_array 10 thrpt 5 5970.395 ± 348.356 ops/s
composed_bs_to_array 100 thrpt 5 479.762 ± 15.819 ops/s
composed_bs_to_array 1000 thrpt 5 45.940 ± 1.306 ops/s
single_bs_to_array 10 thrpt 5 1468430.822 ± 38730.696 ops/s
single_bs_to_array 100 thrpt 5 71313.855 ± 983.643 ops/s
single_bs_to_array 1000 thrpt 5 5526.564 ± 143.654 ops/s
*/
@Setup
def setup(): Unit = {
val bytes = Array.ofDim[Byte](1024 * kb)
bs = ByteString(bytes)
composed = ByteString.empty
for (_ <- 0 to 100) {
composed = composed ++ bs
}
}
@Benchmark
def single_bs_to_array(blackhole: Blackhole): Unit = {
blackhole.consume(bs.toArray[Byte])
}
@Benchmark
def composed_bs_to_array(blackhole: Blackhole): Unit = {
blackhole.consume(composed.toArray[Byte])
}
}

View file

@ -0,0 +1,53 @@
/*
* Copyright (C) 2021-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.util
import org.openjdk.jmh.annotations.Benchmark
import org.openjdk.jmh.annotations.Fork
import org.openjdk.jmh.annotations.Measurement
import org.openjdk.jmh.annotations.Scope
import org.openjdk.jmh.annotations.Setup
import org.openjdk.jmh.annotations.State
import org.openjdk.jmh.annotations.Warmup
import java.util.concurrent.TimeUnit
@State(Scope.Benchmark)
@Fork(1)
@Warmup(iterations = 3, time = 20, timeUnit = TimeUnit.SECONDS)
@Measurement(iterations = 3, time = 10, timeUnit = TimeUnit.SECONDS)
class FastFrequencySketchBenchmark {
private[this] val Capacity = 10000
private[this] val GeneratedSize = 1 << 16
private final val IndexMask = 0xFFFF
private[this] var sketch: FastFrequencySketch[String] = _
private[this] var generated: Array[String] = _
private[this] var index: Int = 0
@Setup
def setup(): Unit = {
sketch = FastFrequencySketch[String](Capacity)
generated = new Array[String](GeneratedSize)
val generator = ZipfianGenerator(GeneratedSize)
for (i <- 0 until GeneratedSize) {
generated(i) = generator.next().intValue.toString
sketch.increment(i.toString)
}
}
@Benchmark
def increment(): Unit = {
sketch.increment(generated(index & IndexMask))
index += 1
}
@Benchmark
def frequency: Int = {
val count = sketch.frequency(generated(index & IndexMask))
index += 1
count
}
}

View file

@ -0,0 +1,53 @@
/*
* Copyright (C) 2021-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.util
import org.openjdk.jmh.annotations.Benchmark
import org.openjdk.jmh.annotations.Fork
import org.openjdk.jmh.annotations.Measurement
import org.openjdk.jmh.annotations.Scope
import org.openjdk.jmh.annotations.Setup
import org.openjdk.jmh.annotations.State
import org.openjdk.jmh.annotations.Warmup
import java.util.concurrent.TimeUnit
@State(Scope.Benchmark)
@Fork(1)
@Warmup(iterations = 3, time = 20, timeUnit = TimeUnit.SECONDS)
@Measurement(iterations = 3, time = 10, timeUnit = TimeUnit.SECONDS)
class FrequencySketchBenchmark {
private[this] val Capacity = 10000
private[this] val GeneratedSize = 1 << 16
private final val IndexMask = 0xFFFF
private[this] var sketch: FrequencySketch[String] = _
private[this] var generated: Array[String] = _
private[this] var index: Int = 0
@Setup
def setup(): Unit = {
sketch = FrequencySketch[String](Capacity)
generated = new Array[String](GeneratedSize)
val generator = ZipfianGenerator(GeneratedSize)
for (i <- 0 until GeneratedSize) {
generated(i) = generator.next().intValue.toString
sketch.increment(i.toString)
}
}
@Benchmark
def increment(): Unit = {
sketch.increment(generated(index & IndexMask))
index += 1
}
@Benchmark
def frequency: Int = {
val count = sketch.frequency(generated(index & IndexMask))
index += 1
count
}
}

View file

@ -0,0 +1,115 @@
/*
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.util
import java.util.concurrent.TimeUnit
import scala.annotation.tailrec
import org.openjdk.jmh.annotations._
@State(Scope.Benchmark)
@BenchmarkMode(Array(Mode.Throughput))
@Fork(1)
@Threads(1)
@Warmup(iterations = 10, time = 5, timeUnit = TimeUnit.MICROSECONDS, batchSize = 1)
@Measurement(iterations = 10, time = 15, timeUnit = TimeUnit.MICROSECONDS, batchSize = 1)
class ImmutableIntMapBench {
@tailrec private[this] final def add(n: Int, c: ImmutableIntMap = ImmutableIntMap.empty): ImmutableIntMap =
if (n >= 0) add(n - 1, c.updated(n, n))
else c
@tailrec private[this] final def contains(n: Int, by: Int, to: Int, in: ImmutableIntMap, b: Boolean): Boolean =
if (n <= to) {
val result = in.contains(n)
contains(n + by, by, to, in, result)
} else b
@tailrec private[this] final def get(n: Int, by: Int, to: Int, in: ImmutableIntMap, b: Int): Int =
if (n <= to) {
val result = in.get(n)
get(n + by, by, to, in, result)
} else b
@tailrec private[this] final def hashCode(n: Int, in: ImmutableIntMap, b: Int): Int =
if (n >= 0) {
val result = in.hashCode
hashCode(n - 1, in, result)
} else b
@tailrec private[this] final def updateIfAbsent(n: Int, by: Int, to: Int, in: ImmutableIntMap): ImmutableIntMap =
if (n <= to) updateIfAbsent(n + by, by, to, in.updateIfAbsent(n, n))
else in
@tailrec private[this] final def getKey(iterations: Int, key: Int, from: ImmutableIntMap): ImmutableIntMap = {
if (iterations > 0 && key != Int.MinValue) {
val k = from.get(key)
getKey(iterations - 1, k, from)
} else from
}
val odd1000 = (0 to 1000).iterator.filter(_ % 2 == 1).foldLeft(ImmutableIntMap.empty)((l, i) => l.updated(i, i))
@Benchmark
@OperationsPerInvocation(1)
def add1(): ImmutableIntMap = add(1)
@Benchmark
@OperationsPerInvocation(10)
def add10(): ImmutableIntMap = add(10)
@Benchmark
@OperationsPerInvocation(100)
def add100(): ImmutableIntMap = add(100)
@Benchmark
@OperationsPerInvocation(1000)
def add1000(): ImmutableIntMap = add(1000)
@Benchmark
@OperationsPerInvocation(10000)
def add10000(): ImmutableIntMap = add(10000)
@Benchmark
@OperationsPerInvocation(500)
def contains(): Boolean = contains(n = 1, by = 2, to = odd1000.size, in = odd1000, b = false)
@Benchmark
@OperationsPerInvocation(500)
def notcontains(): Boolean = contains(n = 0, by = 2, to = odd1000.size, in = odd1000, b = false)
@Benchmark
@OperationsPerInvocation(500)
def get(): Int = get(n = 1, by = 2, to = odd1000.size, in = odd1000, b = Int.MinValue)
@Benchmark
@OperationsPerInvocation(500)
def notget(): Int = get(n = 0, by = 2, to = odd1000.size, in = odd1000, b = Int.MinValue)
@Benchmark
@OperationsPerInvocation(500)
def updateNotAbsent(): ImmutableIntMap = updateIfAbsent(n = 1, by = 2, to = odd1000.size, in = odd1000)
@Benchmark
@OperationsPerInvocation(500)
def updateAbsent(): ImmutableIntMap = updateIfAbsent(n = 0, by = 2, to = odd1000.size, in = odd1000)
@Benchmark
@OperationsPerInvocation(10000)
def hashcode(): Int = hashCode(10000, odd1000, 0)
@Benchmark
@OperationsPerInvocation(1000)
def getMidElement(): ImmutableIntMap = getKey(iterations = 1000, key = 249, from = odd1000)
@Benchmark
@OperationsPerInvocation(1000)
def getLoElement(): ImmutableIntMap = getKey(iterations = 1000, key = 1, from = odd1000)
@Benchmark
@OperationsPerInvocation(1000)
def getHiElement(): ImmutableIntMap = getKey(iterations = 1000, key = 999, from = odd1000)
}

View file

@ -0,0 +1,81 @@
/*
* Copyright (C) 2016-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.util
import java.util
import java.util.concurrent.TimeUnit
import scala.util.Random
import org.openjdk.jmh.annotations.{ Param, _ }
import org.apache.pekko.remote.artery.LruBoundedCache
@State(Scope.Benchmark)
@Measurement(timeUnit = TimeUnit.MICROSECONDS)
class LruBoundedCacheBench {
var javaHashMap: java.util.HashMap[String, String] = _
@Param(Array("1024", "8192"))
var count = 0
@Param(Array("128", "256"))
var stringSize = 0
private var lruCache: LruBoundedCache[String, String] = _
@Param(Array("90", "99"))
var loadFactor: Int = _
var toAdd: String = _
var toRemove: String = _
var toGet: String = _
@Setup
def setup(): Unit = {
val loadF: Double = loadFactor / 100.0
val threshold = (loadF * count).toInt
val random = Random
javaHashMap = new util.HashMap[String, String](count)
lruCache = new LruBoundedCache[String, String](count, threshold) {
override protected def compute(k: String): String = k
override protected def hash(k: String): Int = k.hashCode
override protected def isCacheable(v: String): Boolean = true
override protected def isKeyCacheable(k: 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)
}
}

View file

@ -0,0 +1,30 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.util
import java.util.concurrent.TimeUnit
import org.openjdk.jmh.annotations.{ Benchmark, Measurement, Scope, State }
@State(Scope.Benchmark)
@Measurement(timeUnit = TimeUnit.MICROSECONDS)
class StackBench {
class CustomSecurtyManager extends SecurityManager {
def getTrace: Array[Class[_]] =
getClassContext
}
@Benchmark
def currentThread(): Array[StackTraceElement] = {
Thread.currentThread().getStackTrace
}
@Benchmark
def securityManager(): Array[Class[_]] = {
(new CustomSecurtyManager).getTrace
}
}