diff --git a/.gitignore b/.gitignore
index 8f7a660..8b75dcf 100644
--- a/.gitignore
+++ b/.gitignore
@@ -2,4 +2,7 @@
.idea
target
*.iml
-tools/release
\ No newline at end of file
+tools/release
+
+*.json
+*.log
\ No newline at end of file
diff --git a/openmessaging-benchmark/README.md b/openmessaging-benchmark/README.md
new file mode 100644
index 0000000..c315413
--- /dev/null
+++ b/openmessaging-benchmark/README.md
@@ -0,0 +1,124 @@
+# OpenMessaging Benchmark for Apache Fluss
+
+A benchmark suite based on the [OpenMessaging Benchmark](https://openmessaging.cloud/docs/benchmarks/) (OMB) framework for measuring the performance of [Apache Fluss (Incubating)](https://github.com/apache/fluss).
+
+## Project Structure
+
+```
+openmessaging-benchmark/
+├── driver-api/ # Benchmark driver interface
+├── benchmark-framework/ # Core benchmarking engine
+├── driver-fluss/ # Fluss driver implementation
+│ └── config/fluss.yaml # Fluss driver configuration
+├── package/ # Assembly and packaging
+├── bin/ # Launcher scripts
+├── workloads/ # Workload definitions
+├── payload/ # Pre-generated payload data files
+└── etc/ # Shared build config (checkstyle, spotbugs, etc.)
+```
+
+## Prerequisites
+
+- Java 8+ (Java 11+ recommended for Arrow format)
+- Maven 3.8.6+
+- A running Apache Fluss cluster
+
+## Build
+
+```bash
+cd openmessaging-benchmark
+mvn clean package -DskipTests
+```
+
+## Usage
+
+### Local Mode
+
+Run the benchmark on a single machine:
+
+```bash
+bin/benchmark-local \
+ -d driver-fluss/config/fluss.yaml \
+ workloads/1-topic-1-partition-1kb.yaml
+```
+
+### Distributed Mode
+
+Start worker processes on each worker node:
+
+```bash
+bin/benchmark-worker
+```
+
+Then run the benchmark from the driver node:
+
+```bash
+bin/benchmark \
+ -d driver-fluss/config/fluss.yaml \
+ -w http://worker1:8080,http://worker2:8080 \
+ workloads/1-topic-16-partitions-1kb.yaml
+```
+
+### CLI Options
+
+| Option | Description |
+|--------|-------------|
+| `-d, --drivers` | Driver config file(s), e.g. `driver-fluss/config/fluss.yaml` |
+| `-w, --workers` | Comma-separated worker addresses for distributed mode |
+| `-wf, --workers-file` | YAML file containing worker addresses |
+| `-o, --output` | Output directory for JSON results |
+| `-c, --csv` | Convert JSON results in a directory to CSV |
+
+## Configuration
+
+### Driver Configuration (`driver-fluss/config/fluss.yaml`)
+
+| Parameter | Description | Default |
+|-----------|-------------|---------|
+| `bootstrapServers` | Fluss cluster address | `localhost:9123` |
+| `schema` | Row schema (first field must be `long` for E2E timestamp) | `long-int-int-string-string` |
+| `logFormat` | Log format: `ARROW` or `INDEXED` | `ARROW` |
+| `writerAcks` | Writer acknowledgment mode | `all` |
+| `writerBatchSize` | Writer batch size | `1mb` |
+| `writerBufferMemory` | Writer buffer memory | `32mb` |
+| `writerBatchTimeoutMs` | Writer batch timeout in milliseconds | `100` |
+| `projectFields` | Consumer field projection (`all` or field indices like `0/1/2`) | `all` |
+| `prefetchNum` | Consumer prefetch count | `4` |
+| `fetchMaxBytes` | Consumer max fetch bytes | `16mb` |
+| `clientNettyThreads` | Number of Netty client threads | `1` |
+
+### Workload Configuration (`workloads/`)
+
+| Parameter | Description |
+|-----------|-------------|
+| `topics` | Number of topics |
+| `partitionsPerTopic` | Partitions per topic |
+| `messageSize` | Message size in bytes |
+| `producersPerTopic` | Number of producers per topic |
+| `consumerPerSubscription` | Number of consumers per subscription |
+| `producerRate` | Target produce rate (msg/s), `0` for auto-discovery |
+| `testDurationMinutes` | Benchmark test duration |
+| `warmupDurationMinutes` | Warmup duration before measurement |
+| `useRandomizedPayloads` | Whether to use randomized payloads |
+| `randomBytesRatio` | Ratio of random bytes in payload |
+
+## Metrics
+
+The benchmark collects and reports the following metrics every 10 seconds:
+
+| Metric | Unit | Description |
+|--------|------|-------------|
+| Publish Rate | msg/s | Messages published per second |
+| Publish Throughput | MB/s | Bytes published per second |
+| Consume Rate | msg/s | Messages consumed per second |
+| Consume Throughput | MB/s | Bytes consumed per second |
+| Publish Latency | ms | Latency from send call to ack (avg, P50, P99, P99.9, max) |
+| Publish Delay Latency | us | Delay between intended and actual send time |
+| End-to-End Latency | ms | Latency from publish to consume |
+| Backlog | count | Unconsumed message count |
+
+Results are written to a JSON file (e.g. `1-topic-1-partition-1kb-Fluss-2026-04-26-20-44-27.json`) containing per-interval time series and aggregated percentiles for all metrics.
+
+## License
+
+Apache License, Version 2.0
diff --git a/openmessaging-benchmark/benchmark-framework/pom.xml b/openmessaging-benchmark/benchmark-framework/pom.xml
new file mode 100644
index 0000000..1864161
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/pom.xml
@@ -0,0 +1,147 @@
+
+
+
+ 4.0.0
+
+ io.openmessaging.benchmark
+ messaging-benchmark
+ 0.0.1-SNAPSHOT
+
+
+ benchmark-framework
+
+
+ 9.4.42.v20210604
+ ${project.basedir}/..
+
+
+
+
+ ${project.groupId}
+ driver-api
+ ${project.version}
+
+
+ com.beust
+ jcommander
+
+
+ io.javalin
+ javalin
+ 1.3.0
+
+
+ org.apache.bookkeeper.stats
+ prometheus-metrics-provider
+ ${bookkeeper.version}
+
+
+ io.netty
+ netty-common
+
+
+
+
+ org.apache.commons
+ commons-lang3
+
+
+ org.apache.logging.log4j
+ log4j-slf4j-impl
+
+
+ org.asynchttpclient
+ async-http-client
+ 3.0.0
+
+
+
+ org.eclipse.jetty
+ jetty-server
+ ${jetty.version}
+
+
+ org.eclipse.jetty
+ jetty-util
+ ${jetty.version}
+
+
+ org.hdrhistogram
+ HdrHistogram
+ 2.1.12
+
+
+ org.projectlombok
+ lombok
+ provided
+
+
+ com.github.stefanbirkner
+ system-lambda
+ 1.2.1
+ test
+
+
+ org.assertj
+ assertj-core
+ test
+
+
+ org.junit.jupiter
+ junit-jupiter
+ test
+
+
+ org.mockito
+ mockito-junit-jupiter
+ test
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-assembly-plugin
+ 2.4.1
+
+
+
+ jar-with-dependencies
+
+
+
+
+ util.Microseer
+
+
+
+
+
+ make-assembly
+
+ single
+
+
+ package
+
+
+
+
+
+
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/Benchmark.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/Benchmark.java
new file mode 100644
index 0000000..1d9a0bb
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/Benchmark.java
@@ -0,0 +1,215 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark;
+
+import static java.util.stream.Collectors.toList;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+import io.openmessaging.benchmark.worker.DistributedWorkersEnsemble;
+import io.openmessaging.benchmark.worker.HttpWorkerClient;
+import io.openmessaging.benchmark.worker.LocalWorker;
+import io.openmessaging.benchmark.worker.Worker;
+import java.io.File;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class Benchmark {
+
+ static class Arguments {
+
+ @Parameter(
+ names = {"-c", "--csv"},
+ description = "Print results from this directory to a csv file")
+ String resultsDir;
+
+ @Parameter(
+ names = {"-h", "--help"},
+ description = "Help message",
+ help = true)
+ boolean help;
+
+ @Parameter(
+ names = {"-d", "--drivers"},
+ description = "Drivers list. eg.: driver-fluss/fluss.yaml") // , required = true)
+ public List drivers;
+
+ @Parameter(
+ names = {"-w", "--workers"},
+ description = "List of worker nodes. eg: http://1.2.3.4:8080,http://4.5.6.7:8080")
+ public List workers;
+
+ @Parameter(
+ names = {"-wf", "--workers-file"},
+ description = "Path to a YAML file containing the list of workers addresses")
+ public File workersFile;
+
+ @Parameter(
+ names = {"-x", "--extra"},
+ description = "Allocate extra consumer workers when your backlog builds.")
+ boolean extraConsumers;
+
+ @Parameter(description = "Workloads") // , required = true)
+ public List workloads;
+
+ @Parameter(
+ names = {"-o", "--output"},
+ description = "Output",
+ required = false)
+ public String output;
+ }
+
+ public static void main(String[] args) throws Exception {
+ final Arguments arguments = new Arguments();
+ JCommander jc = new JCommander(arguments);
+ jc.setProgramName("messaging-benchmark");
+
+ try {
+ jc.parse(args);
+ } catch (ParameterException e) {
+ System.err.println(e.getMessage());
+ jc.usage();
+ System.exit(-1);
+ }
+
+ if (arguments.help) {
+ jc.usage();
+ System.exit(-1);
+ }
+
+ if (arguments.resultsDir != null) {
+ ResultsToCsv r = new ResultsToCsv();
+ r.writeAllResultFiles(arguments.resultsDir);
+ System.exit(0);
+ }
+
+ if (arguments.workers != null && arguments.workersFile != null) {
+ System.err.println("Only one between --workers and --workers-file can be specified");
+ System.exit(-1);
+ }
+
+ if (arguments.workers == null && arguments.workersFile == null) {
+ File defaultFile = new File("workers.yaml");
+ if (defaultFile.exists()) {
+ log.info("Using default worker file workers.yaml");
+ arguments.workersFile = defaultFile;
+ }
+ }
+
+ if (arguments.workersFile != null) {
+ log.info("Reading workers list from {}", arguments.workersFile);
+ arguments.workers = mapper.readValue(arguments.workersFile, Workers.class).workers;
+ }
+
+ // Dump configuration variables
+ log.info("Starting benchmark with config: {}", writer.writeValueAsString(arguments));
+
+ Map workloads = new TreeMap<>();
+ for (String path : arguments.workloads) {
+ File file = new File(path);
+ String name = file.getName().substring(0, file.getName().lastIndexOf('.'));
+
+ workloads.put(name, mapper.readValue(file, Workload.class));
+ }
+
+ log.info("Workloads: {}", writer.writeValueAsString(workloads));
+
+ Worker worker;
+
+ if (arguments.workers != null && !arguments.workers.isEmpty()) {
+ List workers =
+ arguments.workers.stream().map(HttpWorkerClient::new).collect(toList());
+ worker = new DistributedWorkersEnsemble(workers, arguments.extraConsumers);
+ } else {
+ // Use local worker implementation
+ worker = new LocalWorker();
+ }
+
+ workloads.forEach(
+ (workloadName, workload) -> {
+ arguments.drivers.forEach(
+ driverConfig -> {
+ try {
+ DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd-HH-mm-ss");
+ File driverConfigFile = new File(driverConfig);
+ DriverConfiguration driverConfiguration =
+ mapper.readValue(driverConfigFile, DriverConfiguration.class);
+ log.info(
+ "--------------- WORKLOAD : {} --- DRIVER : {}---------------",
+ workload.name,
+ driverConfiguration.name);
+
+ // Stop any left over workload
+ worker.stopAll();
+
+ worker.initializeDriver(new File(driverConfig));
+
+ WorkloadGenerator generator =
+ new WorkloadGenerator(driverConfiguration.name, workload, worker);
+
+ TestResult result = generator.run();
+
+ boolean useOutput = (arguments.output != null) && (arguments.output.length() > 0);
+
+ String fileName =
+ useOutput
+ ? arguments.output
+ : String.format(
+ "%s-%s-%s.json",
+ workloadName,
+ driverConfiguration.name,
+ dateFormat.format(new Date()));
+
+ log.info("Writing test result into {}", fileName);
+ writer.writeValue(new File(fileName), result);
+
+ generator.close();
+ } catch (Exception e) {
+ log.error(
+ "Failed to run the workload '{}' for driver '{}'",
+ workload.name,
+ driverConfig,
+ e);
+ } finally {
+ worker.stopAll();
+ }
+ });
+ });
+
+ worker.close();
+ }
+
+ private static final ObjectMapper mapper =
+ new ObjectMapper(new YAMLFactory())
+ .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+ static {
+ mapper.enable(DeserializationFeature.READ_UNKNOWN_ENUM_VALUES_USING_DEFAULT_VALUE);
+ }
+
+ private static final ObjectWriter writer = new ObjectMapper().writerWithDefaultPrettyPrinter();
+
+ private static final Logger log = LoggerFactory.getLogger(Benchmark.class);
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/DriverConfiguration.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/DriverConfiguration.java
new file mode 100644
index 0000000..8e6c858
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/DriverConfiguration.java
@@ -0,0 +1,20 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark;
+
+public class DriverConfiguration {
+ public String name;
+
+ public String driverClass;
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/RateController.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/RateController.java
new file mode 100644
index 0000000..edb4b91
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/RateController.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static lombok.AccessLevel.PACKAGE;
+
+import io.openmessaging.benchmark.utils.Env;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+class RateController {
+ private static final long ONE_SECOND_IN_NANOS = SECONDS.toNanos(1);
+ private final long publishBacklogLimit;
+ private final long receiveBacklogLimit;
+ private final double minRampingFactor;
+ private final double maxRampingFactor;
+
+ @Getter(PACKAGE)
+ private double rampingFactor;
+
+ private long previousTotalPublished = 0;
+ private long previousTotalReceived = 0;
+
+ RateController() {
+ publishBacklogLimit = Env.getLong("PUBLISH_BACKLOG_LIMIT", 1_000);
+ receiveBacklogLimit = Env.getLong("RECEIVE_BACKLOG_LIMIT", 1_000);
+ minRampingFactor = Env.getDouble("MIN_RAMPING_FACTOR", 0.01);
+ maxRampingFactor = Env.getDouble("MAX_RAMPING_FACTOR", 1);
+ rampingFactor = maxRampingFactor;
+ }
+
+ double nextRate(double rate, long periodNanos, long totalPublished, long totalReceived) {
+ long expected = (long) ((rate / ONE_SECOND_IN_NANOS) * periodNanos);
+ long published = totalPublished - previousTotalPublished;
+ long received = totalReceived - previousTotalReceived;
+
+ previousTotalPublished = totalPublished;
+ previousTotalReceived = totalReceived;
+
+ if (log.isDebugEnabled()) {
+ log.debug(
+ "Current rate: {} -- Publish rate {} -- Receive Rate: {}",
+ rate,
+ rate(published, periodNanos),
+ rate(received, periodNanos));
+ }
+
+ long receiveBacklog = totalPublished - totalReceived;
+ if (receiveBacklog > receiveBacklogLimit) {
+ return nextRate(periodNanos, received, expected, receiveBacklog, "Receive");
+ }
+
+ long publishBacklog = expected - published;
+ if (publishBacklog > publishBacklogLimit) {
+ return nextRate(periodNanos, published, expected, publishBacklog, "Publish");
+ }
+
+ rampUp();
+
+ return rate + (rate * rampingFactor);
+ }
+
+ private double nextRate(long periodNanos, long actual, long expected, long backlog, String type) {
+ log.debug("{} backlog: {}", type, backlog);
+ rampDown();
+ long nextExpected = Math.max(0, expected - backlog);
+ double nextExpectedRate = rate(nextExpected, periodNanos);
+ double actualRate = rate(actual, periodNanos);
+ return Math.min(actualRate, nextExpectedRate);
+ }
+
+ private double rate(long count, long periodNanos) {
+ return (count / (double) periodNanos) * ONE_SECOND_IN_NANOS;
+ }
+
+ private void rampUp() {
+ rampingFactor = Math.min(maxRampingFactor, rampingFactor * 2);
+ }
+
+ private void rampDown() {
+ rampingFactor = Math.max(minRampingFactor, rampingFactor / 2);
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/ResultsToCsv.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/ResultsToCsv.java
new file mode 100644
index 0000000..9ee84ec
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/ResultsToCsv.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.openmessaging.benchmark;
+
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.text.MessageFormat;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.HdrHistogram.Histogram;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ResultsToCsv {
+
+ private static final Logger log = LoggerFactory.getLogger(ResultsToCsv.class);
+
+ public void writeAllResultFiles(String directory) {
+ try {
+ File dir = new File(directory);
+ File[] directoryListing = dir.listFiles();
+ if (directoryListing == null) {
+ throw new IllegalArgumentException("Not a directory: " + directory);
+ }
+ Arrays.sort(directoryListing);
+
+ List lines = new ArrayList<>();
+ lines.add(
+ "topics,partitions,message-size,producers-per-topic,consumers-per-topic,"
+ + "prod-rate-min,prod-rate-avg,prod-rate-std-dev,prod-rate-max,"
+ + "con-rate-min,con-rate-avg,con-rate-std-dev,con-rate-max,");
+
+ List results = new ArrayList<>();
+ for (File file : directoryListing) {
+ if (file.isFile() && file.getAbsolutePath().endsWith(".json")) {
+ ObjectMapper objectMapper = new ObjectMapper();
+ TestResult tr =
+ objectMapper.readValue(new File(file.getAbsolutePath()), TestResult.class);
+ results.add(tr);
+ }
+ }
+
+ List sortedResults =
+ results.stream()
+ .sorted(
+ Comparator.comparing(TestResult::getMessageSize)
+ .thenComparing(TestResult::getTopics)
+ .thenComparing(TestResult::getPartitions))
+ .collect(Collectors.toList());
+ for (TestResult tr : sortedResults) {
+ lines.add(extractResults(tr));
+ }
+
+ String resultsFileName = "results-" + Instant.now().getEpochSecond() + ".csv";
+ try (FileWriter writer = new FileWriter(resultsFileName)) {
+ for (String str : lines) {
+ writer.write(str + System.lineSeparator());
+ }
+ log.info("Results extracted into CSV " + resultsFileName);
+ }
+ } catch (IOException e) {
+ log.error("Failed creating csv file.", e);
+ }
+ }
+
+ public String extractResults(TestResult tr) {
+ try {
+ Histogram prodRateHistogram = new Histogram(10000000, 1);
+ Histogram conRateHistogram = new Histogram(10000000, 1);
+
+ for (Double rate : tr.publishRate) {
+ prodRateHistogram.recordValueWithCount(rate.longValue(), 2);
+ }
+
+ for (Double rate : tr.consumeRate) {
+ conRateHistogram.recordValueWithCount(rate.longValue(), 2);
+ }
+
+ String line =
+ MessageFormat.format(
+ "{0,number,#},{1,number,#},{2,number,#},{3,number,#},{4,number,#},"
+ + "{5,number,#},{6,number,#},{7,number,#.##},{8,number,#},"
+ + "{9,number,#},{10,number,#},{11,number,#.##},{12,number,#}",
+ tr.topics,
+ tr.partitions,
+ tr.messageSize,
+ tr.producersPerTopic,
+ tr.consumersPerTopic,
+ prodRateHistogram.getMinNonZeroValue(),
+ prodRateHistogram.getMean(),
+ prodRateHistogram.getStdDeviation(),
+ prodRateHistogram.getMaxValue(),
+ conRateHistogram.getMinNonZeroValue(),
+ conRateHistogram.getMean(),
+ conRateHistogram.getStdDeviation(),
+ conRateHistogram.getMaxValue());
+
+ return line;
+ } catch (Exception e) {
+ log.error("Error writing results csv", e);
+ throw new RuntimeException(e);
+ }
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/TestResult.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/TestResult.java
new file mode 100644
index 0000000..d4a3471
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/TestResult.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark;
+
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+public class TestResult {
+ public String workload;
+ public String driver;
+ public long messageSize;
+ public int topics;
+ public int partitions;
+ public int producersPerTopic;
+ public int consumersPerTopic;
+
+ public List publishRate = new ArrayList<>();
+ public List publishThroughputMBps = new ArrayList<>();
+ public List publishErrorRate = new ArrayList<>();
+ public List consumeRate = new ArrayList<>();
+ public List consumeThroughputMBps = new ArrayList<>();
+ public List backlog = new ArrayList<>();
+
+ public List publishLatencyAvg = new ArrayList<>();
+ public List publishLatency50pct = new ArrayList<>();
+ public List publishLatency75pct = new ArrayList<>();
+ public List publishLatency95pct = new ArrayList<>();
+ public List publishLatency99pct = new ArrayList<>();
+ public List publishLatency999pct = new ArrayList<>();
+ public List publishLatency9999pct = new ArrayList<>();
+ public List publishLatencyMax = new ArrayList<>();
+
+ public List publishDelayLatencyAvg = new ArrayList<>();
+ public List publishDelayLatency50pct = new ArrayList<>();
+ public List publishDelayLatency75pct = new ArrayList<>();
+ public List publishDelayLatency95pct = new ArrayList<>();
+ public List publishDelayLatency99pct = new ArrayList<>();
+ public List publishDelayLatency999pct = new ArrayList<>();
+ public List publishDelayLatency9999pct = new ArrayList<>();
+ public List publishDelayLatencyMax = new ArrayList<>();
+
+ public double aggregatedPublishLatencyAvg;
+ public double aggregatedPublishLatency50pct;
+ public double aggregatedPublishLatency75pct;
+ public double aggregatedPublishLatency95pct;
+ public double aggregatedPublishLatency99pct;
+ public double aggregatedPublishLatency999pct;
+ public double aggregatedPublishLatency9999pct;
+ public double aggregatedPublishLatencyMax;
+
+ public double aggregatedPublishDelayLatencyAvg;
+ public long aggregatedPublishDelayLatency50pct;
+ public long aggregatedPublishDelayLatency75pct;
+ public long aggregatedPublishDelayLatency95pct;
+ public long aggregatedPublishDelayLatency99pct;
+ public long aggregatedPublishDelayLatency999pct;
+ public long aggregatedPublishDelayLatency9999pct;
+ public long aggregatedPublishDelayLatencyMax;
+
+ public Map aggregatedPublishLatencyQuantiles = new TreeMap<>();
+
+ public Map aggregatedPublishDelayLatencyQuantiles = new TreeMap<>();
+
+ // End to end latencies (from producer to consumer)
+ // Latencies are expressed in milliseconds (without decimals)
+
+ public List endToEndLatencyAvg = new ArrayList<>();
+ public List endToEndLatency50pct = new ArrayList<>();
+ public List endToEndLatency75pct = new ArrayList<>();
+ public List endToEndLatency95pct = new ArrayList<>();
+ public List endToEndLatency99pct = new ArrayList<>();
+ public List endToEndLatency999pct = new ArrayList<>();
+ public List endToEndLatency9999pct = new ArrayList<>();
+ public List endToEndLatencyMax = new ArrayList<>();
+
+ public Map aggregatedEndToEndLatencyQuantiles = new TreeMap<>();
+
+ public double aggregatedEndToEndLatencyAvg;
+ public double aggregatedEndToEndLatency50pct;
+ public double aggregatedEndToEndLatency75pct;
+ public double aggregatedEndToEndLatency95pct;
+ public double aggregatedEndToEndLatency99pct;
+ public double aggregatedEndToEndLatency999pct;
+ public double aggregatedEndToEndLatency9999pct;
+ public double aggregatedEndToEndLatencyMax;
+
+ public int getTopics() {
+ return topics;
+ }
+
+ public int getPartitions() {
+ return partitions;
+ }
+
+ public long getMessageSize() {
+ return messageSize;
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/Workers.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/Workers.java
new file mode 100644
index 0000000..fbc16e6
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/Workers.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark;
+
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class Workers {
+ public List workers = new ArrayList<>();
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/Workload.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/Workload.java
new file mode 100644
index 0000000..584a606
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/Workload.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark;
+
+
+import io.openmessaging.benchmark.utils.distributor.KeyDistributorType;
+
+public class Workload {
+ public String name;
+
+ /** Number of topics to create in the test. */
+ public int topics;
+
+ /** Number of partitions each topic will contain. */
+ public int partitionsPerTopic;
+
+ public KeyDistributorType keyDistributor = KeyDistributorType.NO_KEY;
+
+ public int messageSize;
+
+ public boolean useRandomizedPayloads;
+ public double randomBytesRatio;
+ public int randomizedPayloadPoolSize;
+
+ public String payloadFile;
+
+ public int subscriptionsPerTopic;
+
+ public int producersPerTopic;
+
+ public int consumerPerSubscription;
+
+ public int producerRate;
+
+ /**
+ * If the consumer backlog is > 0, the generator will accumulate messages until the requested
+ * amount of storage is retained and then it will start the consumers to drain it.
+ *
+ *
The testDurationMinutes will be overruled to allow the test to complete when the consumer
+ * has drained all the backlog and it's on par with the producer
+ */
+ public long consumerBacklogSizeGB = 0;
+ /**
+ * The ratio of the backlog that can remain and yet the backlog still be considered empty, and
+ * thus the workload can complete at the end of the configured duration. In some systems it is not
+ * feasible for the backlog to be drained fully and thus the workload will run indefinitely. In
+ * such circumstances, one may be content to achieve a partial drain such as 99% of the backlog.
+ * The value should be on somewhere between 0.0 and 1.0, where 1.0 indicates that the backlog
+ * should be fully drained, and 0.0 indicates a best effort, where the workload will complete
+ * after the specified time irrespective of how much of the backlog has been drained.
+ */
+ public double backlogDrainRatio = 1.0;
+
+ public int testDurationMinutes;
+
+ public int warmupDurationMinutes = 1;
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/WorkloadGenerator.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/WorkloadGenerator.java
new file mode 100644
index 0000000..3025d17
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/WorkloadGenerator.java
@@ -0,0 +1,548 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark;
+
+import static java.util.concurrent.TimeUnit.MINUTES;
+
+import io.netty.util.concurrent.DefaultThreadFactory;
+import io.openmessaging.benchmark.utils.PaddingDecimalFormat;
+import io.openmessaging.benchmark.utils.RandomGenerator;
+import io.openmessaging.benchmark.utils.Timer;
+import io.openmessaging.benchmark.utils.payload.FilePayloadReader;
+import io.openmessaging.benchmark.utils.payload.PayloadReader;
+import io.openmessaging.benchmark.worker.Worker;
+import io.openmessaging.benchmark.worker.commands.ConsumerAssignment;
+import io.openmessaging.benchmark.worker.commands.CountersStats;
+import io.openmessaging.benchmark.worker.commands.CumulativeLatencies;
+import io.openmessaging.benchmark.worker.commands.PeriodStats;
+import io.openmessaging.benchmark.worker.commands.ProducerWorkAssignment;
+import io.openmessaging.benchmark.worker.commands.TopicSubscription;
+import io.openmessaging.benchmark.worker.commands.TopicsInfo;
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.lang3.ArrayUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class WorkloadGenerator implements AutoCloseable {
+
+ private final String driverName;
+ private final Workload workload;
+ private final Worker worker;
+
+ private final ExecutorService executor =
+ Executors.newCachedThreadPool(new DefaultThreadFactory("messaging-benchmark"));
+
+ private volatile boolean runCompleted = false;
+ private volatile boolean needToWaitForBacklogDraining = false;
+
+ private volatile double targetPublishRate;
+
+ public WorkloadGenerator(String driverName, Workload workload, Worker worker) {
+ this.driverName = driverName;
+ this.workload = workload;
+ this.worker = worker;
+
+ if (workload.consumerBacklogSizeGB > 0 && workload.producerRate == 0) {
+ throw new IllegalArgumentException(
+ "Cannot probe producer sustainable rate when building backlog");
+ }
+ }
+
+ public TestResult run() throws Exception {
+ Timer timer = new Timer();
+ List topics =
+ worker.createTopics(new TopicsInfo(workload.topics, workload.partitionsPerTopic));
+ log.info("Created {} topics in {} ms", topics.size(), timer.elapsedMillis());
+
+ createConsumers(topics);
+ createProducers(topics);
+
+ ensureTopicsAreReady();
+
+ if (workload.producerRate > 0) {
+ targetPublishRate = workload.producerRate;
+ } else {
+ // Producer rate is 0 and we need to discover the sustainable rate
+ targetPublishRate = 10000;
+
+ executor.execute(
+ () -> {
+ // Run background controller to adjust rate
+ try {
+ findMaximumSustainableRate(targetPublishRate);
+ } catch (IOException e) {
+ log.warn("Failure in finding max sustainable rate", e);
+ }
+ });
+ }
+
+ final PayloadReader payloadReader = new FilePayloadReader(workload.messageSize);
+
+ ProducerWorkAssignment producerWorkAssignment = new ProducerWorkAssignment();
+ producerWorkAssignment.keyDistributorType = workload.keyDistributor;
+ producerWorkAssignment.publishRate = targetPublishRate;
+ producerWorkAssignment.payloadData = new ArrayList<>();
+
+ if (workload.useRandomizedPayloads) {
+ // create messages that are part random and part zeros
+ // better for testing effects of compression
+ Random r = new Random();
+ int randomBytes = (int) (workload.messageSize * workload.randomBytesRatio);
+ int zerodBytes = workload.messageSize - randomBytes;
+ for (int i = 0; i < workload.randomizedPayloadPoolSize; i++) {
+ byte[] randArray = new byte[randomBytes];
+ r.nextBytes(randArray);
+ byte[] zerodArray = new byte[zerodBytes];
+ byte[] combined = ArrayUtils.addAll(randArray, zerodArray);
+ producerWorkAssignment.payloadData.add(combined);
+ }
+ } else {
+ producerWorkAssignment.payloadData.add(payloadReader.load(workload.payloadFile));
+ }
+
+ worker.startLoad(producerWorkAssignment);
+
+ if (workload.warmupDurationMinutes > 0) {
+ log.info("----- Starting warm-up traffic ({}m) ------", workload.warmupDurationMinutes);
+ printAndCollectStats(workload.warmupDurationMinutes, TimeUnit.MINUTES);
+ }
+
+ if (workload.consumerBacklogSizeGB > 0) {
+ executor.execute(
+ () -> {
+ try {
+ buildAndDrainBacklog(workload.testDurationMinutes);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ });
+ }
+
+ worker.resetStats();
+ log.info("----- Starting benchmark traffic ({}m)------", workload.testDurationMinutes);
+
+ TestResult result = printAndCollectStats(workload.testDurationMinutes, MINUTES);
+ runCompleted = true;
+
+ worker.stopAll();
+ return result;
+ }
+
+ private void ensureTopicsAreReady() throws IOException {
+ log.info("Waiting for consumers to be ready");
+ // This is work around the fact that there's no way to have a consumer ready in Kafka
+ // without
+ // first publishing
+ // some message on the topic, which will then trigger the partitions assignment to the
+ // consumers
+
+ int expectedMessages = workload.topics * workload.subscriptionsPerTopic;
+
+ // In this case we just publish 1 message and then wait for consumers to receive the data
+ worker.probeProducers();
+
+ long start = System.currentTimeMillis();
+ long end = start + 60 * 1000;
+ while (System.currentTimeMillis() < end) {
+ CountersStats stats = worker.getCountersStats();
+
+ log.info(
+ "Waiting for topics to be ready -- Sent: {}, Received: {}",
+ stats.messagesSent,
+ stats.messagesReceived);
+ if (stats.messagesReceived < expectedMessages) {
+ try {
+ Thread.sleep(2_000);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ } else {
+ break;
+ }
+ }
+
+ if (System.currentTimeMillis() >= end) {
+ throw new RuntimeException("Timed out waiting for consumers to be ready");
+ } else {
+ log.info("All consumers are ready");
+ }
+ }
+
+ /**
+ * Adjust the publish rate to a level that is sustainable, meaning that we can consume all the
+ * messages that are being produced.
+ *
+ * @param currentRate
+ */
+ private void findMaximumSustainableRate(double currentRate) throws IOException {
+ CountersStats stats = worker.getCountersStats();
+
+ int controlPeriodMillis = 3000;
+ long lastControlTimestamp = System.nanoTime();
+
+ RateController rateController = new RateController();
+
+ while (!runCompleted) {
+ // Check every few seconds and adjust the rate
+ try {
+ Thread.sleep(controlPeriodMillis);
+ } catch (InterruptedException e) {
+ return;
+ }
+
+ // Consider multiple copies when using multiple subscriptions
+ stats = worker.getCountersStats();
+ long currentTime = System.nanoTime();
+ long periodNanos = currentTime - lastControlTimestamp;
+
+ lastControlTimestamp = currentTime;
+
+ currentRate =
+ rateController.nextRate(
+ currentRate, periodNanos, stats.messagesSent, stats.messagesReceived);
+ worker.adjustPublishRate(currentRate);
+ }
+ }
+
+ @Override
+ public void close() throws Exception {
+ worker.stopAll();
+ executor.shutdownNow();
+ }
+
+ private void createConsumers(List topics) throws IOException {
+ ConsumerAssignment consumerAssignment = new ConsumerAssignment();
+
+ for (String topic : topics) {
+ for (int i = 0; i < workload.subscriptionsPerTopic; i++) {
+ String subscriptionName =
+ String.format("sub-%03d-%s", i, RandomGenerator.genRandomBinaryString());
+ for (int j = 0; j < workload.consumerPerSubscription; j++) {
+ consumerAssignment.topicsSubscriptions.add(
+ new TopicSubscription(
+ topic,
+ j,
+ subscriptionName,
+ workload.partitionsPerTopic,
+ workload.consumerPerSubscription));
+ }
+ }
+ }
+
+ Collections.shuffle(consumerAssignment.topicsSubscriptions);
+
+ Timer timer = new Timer();
+
+ worker.createConsumers(consumerAssignment);
+ log.info(
+ "Created {} consumers in {} ms",
+ consumerAssignment.topicsSubscriptions.size(),
+ timer.elapsedMillis());
+ }
+
+ private void createProducers(List topics) throws IOException {
+ List fullListOfTopics = new ArrayList<>();
+
+ // Add the topic multiple times, one for each producer
+ for (int i = 0; i < workload.producersPerTopic; i++) {
+ fullListOfTopics.addAll(topics);
+ }
+
+ Collections.shuffle(fullListOfTopics);
+
+ Timer timer = new Timer();
+
+ worker.createProducers(fullListOfTopics);
+ log.info("Created {} producers in {} ms", fullListOfTopics.size(), timer.elapsedMillis());
+ }
+
+ private void buildAndDrainBacklog(int testDurationMinutes) throws IOException {
+ Timer timer = new Timer();
+ log.info("Stopping all consumers to build backlog");
+ worker.pauseConsumers();
+
+ this.needToWaitForBacklogDraining = true;
+
+ long requestedBacklogSize = workload.consumerBacklogSizeGB * 1024 * 1024 * 1024;
+
+ while (true) {
+ CountersStats stats = worker.getCountersStats();
+ long currentBacklogSize =
+ (workload.subscriptionsPerTopic * stats.messagesSent - stats.messagesReceived)
+ * workload.messageSize;
+
+ if (currentBacklogSize >= requestedBacklogSize) {
+ break;
+ }
+
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ log.info("--- Completed backlog build in {} s ---", timer.elapsedSeconds());
+ timer = new Timer();
+ log.info("--- Start draining backlog ---");
+
+ worker.resumeConsumers();
+
+ long backlogMessageCapacity = requestedBacklogSize / workload.messageSize;
+ long backlogEmptyLevel = (long) ((1.0 - workload.backlogDrainRatio) * backlogMessageCapacity);
+ final long minBacklog = Math.max(1000L, backlogEmptyLevel);
+
+ while (true) {
+ CountersStats stats = worker.getCountersStats();
+ long currentBacklog =
+ workload.subscriptionsPerTopic * stats.messagesSent - stats.messagesReceived;
+ if (currentBacklog <= minBacklog) {
+ log.info("--- Completed backlog draining in {} s ---", timer.elapsedSeconds());
+
+ try {
+ Thread.sleep(MINUTES.toMillis(testDurationMinutes));
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+
+ needToWaitForBacklogDraining = false;
+ return;
+ }
+
+ try {
+ Thread.sleep(100);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ }
+
+ @SuppressWarnings({"checkstyle:LineLength", "checkstyle:MethodLength"})
+ private TestResult printAndCollectStats(long testDurations, TimeUnit unit) throws IOException {
+ long startTime = System.nanoTime();
+
+ // Print report stats
+ long oldTime = System.nanoTime();
+
+ long testEndTime = testDurations > 0 ? startTime + unit.toNanos(testDurations) : Long.MAX_VALUE;
+
+ TestResult result = new TestResult();
+ result.workload = workload.name;
+ result.driver = driverName;
+ result.topics = workload.topics;
+ result.partitions = workload.partitionsPerTopic;
+ result.messageSize = workload.messageSize;
+ result.producersPerTopic = workload.producersPerTopic;
+ result.consumersPerTopic = workload.consumerPerSubscription;
+
+ while (true) {
+ try {
+ Thread.sleep(10000);
+ } catch (InterruptedException e) {
+ break;
+ }
+
+ PeriodStats stats = worker.getPeriodStats();
+
+ long now = System.nanoTime();
+ double elapsed = (now - oldTime) / 1e9;
+
+ double publishRate = stats.messagesSent / elapsed;
+ double publishThroughput = stats.bytesSent / elapsed / 1024 / 1024;
+ double errorRate = stats.messageSendErrors / elapsed;
+
+ double consumeRate = stats.messagesReceived / elapsed;
+ double consumeThroughput = stats.bytesReceived / elapsed / 1024 / 1024;
+
+ long currentBacklog =
+ Math.max(
+ 0L,
+ workload.subscriptionsPerTopic * stats.totalMessagesSent
+ - stats.totalMessagesReceived);
+
+ log.info(
+ "Pub rate {} msg/s / {} MB/s | Pub err {} err/s | Cons rate {} msg/s / {} MB/s | Backlog: {} K | Pub Latency (ms) avg: {} - 50%: {} - 99%: {} - 99.9%: {} - Max: {} | Pub Delay Latency (us) avg: {} - 50%: {} - 99%: {} - 99.9%: {} - Max: {}",
+ rateFormat.format(publishRate),
+ throughputFormat.format(publishThroughput),
+ rateFormat.format(errorRate),
+ rateFormat.format(consumeRate),
+ throughputFormat.format(consumeThroughput),
+ dec.format(currentBacklog / 1000.0), //
+ dec.format(microsToMillis(stats.publishLatency.getMean())),
+ dec.format(microsToMillis(stats.publishLatency.getValueAtPercentile(50))),
+ dec.format(microsToMillis(stats.publishLatency.getValueAtPercentile(99))),
+ dec.format(microsToMillis(stats.publishLatency.getValueAtPercentile(99.9))),
+ throughputFormat.format(microsToMillis(stats.publishLatency.getMaxValue())),
+ dec.format(stats.publishDelayLatency.getMean()),
+ dec.format(stats.publishDelayLatency.getValueAtPercentile(50)),
+ dec.format(stats.publishDelayLatency.getValueAtPercentile(99)),
+ dec.format(stats.publishDelayLatency.getValueAtPercentile(99.9)),
+ throughputFormat.format(stats.publishDelayLatency.getMaxValue()));
+
+ result.publishRate.add(publishRate);
+ result.publishThroughputMBps.add(publishThroughput);
+ result.publishErrorRate.add(errorRate);
+ result.consumeRate.add(consumeRate);
+ result.consumeThroughputMBps.add(consumeThroughput);
+ result.backlog.add(currentBacklog);
+ result.publishLatencyAvg.add(microsToMillis(stats.publishLatency.getMean()));
+ result.publishLatency50pct.add(microsToMillis(stats.publishLatency.getValueAtPercentile(50)));
+ result.publishLatency75pct.add(microsToMillis(stats.publishLatency.getValueAtPercentile(75)));
+ result.publishLatency95pct.add(microsToMillis(stats.publishLatency.getValueAtPercentile(95)));
+ result.publishLatency99pct.add(microsToMillis(stats.publishLatency.getValueAtPercentile(99)));
+ result.publishLatency999pct.add(
+ microsToMillis(stats.publishLatency.getValueAtPercentile(99.9)));
+ result.publishLatency9999pct.add(
+ microsToMillis(stats.publishLatency.getValueAtPercentile(99.99)));
+ result.publishLatencyMax.add(microsToMillis(stats.publishLatency.getMaxValue()));
+
+ result.publishDelayLatencyAvg.add(stats.publishDelayLatency.getMean());
+ result.publishDelayLatency50pct.add(stats.publishDelayLatency.getValueAtPercentile(50));
+ result.publishDelayLatency75pct.add(stats.publishDelayLatency.getValueAtPercentile(75));
+ result.publishDelayLatency95pct.add(stats.publishDelayLatency.getValueAtPercentile(95));
+ result.publishDelayLatency99pct.add(stats.publishDelayLatency.getValueAtPercentile(99));
+ result.publishDelayLatency999pct.add(stats.publishDelayLatency.getValueAtPercentile(99.9));
+ result.publishDelayLatency9999pct.add(stats.publishDelayLatency.getValueAtPercentile(99.99));
+ result.publishDelayLatencyMax.add(stats.publishDelayLatency.getMaxValue());
+
+ result.endToEndLatencyAvg.add(microsToMillis(stats.endToEndLatency.getMean()));
+ result.endToEndLatency50pct.add(
+ microsToMillis(stats.endToEndLatency.getValueAtPercentile(50)));
+ result.endToEndLatency75pct.add(
+ microsToMillis(stats.endToEndLatency.getValueAtPercentile(75)));
+ result.endToEndLatency95pct.add(
+ microsToMillis(stats.endToEndLatency.getValueAtPercentile(95)));
+ result.endToEndLatency99pct.add(
+ microsToMillis(stats.endToEndLatency.getValueAtPercentile(99)));
+ result.endToEndLatency999pct.add(
+ microsToMillis(stats.endToEndLatency.getValueAtPercentile(99.9)));
+ result.endToEndLatency9999pct.add(
+ microsToMillis(stats.endToEndLatency.getValueAtPercentile(99.99)));
+ result.endToEndLatencyMax.add(microsToMillis(stats.endToEndLatency.getMaxValue()));
+
+ if (now >= testEndTime && !needToWaitForBacklogDraining) {
+ CumulativeLatencies agg = worker.getCumulativeLatencies();
+ log.info(
+ "----- Aggregated Pub Latency (ms) avg: {} - 50%: {} - 95%: {} - 99%: {} - 99.9%: {} - 99.99%: {} - Max: {} | Pub Delay (us) avg: {} - 50%: {} - 95%: {} - 99%: {} - 99.9%: {} - 99.99%: {} - Max: {}",
+ dec.format(agg.publishLatency.getMean() / 1000.0),
+ dec.format(agg.publishLatency.getValueAtPercentile(50) / 1000.0),
+ dec.format(agg.publishLatency.getValueAtPercentile(95) / 1000.0),
+ dec.format(agg.publishLatency.getValueAtPercentile(99) / 1000.0),
+ dec.format(agg.publishLatency.getValueAtPercentile(99.9) / 1000.0),
+ dec.format(agg.publishLatency.getValueAtPercentile(99.99) / 1000.0),
+ throughputFormat.format(agg.publishLatency.getMaxValue() / 1000.0),
+ dec.format(agg.publishDelayLatency.getMean()),
+ dec.format(agg.publishDelayLatency.getValueAtPercentile(50)),
+ dec.format(agg.publishDelayLatency.getValueAtPercentile(95)),
+ dec.format(agg.publishDelayLatency.getValueAtPercentile(99)),
+ dec.format(agg.publishDelayLatency.getValueAtPercentile(99.9)),
+ dec.format(agg.publishDelayLatency.getValueAtPercentile(99.99)),
+ throughputFormat.format(agg.publishDelayLatency.getMaxValue()));
+
+ result.aggregatedPublishLatencyAvg = agg.publishLatency.getMean() / 1000.0;
+ result.aggregatedPublishLatency50pct = agg.publishLatency.getValueAtPercentile(50) / 1000.0;
+ result.aggregatedPublishLatency75pct = agg.publishLatency.getValueAtPercentile(75) / 1000.0;
+ result.aggregatedPublishLatency95pct = agg.publishLatency.getValueAtPercentile(95) / 1000.0;
+ result.aggregatedPublishLatency99pct = agg.publishLatency.getValueAtPercentile(99) / 1000.0;
+ result.aggregatedPublishLatency999pct =
+ agg.publishLatency.getValueAtPercentile(99.9) / 1000.0;
+ result.aggregatedPublishLatency9999pct =
+ agg.publishLatency.getValueAtPercentile(99.99) / 1000.0;
+ result.aggregatedPublishLatencyMax = agg.publishLatency.getMaxValue() / 1000.0;
+
+ result.aggregatedPublishDelayLatencyAvg = agg.publishDelayLatency.getMean();
+ result.aggregatedPublishDelayLatency50pct =
+ agg.publishDelayLatency.getValueAtPercentile(50);
+ result.aggregatedPublishDelayLatency75pct =
+ agg.publishDelayLatency.getValueAtPercentile(75);
+ result.aggregatedPublishDelayLatency95pct =
+ agg.publishDelayLatency.getValueAtPercentile(95);
+ result.aggregatedPublishDelayLatency99pct =
+ agg.publishDelayLatency.getValueAtPercentile(99);
+ result.aggregatedPublishDelayLatency999pct =
+ agg.publishDelayLatency.getValueAtPercentile(99.9);
+ result.aggregatedPublishDelayLatency9999pct =
+ agg.publishDelayLatency.getValueAtPercentile(99.99);
+ result.aggregatedPublishDelayLatencyMax = agg.publishDelayLatency.getMaxValue();
+
+ result.aggregatedEndToEndLatencyAvg = agg.endToEndLatency.getMean() / 1000.0;
+ result.aggregatedEndToEndLatency50pct =
+ agg.endToEndLatency.getValueAtPercentile(50) / 1000.0;
+ result.aggregatedEndToEndLatency75pct =
+ agg.endToEndLatency.getValueAtPercentile(75) / 1000.0;
+ result.aggregatedEndToEndLatency95pct =
+ agg.endToEndLatency.getValueAtPercentile(95) / 1000.0;
+ result.aggregatedEndToEndLatency99pct =
+ agg.endToEndLatency.getValueAtPercentile(99) / 1000.0;
+ result.aggregatedEndToEndLatency999pct =
+ agg.endToEndLatency.getValueAtPercentile(99.9) / 1000.0;
+ result.aggregatedEndToEndLatency9999pct =
+ agg.endToEndLatency.getValueAtPercentile(99.99) / 1000.0;
+ result.aggregatedEndToEndLatencyMax = agg.endToEndLatency.getMaxValue() / 1000.0;
+
+ agg.publishLatency
+ .percentiles(100)
+ .forEach(
+ value -> {
+ result.aggregatedPublishLatencyQuantiles.put(
+ value.getPercentile(), value.getValueIteratedTo() / 1000.0);
+ });
+
+ agg.publishDelayLatency
+ .percentiles(100)
+ .forEach(
+ value -> {
+ result.aggregatedPublishDelayLatencyQuantiles.put(
+ value.getPercentile(), value.getValueIteratedTo());
+ });
+
+ agg.endToEndLatency
+ .percentiles(100)
+ .forEach(
+ value -> {
+ result.aggregatedEndToEndLatencyQuantiles.put(
+ value.getPercentile(), microsToMillis(value.getValueIteratedTo()));
+ });
+
+ break;
+ }
+
+ oldTime = now;
+ }
+
+ return result;
+ }
+
+ private static final DecimalFormat rateFormat = new PaddingDecimalFormat("0.0", 7);
+ private static final DecimalFormat throughputFormat = new PaddingDecimalFormat("0.0", 4);
+ private static final DecimalFormat dec = new PaddingDecimalFormat("0.0", 4);
+
+ private static double microsToMillis(double timeInMicros) {
+ return timeInMicros / 1000.0;
+ }
+
+ private static double microsToMillis(long timeInMicros) {
+ return timeInMicros / 1000.0;
+ }
+
+ private static final Logger log = LoggerFactory.getLogger(WorkloadGenerator.class);
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/Env.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/Env.java
new file mode 100644
index 0000000..366b2ca
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/Env.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.utils;
+
+
+import java.util.Optional;
+import java.util.function.Function;
+
+public final class Env {
+ private Env() {}
+
+ public static long getLong(String key, long defaultValue) {
+ return get(key, Long::parseLong, defaultValue);
+ }
+
+ public static double getDouble(String key, double defaultValue) {
+ return get(key, Double::parseDouble, defaultValue);
+ }
+
+ public static T get(String key, Function function, T defaultValue) {
+ return Optional.ofNullable(System.getenv(key)).map(function).orElse(defaultValue);
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/ListPartition.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/ListPartition.java
new file mode 100644
index 0000000..01a1ae5
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/ListPartition.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.utils;
+
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class ListPartition {
+
+ /**
+ * partition a list to specified size.
+ *
+ * @param originList
+ * @param size
+ * @param
+ * @return the partitioned list
+ */
+ public static List> partitionList(List originList, int size) {
+
+ List> resultList = new ArrayList<>();
+ if (null == originList || 0 == originList.size() || size <= 0) {
+ return resultList;
+ }
+ if (originList.size() <= size) {
+ for (T item : originList) {
+ List resultItemList = new ArrayList<>();
+ resultItemList.add(item);
+ resultList.add(resultItemList);
+ }
+ for (int i = 0; i < (size - originList.size()); i++) {
+ resultList.add(new ArrayList<>());
+ }
+ return resultList;
+ }
+
+ for (int i = 0; i < size; i++) {
+ resultList.add(new ArrayList<>());
+ }
+ int count = 0;
+ for (T item : originList) {
+ int index = count % size;
+ resultList.get(index).add(item);
+ count++;
+ }
+ return resultList;
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/PaddingDecimalFormat.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/PaddingDecimalFormat.java
new file mode 100644
index 0000000..a598ddb
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/PaddingDecimalFormat.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.utils;
+
+
+import java.text.DecimalFormat;
+import java.text.DecimalFormatSymbols;
+import java.text.FieldPosition;
+
+@SuppressWarnings("serial")
+public class PaddingDecimalFormat extends DecimalFormat {
+ private int minimumLength;
+
+ /**
+ * Creates a PaddingDecimalFormat using the given pattern and minimum minimumLength and the
+ * symbols for the default locale.
+ *
+ * @param pattern
+ * @param minLength
+ */
+ public PaddingDecimalFormat(String pattern, int minLength) {
+ super(pattern);
+ minimumLength = minLength;
+ }
+
+ /**
+ * Creates a PaddingDecimalFormat using the given pattern, symbols and minimum minimumLength.
+ *
+ * @param pattern
+ * @param symbols
+ * @param minLength
+ */
+ public PaddingDecimalFormat(String pattern, DecimalFormatSymbols symbols, int minLength) {
+ super(pattern, symbols);
+ minimumLength = minLength;
+ }
+
+ @Override
+ public StringBuffer format(double number, StringBuffer toAppendTo, FieldPosition pos) {
+ int initLength = toAppendTo.length();
+ super.format(number, toAppendTo, pos);
+ return pad(toAppendTo, initLength);
+ }
+
+ @Override
+ public StringBuffer format(long number, StringBuffer toAppendTo, FieldPosition pos) {
+ int initLength = toAppendTo.length();
+ super.format(number, toAppendTo, pos);
+ return pad(toAppendTo, initLength);
+ }
+
+ private StringBuffer pad(StringBuffer toAppendTo, int initLength) {
+ int numLength = toAppendTo.length() - initLength;
+ int padLength = minimumLength - numLength;
+ if (padLength > 0) {
+ StringBuffer pad = new StringBuffer(padLength);
+ for (int i = 0; i < padLength; i++) {
+ pad.append(' ');
+ }
+ toAppendTo.insert(initLength, pad);
+ }
+ return toAppendTo;
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/RandomGenerator.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/RandomGenerator.java
new file mode 100644
index 0000000..f6a3424
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/RandomGenerator.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.utils;
+
+
+import com.google.common.io.BaseEncoding;
+import java.util.Random;
+
+public class RandomGenerator {
+
+ private static final Random random = new Random();
+
+ public static final String genRandomBinaryString() {
+ byte[] buffer = new byte[5];
+ random.nextBytes(buffer);
+ return BaseEncoding.base64Url().omitPadding().encode(buffer);
+ }
+
+ public static byte[] genRandomBinaryString(int bytesSize) {
+ byte[] buffer = new byte[bytesSize];
+ random.nextBytes(buffer);
+ return buffer;
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/Timer.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/Timer.java
new file mode 100644
index 0000000..bb3ac28
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/Timer.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.utils;
+
+
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+public class Timer {
+ private final long startTime;
+ private final Supplier nanoClock;
+
+ Timer(Supplier nanoClock) {
+ this.nanoClock = nanoClock;
+ startTime = this.nanoClock.get();
+ }
+
+ public Timer() {
+ this(System::nanoTime);
+ }
+
+ public double elapsedMillis() {
+ return elapsed(TimeUnit.MILLISECONDS);
+ }
+
+ public double elapsedSeconds() {
+ return elapsed(TimeUnit.SECONDS);
+ }
+
+ private double elapsed(TimeUnit unit) {
+ long now = nanoClock.get();
+ return (now - startTime) / (double) unit.toNanos(1);
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/UniformRateLimiter.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/UniformRateLimiter.java
new file mode 100644
index 0000000..1fc69b1
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/UniformRateLimiter.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.utils;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+import java.util.concurrent.atomic.AtomicLongFieldUpdater;
+import java.util.concurrent.locks.LockSupport;
+import java.util.function.Supplier;
+
+/**
+ * Provides a next operation time for rate limited operation streams.
+ * The rate limiter is thread safe and can be shared by all threads.
+ */
+public final class UniformRateLimiter {
+
+ private static final AtomicLongFieldUpdater V_TIME_UPDATER =
+ AtomicLongFieldUpdater.newUpdater(UniformRateLimiter.class, "virtualTime");
+ private static final AtomicLongFieldUpdater START_UPDATER =
+ AtomicLongFieldUpdater.newUpdater(UniformRateLimiter.class, "start");
+ private static final double ONE_SEC_IN_NS = SECONDS.toNanos(1);
+ private volatile long start = Long.MIN_VALUE;
+ private volatile long virtualTime;
+ private final double opsPerSec;
+ private final long intervalNs;
+ private final Supplier nanoClock;
+
+ UniformRateLimiter(final double opsPerSec, Supplier nanoClock) {
+ if (Double.isNaN(opsPerSec) || Double.isInfinite(opsPerSec)) {
+ throw new IllegalArgumentException("opsPerSec cannot be Nan or Infinite");
+ }
+ if (opsPerSec <= 0) {
+ throw new IllegalArgumentException("opsPerSec must be greater then 0");
+ }
+ this.opsPerSec = opsPerSec;
+ intervalNs = Math.round(ONE_SEC_IN_NS / opsPerSec);
+ this.nanoClock = nanoClock;
+ }
+
+ public UniformRateLimiter(final double opsPerSec) {
+ this(opsPerSec, System::nanoTime);
+ }
+
+ public double getOpsPerSec() {
+ return opsPerSec;
+ }
+
+ public long getIntervalNs() {
+ return intervalNs;
+ }
+
+ public long acquire() {
+ final long currOpIndex = V_TIME_UPDATER.getAndIncrement(this);
+ long start = this.start;
+ if (start == Long.MIN_VALUE) {
+ start = nanoClock.get();
+ if (!START_UPDATER.compareAndSet(this, Long.MIN_VALUE, start)) {
+ start = this.start;
+ assert start != Long.MIN_VALUE;
+ }
+ }
+ return start + currOpIndex * intervalNs;
+ }
+
+ public static void uninterruptibleSleepNs(final long intendedTime) {
+ long sleepNs;
+ while ((sleepNs = (intendedTime - System.nanoTime())) > 0) {
+ LockSupport.parkNanos(sleepNs);
+ }
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/distributor/KeyDistributor.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/distributor/KeyDistributor.java
new file mode 100644
index 0000000..14576c9
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/distributor/KeyDistributor.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.utils.distributor;
+
+
+import com.google.common.io.BaseEncoding;
+import java.util.Random;
+
+public abstract class KeyDistributor {
+
+ private static final int UNIQUE_COUNT = 10_000;
+ private static final int KEY_BYTE_SIZE = 7;
+
+ private static final String[] randomKeys = new String[UNIQUE_COUNT];
+
+ static {
+ // Generate a number of random keys to be used when publishing
+ byte[] buffer = new byte[KEY_BYTE_SIZE];
+ Random random = new Random();
+ for (int i = 0; i < randomKeys.length; i++) {
+ random.nextBytes(buffer);
+ randomKeys[i] = BaseEncoding.base64Url().omitPadding().encode(buffer);
+ }
+ }
+
+ protected String get(int index) {
+ return randomKeys[index];
+ }
+
+ protected int getLength() {
+ return UNIQUE_COUNT;
+ }
+
+ public abstract String next();
+
+ public static KeyDistributor build(KeyDistributorType keyType) {
+ KeyDistributor keyDistributor = null;
+ switch (keyType) {
+ case NO_KEY:
+ keyDistributor = new NoKeyDistributor();
+ break;
+ case KEY_ROUND_ROBIN:
+ keyDistributor = new KeyRoundRobin();
+ break;
+ case RANDOM_NANO:
+ keyDistributor = new RandomNano();
+ break;
+ default:
+ throw new IllegalStateException("Unexpected KeyDistributorType: " + keyType);
+ }
+ return keyDistributor;
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/distributor/KeyDistributorType.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/distributor/KeyDistributorType.java
new file mode 100644
index 0000000..3cb1cf1
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/distributor/KeyDistributorType.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.utils.distributor;
+
+
+import com.fasterxml.jackson.annotation.JsonEnumDefaultValue;
+
+public enum KeyDistributorType {
+ /** Key distributor that returns null keys to have default publish semantics. */
+ @JsonEnumDefaultValue
+ NO_KEY,
+
+ /** Genarate a finite number of "keys" and cycle through them in round-robin fashion. */
+ KEY_ROUND_ROBIN,
+
+ /** Random distribution based on System.nanoTime(). */
+ RANDOM_NANO,
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/distributor/KeyRoundRobin.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/distributor/KeyRoundRobin.java
new file mode 100644
index 0000000..bb600e2
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/distributor/KeyRoundRobin.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.utils.distributor;
+
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+@NotThreadSafe
+public class KeyRoundRobin extends KeyDistributor {
+
+ private int currentIndex = 0;
+
+ @Override
+ public String next() {
+ if (++currentIndex >= getLength()) {
+ currentIndex = 0;
+ }
+ return get(currentIndex);
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/distributor/NoKeyDistributor.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/distributor/NoKeyDistributor.java
new file mode 100644
index 0000000..593f0e5
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/distributor/NoKeyDistributor.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.utils.distributor;
+
+public class NoKeyDistributor extends KeyDistributor {
+
+ @Override
+ public String next() {
+ return null;
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/distributor/RandomNano.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/distributor/RandomNano.java
new file mode 100644
index 0000000..cedbf1b
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/distributor/RandomNano.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.utils.distributor;
+
+
+import javax.annotation.concurrent.ThreadSafe;
+
+@ThreadSafe
+public class RandomNano extends KeyDistributor {
+
+ public String next() {
+ int randomIndex = Math.abs((int) System.nanoTime() % getLength());
+ return get(randomIndex);
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/payload/FilePayloadReader.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/payload/FilePayloadReader.java
new file mode 100644
index 0000000..2fe60e0
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/payload/FilePayloadReader.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.utils.payload;
+
+import static java.nio.file.Files.readAllBytes;
+
+import java.io.File;
+import java.io.IOException;
+import java.text.MessageFormat;
+
+public class FilePayloadReader implements PayloadReader {
+
+ private final int expectedLength;
+
+ public FilePayloadReader(int expectedLength) {
+ this.expectedLength = expectedLength;
+ }
+
+ @Override
+ public byte[] load(String resourceName) {
+ byte[] payload;
+ try {
+ payload = readAllBytes(new File(resourceName).toPath());
+ checkPayloadLength(payload);
+ return payload;
+ } catch (IOException e) {
+ throw new PayloadException(e.getMessage());
+ }
+ }
+
+ private void checkPayloadLength(byte[] payload) {
+ if (expectedLength != payload.length) {
+ throw new PayloadException(
+ MessageFormat.format(
+ "Payload length mismatch. Actual is: {0}, but expected: {1} ",
+ payload.length, expectedLength));
+ }
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/payload/PayloadException.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/payload/PayloadException.java
new file mode 100644
index 0000000..a3887ac
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/payload/PayloadException.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.utils.payload;
+
+public class PayloadException extends RuntimeException {
+
+ public PayloadException(String message) {
+ super(message);
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/payload/PayloadReader.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/payload/PayloadReader.java
new file mode 100644
index 0000000..97d82aa
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/payload/PayloadReader.java
@@ -0,0 +1,19 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.utils.payload;
+
+public interface PayloadReader {
+
+ byte[] load(String resourceName);
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/BenchmarkWorker.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/BenchmarkWorker.java
new file mode 100644
index 0000000..8723384
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/BenchmarkWorker.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker;
+
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParameterException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import io.javalin.Javalin;
+import org.apache.bookkeeper.stats.Stats;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider;
+import org.apache.commons.configuration.CompositeConfiguration;
+import org.apache.commons.configuration.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** A benchmark worker that listen for tasks to perform. */
+public class BenchmarkWorker {
+
+ static class Arguments {
+
+ @Parameter(
+ names = {"-h", "--help"},
+ description = "Help message",
+ help = true)
+ boolean help;
+
+ @Parameter(
+ names = {"-p", "--port"},
+ description = "HTTP port to listen on")
+ public int httpPort = 8080;
+
+ @Parameter(
+ names = {"-sp", "--stats-port"},
+ description = "Stats port to listen on")
+ public int statsPort = 8081;
+ }
+
+ public static void main(String[] args) throws Exception {
+ final Arguments arguments = new Arguments();
+ JCommander jc = new JCommander(arguments);
+ jc.setProgramName("benchmark-worker");
+
+ try {
+ jc.parse(args);
+ } catch (ParameterException e) {
+ System.err.println(e.getMessage());
+ jc.usage();
+ System.exit(-1);
+ }
+
+ if (arguments.help) {
+ jc.usage();
+ System.exit(-1);
+ }
+
+ Configuration conf = new CompositeConfiguration();
+ conf.setProperty(Stats.STATS_PROVIDER_CLASS, PrometheusMetricsProvider.class.getName());
+ conf.setProperty("prometheusStatsHttpPort", arguments.statsPort);
+ Stats.loadStatsProvider(conf);
+ StatsProvider provider = Stats.get();
+ provider.start(conf);
+
+ Runtime.getRuntime()
+ .addShutdownHook(new Thread(() -> provider.stop(), "benchmark-worker-shutdown-thread"));
+
+ // Dump configuration variables
+ log.info("Starting benchmark with config: {}", writer.writeValueAsString(arguments));
+
+ // Start web server
+ Javalin app = Javalin.start(arguments.httpPort);
+
+ new WorkerHandler(app, provider.getStatsLogger("benchmark"));
+ }
+
+ private static final ObjectWriter writer = new ObjectMapper().writerWithDefaultPrettyPrinter();
+
+ private static final Logger log = LoggerFactory.getLogger(BenchmarkWorker.class);
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/DistributedWorkersEnsemble.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/DistributedWorkersEnsemble.java
new file mode 100644
index 0000000..9e8d15f
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/DistributedWorkersEnsemble.java
@@ -0,0 +1,319 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker;
+
+import static java.util.Collections.unmodifiableList;
+import static java.util.stream.Collectors.joining;
+
+import com.beust.jcommander.internal.Maps;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import io.openmessaging.benchmark.utils.ListPartition;
+import io.openmessaging.benchmark.worker.commands.ConsumerAssignment;
+import io.openmessaging.benchmark.worker.commands.CountersStats;
+import io.openmessaging.benchmark.worker.commands.CumulativeLatencies;
+import io.openmessaging.benchmark.worker.commands.PeriodStats;
+import io.openmessaging.benchmark.worker.commands.ProducerWorkAssignment;
+import io.openmessaging.benchmark.worker.commands.TopicSubscription;
+import io.openmessaging.benchmark.worker.commands.TopicsInfo;
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DistributedWorkersEnsemble implements Worker {
+ private final Thread shutdownHook = new Thread(this::stopAll);
+ private final List workers;
+ private final List producerWorkers;
+ private final List consumerWorkers;
+ private final Worker leader;
+
+ private int numberOfUsedProducerWorkers;
+
+ public DistributedWorkersEnsemble(List workers, boolean extraConsumerWorkers) {
+ Preconditions.checkArgument(workers.size() > 1);
+ this.workers = unmodifiableList(workers);
+ leader = workers.get(0);
+ int numberOfProducerWorkers = getNumberOfProducerWorkers(workers, extraConsumerWorkers);
+ List> partitions =
+ Lists.partition(Lists.reverse(workers), workers.size() - numberOfProducerWorkers);
+ this.producerWorkers = partitions.get(1);
+ this.consumerWorkers = partitions.get(0);
+
+ log.info(
+ "Workers list - producers: [{}]",
+ producerWorkers.stream().map(Worker::id).collect(joining(",")));
+ log.info(
+ "Workers list - consumers: {}",
+ consumerWorkers.stream().map(Worker::id).collect(joining(",")));
+
+ Runtime.getRuntime().addShutdownHook(shutdownHook);
+ }
+
+ /*
+ * For driver-jms extra consumers are required. If there is an odd number of workers then allocate the extra
+ * to consumption.
+ */
+ @VisibleForTesting
+ static int getNumberOfProducerWorkers(List workers, boolean extraConsumerWorkers) {
+ return extraConsumerWorkers ? (workers.size() + 2) / 3 : workers.size() / 2;
+ }
+
+ @Override
+ public void initializeDriver(File configurationFile) throws IOException {
+ workers.parallelStream()
+ .forEach(
+ w -> {
+ try {
+ w.initializeDriver(configurationFile);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ });
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public List createTopics(TopicsInfo topicsInfo) throws IOException {
+ return leader.createTopics(topicsInfo);
+ }
+
+ @Override
+ public void createProducers(List topics) {
+ List> topicsPerProducer =
+ ListPartition.partitionList(topics, producerWorkers.size());
+ Map> topicsPerProducerMap = Maps.newHashMap();
+ int i = 0;
+ for (List assignedTopics : topicsPerProducer) {
+ topicsPerProducerMap.put(producerWorkers.get(i++), assignedTopics);
+ }
+
+ // Number of actually used workers might be less than available workers
+ numberOfUsedProducerWorkers =
+ (int) topicsPerProducerMap.values().stream().filter(t -> !t.isEmpty()).count();
+ log.debug(
+ "Producing worker count: {} of {}", numberOfUsedProducerWorkers, producerWorkers.size());
+ topicsPerProducerMap.entrySet().parallelStream()
+ .forEach(
+ e -> {
+ try {
+ e.getKey().createProducers(e.getValue());
+ } catch (IOException ex) {
+ throw new RuntimeException(ex);
+ }
+ });
+ }
+
+ @Override
+ public void startLoad(ProducerWorkAssignment producerWorkAssignment) throws IOException {
+ // Reduce the publish rate across all the brokers
+ double newRate = producerWorkAssignment.publishRate / numberOfUsedProducerWorkers;
+ log.debug("Setting worker assigned publish rate to {} msgs/sec", newRate);
+ // Reduce the publish rate across all the brokers
+ producerWorkers.parallelStream()
+ .forEach(
+ w -> {
+ try {
+ w.startLoad(producerWorkAssignment.withPublishRate(newRate));
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ });
+ }
+
+ @Override
+ public void probeProducers() throws IOException {
+ producerWorkers.parallelStream()
+ .forEach(
+ w -> {
+ try {
+ w.probeProducers();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ });
+ }
+
+ @Override
+ public void adjustPublishRate(double publishRate) throws IOException {
+ double newRate = publishRate / numberOfUsedProducerWorkers;
+ log.debug("Adjusting producer publish rate to {} msgs/sec", newRate);
+ producerWorkers.parallelStream()
+ .forEach(
+ w -> {
+ try {
+ w.adjustPublishRate(newRate);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ });
+ }
+
+ @Override
+ public void stopAll() {
+ workers.parallelStream().forEach(Worker::stopAll);
+ }
+
+ @Override
+ public String id() {
+ return "Ensemble[" + workers.stream().map(Worker::id).collect(joining(",")) + "]";
+ }
+
+ @Override
+ public void pauseConsumers() throws IOException {
+ consumerWorkers.parallelStream()
+ .forEach(
+ w -> {
+ try {
+ w.pauseConsumers();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ });
+ }
+
+ @Override
+ public void resumeConsumers() throws IOException {
+ consumerWorkers.parallelStream()
+ .forEach(
+ w -> {
+ try {
+ w.resumeConsumers();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ });
+ }
+
+ @Override
+ public void pauseProducers() throws IOException {
+ producerWorkers.parallelStream()
+ .forEach(
+ w -> {
+ try {
+ w.pauseProducers();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ });
+ }
+
+ @Override
+ public void resumeProducers() throws IOException {
+ producerWorkers.parallelStream()
+ .forEach(
+ w -> {
+ try {
+ w.resumeProducers();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ });
+ }
+
+ @Override
+ public void createConsumers(ConsumerAssignment overallConsumerAssignment) {
+ List> subscriptionsPerConsumer =
+ ListPartition.partitionList(
+ overallConsumerAssignment.topicsSubscriptions, consumerWorkers.size());
+ Map topicsPerWorkerMap = Maps.newHashMap();
+ int i = 0;
+ for (List tsl : subscriptionsPerConsumer) {
+ ConsumerAssignment individualAssignment = new ConsumerAssignment();
+ individualAssignment.topicsSubscriptions = tsl;
+ topicsPerWorkerMap.put(consumerWorkers.get(i++), individualAssignment);
+ }
+ topicsPerWorkerMap.entrySet().parallelStream()
+ .forEach(
+ e -> {
+ try {
+ e.getKey().createConsumers(e.getValue());
+ } catch (IOException ex) {
+ throw new RuntimeException(ex);
+ }
+ });
+ }
+
+ @Override
+ public PeriodStats getPeriodStats() {
+ return workers.parallelStream()
+ .map(
+ w -> {
+ try {
+ return w.getPeriodStats();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ })
+ .reduce(new PeriodStats(), PeriodStats::plus);
+ }
+
+ @Override
+ public CumulativeLatencies getCumulativeLatencies() {
+ return workers.parallelStream()
+ .map(
+ w -> {
+ try {
+ return w.getCumulativeLatencies();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ })
+ .reduce(new CumulativeLatencies(), CumulativeLatencies::plus);
+ }
+
+ @Override
+ public CountersStats getCountersStats() throws IOException {
+ return workers.parallelStream()
+ .map(
+ w -> {
+ try {
+ return w.getCountersStats();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ })
+ .reduce(new CountersStats(), CountersStats::plus);
+ }
+
+ @Override
+ public void resetStats() throws IOException {
+ workers.parallelStream()
+ .forEach(
+ w -> {
+ try {
+ w.resetStats();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ });
+ }
+
+ @Override
+ public void close() throws Exception {
+ Runtime.getRuntime().removeShutdownHook(shutdownHook);
+ for (Worker w : workers) {
+ try {
+ w.close();
+ } catch (Exception ignored) {
+ log.trace("Ignored error while closing worker {}", w, ignored);
+ }
+ }
+ }
+
+ private static final Logger log = LoggerFactory.getLogger(DistributedWorkersEnsemble.class);
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/HttpWorkerClient.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/HttpWorkerClient.java
new file mode 100644
index 0000000..7344831
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/HttpWorkerClient.java
@@ -0,0 +1,249 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker;
+
+import static io.openmessaging.benchmark.worker.WorkerHandler.ADJUST_PUBLISH_RATE;
+import static io.openmessaging.benchmark.worker.WorkerHandler.COUNTERS_STATS;
+import static io.openmessaging.benchmark.worker.WorkerHandler.CREATE_CONSUMERS;
+import static io.openmessaging.benchmark.worker.WorkerHandler.CREATE_PRODUCERS;
+import static io.openmessaging.benchmark.worker.WorkerHandler.CREATE_TOPICS;
+import static io.openmessaging.benchmark.worker.WorkerHandler.CUMULATIVE_LATENCIES;
+import static io.openmessaging.benchmark.worker.WorkerHandler.INITIALIZE_DRIVER;
+import static io.openmessaging.benchmark.worker.WorkerHandler.PAUSE_CONSUMERS;
+import static io.openmessaging.benchmark.worker.WorkerHandler.PAUSE_PRODUCERS;
+import static io.openmessaging.benchmark.worker.WorkerHandler.PERIOD_STATS;
+import static io.openmessaging.benchmark.worker.WorkerHandler.PROBE_PRODUCERS;
+import static io.openmessaging.benchmark.worker.WorkerHandler.RESET_STATS;
+import static io.openmessaging.benchmark.worker.WorkerHandler.RESUME_CONSUMERS;
+import static io.openmessaging.benchmark.worker.WorkerHandler.RESUME_PRODUCERS;
+import static io.openmessaging.benchmark.worker.WorkerHandler.START_LOAD;
+import static io.openmessaging.benchmark.worker.WorkerHandler.STOP_ALL;
+import static org.asynchttpclient.Dsl.asyncHttpClient;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.base.Preconditions;
+import io.openmessaging.benchmark.worker.commands.ConsumerAssignment;
+import io.openmessaging.benchmark.worker.commands.CountersStats;
+import io.openmessaging.benchmark.worker.commands.CumulativeLatencies;
+import io.openmessaging.benchmark.worker.commands.PeriodStats;
+import io.openmessaging.benchmark.worker.commands.ProducerWorkAssignment;
+import io.openmessaging.benchmark.worker.commands.TopicsInfo;
+import io.openmessaging.benchmark.worker.jackson.ObjectMappers;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.time.Duration;
+import java.util.List;
+import org.asynchttpclient.AsyncHttpClient;
+import org.asynchttpclient.Dsl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HttpWorkerClient implements Worker {
+
+ private static final byte[] EMPTY_BODY = new byte[0];
+ private static final int HTTP_OK = 200;
+
+ private final AsyncHttpClient httpClient;
+ private final String host;
+
+ public HttpWorkerClient(String host) {
+ this(
+ asyncHttpClient(
+ Dsl.config()
+ .setReadTimeout(Duration.ofMillis(600000))
+ .setRequestTimeout(Duration.ofMillis(600000))),
+ host);
+ }
+
+ HttpWorkerClient(AsyncHttpClient httpClient, String host) {
+ this.httpClient = httpClient;
+ this.host = host;
+ }
+
+ @Override
+ public void initializeDriver(File configurationFile) throws IOException {
+ byte[] confFileContent = Files.readAllBytes(Paths.get(configurationFile.toString()));
+ sendPost(INITIALIZE_DRIVER, confFileContent);
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public List createTopics(TopicsInfo topicsInfo) throws IOException {
+ return (List) post(CREATE_TOPICS, writer.writeValueAsBytes(topicsInfo), List.class);
+ }
+
+ @Override
+ public void createProducers(List topics) throws IOException {
+ sendPost(CREATE_PRODUCERS, writer.writeValueAsBytes(topics));
+ }
+
+ @Override
+ public void createConsumers(ConsumerAssignment consumerAssignment) throws IOException {
+ sendPost(CREATE_CONSUMERS, writer.writeValueAsBytes(consumerAssignment));
+ }
+
+ @Override
+ public void probeProducers() throws IOException {
+ sendPost(PROBE_PRODUCERS);
+ }
+
+ @Override
+ public void startLoad(ProducerWorkAssignment producerWorkAssignment) throws IOException {
+ log.debug(
+ "Setting worker assigned publish rate to {} msgs/sec", producerWorkAssignment.publishRate);
+ sendPost(START_LOAD, writer.writeValueAsBytes(producerWorkAssignment));
+ }
+
+ @Override
+ public void adjustPublishRate(double publishRate) throws IOException {
+ log.debug("Adjusting worker publish rate to {} msgs/sec", publishRate);
+ sendPost(ADJUST_PUBLISH_RATE, writer.writeValueAsBytes(publishRate));
+ }
+
+ @Override
+ public void pauseConsumers() throws IOException {
+ sendPost(PAUSE_CONSUMERS);
+ }
+
+ @Override
+ public void resumeConsumers() throws IOException {
+ sendPost(RESUME_CONSUMERS);
+ }
+
+ @Override
+ public void pauseProducers() throws IOException {
+ sendPost(PAUSE_PRODUCERS);
+ }
+
+ @Override
+ public void resumeProducers() throws IOException {
+ sendPost(RESUME_PRODUCERS);
+ }
+
+ @Override
+ public CountersStats getCountersStats() throws IOException {
+ return get(COUNTERS_STATS, CountersStats.class);
+ }
+
+ @Override
+ public PeriodStats getPeriodStats() throws IOException {
+ return get(PERIOD_STATS, PeriodStats.class);
+ }
+
+ @Override
+ public CumulativeLatencies getCumulativeLatencies() throws IOException {
+ return get(CUMULATIVE_LATENCIES, CumulativeLatencies.class);
+ }
+
+ @Override
+ public void resetStats() throws IOException {
+ sendPost(RESET_STATS);
+ }
+
+ @Override
+ public void stopAll() {
+ sendPost(STOP_ALL);
+ }
+
+ @Override
+ public String id() {
+ return host;
+ }
+
+ @Override
+ public void close() throws Exception {
+ httpClient.close();
+ }
+
+ private void sendPost(String path) {
+ sendPost(path, EMPTY_BODY);
+ }
+
+ private void sendPost(String path, byte[] body) {
+ httpClient
+ .preparePost(host + path)
+ .setBody(body)
+ .execute()
+ .toCompletableFuture()
+ .thenApply(
+ response -> {
+ if (response.getStatusCode() != HTTP_OK) {
+ log.error(
+ "Failed to do HTTP post request to {}{} -- code: {}",
+ host,
+ path,
+ response.getStatusCode());
+ }
+ Preconditions.checkArgument(response.getStatusCode() == HTTP_OK);
+ return (Void) null;
+ })
+ .join();
+ }
+
+ private T get(String path, Class clazz) {
+ return httpClient
+ .prepareGet(host + path)
+ .execute()
+ .toCompletableFuture()
+ .thenApply(
+ response -> {
+ try {
+ if (response.getStatusCode() != HTTP_OK) {
+ log.error(
+ "Failed to do HTTP get request to {}{} -- code: {}",
+ host,
+ path,
+ response.getStatusCode());
+ }
+ Preconditions.checkArgument(response.getStatusCode() == HTTP_OK);
+ return mapper.readValue(response.getResponseBody(), clazz);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ })
+ .join();
+ }
+
+ private T post(String path, byte[] body, Class clazz) {
+ return httpClient
+ .preparePost(host + path)
+ .setBody(body)
+ .execute()
+ .toCompletableFuture()
+ .thenApply(
+ response -> {
+ try {
+ if (response.getStatusCode() != HTTP_OK) {
+ log.error(
+ "Failed to do HTTP post request to {}{} -- code: {}",
+ host,
+ path,
+ response.getStatusCode());
+ }
+ Preconditions.checkArgument(response.getStatusCode() == HTTP_OK);
+ return mapper.readValue(response.getResponseBody(), clazz);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ })
+ .join();
+ }
+
+ private static final ObjectMapper mapper = ObjectMappers.DEFAULT.mapper();
+ private static final ObjectWriter writer = ObjectMappers.DEFAULT.writer();
+ private static final Logger log = LoggerFactory.getLogger(HttpWorkerClient.class);
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/LocalWorker.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/LocalWorker.java
new file mode 100644
index 0000000..923c6b9
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/LocalWorker.java
@@ -0,0 +1,363 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker;
+
+import static java.util.stream.Collectors.toList;
+
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+import com.google.common.base.Preconditions;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import io.openmessaging.benchmark.DriverConfiguration;
+import io.openmessaging.benchmark.driver.BenchmarkConsumer;
+import io.openmessaging.benchmark.driver.BenchmarkDriver;
+import io.openmessaging.benchmark.driver.BenchmarkDriver.ConsumerInfo;
+import io.openmessaging.benchmark.driver.BenchmarkDriver.ProducerInfo;
+import io.openmessaging.benchmark.driver.BenchmarkDriver.TopicInfo;
+import io.openmessaging.benchmark.driver.BenchmarkProducer;
+import io.openmessaging.benchmark.driver.ConsumerCallback;
+import io.openmessaging.benchmark.utils.RandomGenerator;
+import io.openmessaging.benchmark.utils.Timer;
+import io.openmessaging.benchmark.utils.UniformRateLimiter;
+import io.openmessaging.benchmark.utils.distributor.KeyDistributor;
+import io.openmessaging.benchmark.worker.commands.ConsumerAssignment;
+import io.openmessaging.benchmark.worker.commands.CountersStats;
+import io.openmessaging.benchmark.worker.commands.CumulativeLatencies;
+import io.openmessaging.benchmark.worker.commands.PeriodStats;
+import io.openmessaging.benchmark.worker.commands.ProducerWorkAssignment;
+import io.openmessaging.benchmark.worker.commands.TopicsInfo;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.TreeMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.IntStream;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class LocalWorker implements Worker, ConsumerCallback {
+
+ private BenchmarkDriver benchmarkDriver = null;
+ private final List producers = new ArrayList<>();
+ private final List consumers = new ArrayList<>();
+ private volatile MessageProducer messageProducer;
+ private final ExecutorService executor =
+ Executors.newCachedThreadPool(new DefaultThreadFactory("local-worker"));
+ private final WorkerStats stats;
+ private boolean testCompleted = false;
+ private boolean consumersArePaused = false;
+ private boolean producersArePaused = false;
+
+ public LocalWorker() {
+ this(NullStatsLogger.INSTANCE);
+ }
+
+ public LocalWorker(StatsLogger statsLogger) {
+ stats = new WorkerStats(statsLogger);
+ updateMessageProducer(1.0);
+ }
+
+ @Override
+ public void initializeDriver(File driverConfigFile) throws IOException {
+ Preconditions.checkArgument(benchmarkDriver == null);
+ testCompleted = false;
+
+ DriverConfiguration driverConfiguration =
+ mapper.readValue(driverConfigFile, DriverConfiguration.class);
+
+ log.info("Driver: {}", writer.writeValueAsString(driverConfiguration));
+
+ try {
+ benchmarkDriver =
+ (BenchmarkDriver) Class.forName(driverConfiguration.driverClass).newInstance();
+ benchmarkDriver.initialize(driverConfigFile, stats.getStatsLogger());
+ } catch (InstantiationException
+ | IllegalAccessException
+ | ClassNotFoundException
+ | InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public List createTopics(TopicsInfo topicsInfo) {
+ Timer timer = new Timer();
+
+ List topicInfos =
+ IntStream.range(0, topicsInfo.numberOfTopics)
+ .mapToObj(
+ i -> new TopicInfo(generateTopicName(i), topicsInfo.numberOfPartitionsPerTopic))
+ .collect(toList());
+
+ benchmarkDriver.createTopics(topicInfos).join();
+
+ List topics = topicInfos.stream().map(TopicInfo::getTopic).collect(toList());
+
+ log.info("Created {} topics in {} ms", topics.size(), timer.elapsedMillis());
+ return topics;
+ }
+
+ private String generateTopicName(int i) {
+ return String.format(
+ "%s_%07d_%s",
+ benchmarkDriver.getTopicNamePrefix(),
+ i,
+ RandomGenerator.genRandomBinaryString().replace("-", ""));
+ }
+
+ @Override
+ public void createProducers(List topics) {
+ Timer timer = new Timer();
+ AtomicInteger index = new AtomicInteger();
+
+ producers.addAll(
+ benchmarkDriver
+ .createProducers(
+ topics.stream()
+ .map(t -> new ProducerInfo(index.getAndIncrement(), t))
+ .collect(toList()))
+ .join());
+
+ log.info("Created {} producers in {} ms", producers.size(), timer.elapsedMillis());
+ }
+
+ @Override
+ public void createConsumers(ConsumerAssignment consumerAssignment) {
+ Timer timer = new Timer();
+
+ consumers.addAll(
+ benchmarkDriver
+ .createConsumers(
+ consumerAssignment.topicsSubscriptions.stream()
+ .map(
+ c ->
+ new ConsumerInfo(
+ c.subId,
+ c.partitionsPerTopic,
+ c.partitionsPerSubscription,
+ c.topic,
+ c.subscription,
+ this))
+ .collect(toList()))
+ .join());
+
+ log.info("Created {} consumers in {} ms", consumers.size(), timer.elapsedMillis());
+ }
+
+ @Override
+ public void startLoad(ProducerWorkAssignment producerWorkAssignment) {
+ int processors = Runtime.getRuntime().availableProcessors();
+
+ updateMessageProducer(producerWorkAssignment.publishRate);
+
+ Map> processorAssignment = new TreeMap<>();
+
+ int processorIdx = 0;
+ for (BenchmarkProducer p : producers) {
+ processorAssignment
+ .computeIfAbsent(processorIdx, x -> new ArrayList())
+ .add(p);
+
+ processorIdx = (processorIdx + 1) % processors;
+ }
+
+ processorAssignment
+ .values()
+ .forEach(
+ producers ->
+ submitProducersToExecutor(
+ producers,
+ KeyDistributor.build(producerWorkAssignment.keyDistributorType),
+ producerWorkAssignment.payloadData));
+ }
+
+ @Override
+ public void probeProducers() throws IOException {
+ producers.forEach(
+ producer ->
+ producer.sendAsync(Optional.of("key"), new byte[10]).thenRun(stats::recordMessageSent));
+ }
+
+ private void submitProducersToExecutor(
+ List producers, KeyDistributor keyDistributor, List payloads) {
+ ThreadLocalRandom r = ThreadLocalRandom.current();
+ int payloadCount = payloads.size();
+ executor.submit(
+ () -> {
+ try {
+ while (!testCompleted) {
+ while (producersArePaused) {
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+
+ producers.forEach(
+ p ->
+ messageProducer.sendMessage(
+ p,
+ Optional.ofNullable(keyDistributor.next()),
+ payloads.get(r.nextInt(payloadCount))));
+ }
+ } catch (Throwable t) {
+ log.error("Got error", t);
+ }
+ });
+ }
+
+ @Override
+ public void adjustPublishRate(double publishRate) {
+ if (publishRate < 1.0) {
+ updateMessageProducer(1.0);
+ return;
+ }
+ updateMessageProducer(publishRate);
+ }
+
+ private void updateMessageProducer(double publishRate) {
+ messageProducer = new MessageProducer(new UniformRateLimiter(publishRate), stats);
+ }
+
+ @Override
+ public PeriodStats getPeriodStats() {
+ return stats.toPeriodStats();
+ }
+
+ @Override
+ public CumulativeLatencies getCumulativeLatencies() {
+ return stats.toCumulativeLatencies();
+ }
+
+ @Override
+ public CountersStats getCountersStats() throws IOException {
+ return stats.toCountersStats();
+ }
+
+ @Override
+ public void messageReceived(byte[] data, long publishTimestamp) {
+ internalMessageReceived(data.length, publishTimestamp);
+ }
+
+ @Override
+ public void messageReceived(ByteBuffer data, long publishTimestamp) {
+ internalMessageReceived(data.remaining(), publishTimestamp);
+ }
+
+ public void internalMessageReceived(int size, long publishTimestamp) {
+ long now = System.currentTimeMillis();
+ long endToEndLatencyMicros = TimeUnit.MILLISECONDS.toMicros(now - publishTimestamp);
+ stats.recordMessageReceived(size, endToEndLatencyMicros);
+
+ while (consumersArePaused) {
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+
+ @Override
+ public void pauseConsumers() throws IOException {
+ consumersArePaused = true;
+ log.info("Pausing consumers");
+ }
+
+ @Override
+ public void resumeConsumers() throws IOException {
+ consumersArePaused = false;
+ log.info("Resuming consumers");
+ }
+
+ @Override
+ public void pauseProducers() throws IOException {
+ producersArePaused = true;
+ log.info("Pausing producers");
+ }
+
+ @Override
+ public void resumeProducers() throws IOException {
+ producersArePaused = false;
+ log.info("Resuming producers");
+ }
+
+ @Override
+ public void resetStats() throws IOException {
+ stats.resetLatencies();
+ }
+
+ @Override
+ public void stopAll() {
+ testCompleted = true;
+ consumersArePaused = false;
+ stats.reset();
+
+ try {
+ Thread.sleep(100);
+
+ for (BenchmarkProducer producer : producers) {
+ producer.close();
+ }
+ producers.clear();
+
+ for (BenchmarkConsumer consumer : consumers) {
+ consumer.close();
+ }
+ consumers.clear();
+
+ if (benchmarkDriver != null) {
+ benchmarkDriver.close();
+ benchmarkDriver = null;
+ }
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public String id() {
+ return "local";
+ }
+
+ @Override
+ public void close() throws Exception {
+ executor.shutdown();
+ }
+
+ private static final ObjectWriter writer = new ObjectMapper().writerWithDefaultPrettyPrinter();
+
+ private static final ObjectMapper mapper =
+ new ObjectMapper(new YAMLFactory())
+ .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+ static {
+ mapper.enable(DeserializationFeature.READ_UNKNOWN_ENUM_VALUES_USING_DEFAULT_VALUE);
+ }
+
+ private static final Logger log = LoggerFactory.getLogger(LocalWorker.class);
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/MessageProducer.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/MessageProducer.java
new file mode 100644
index 0000000..bf1191e
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/MessageProducer.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker;
+
+import static io.openmessaging.benchmark.utils.UniformRateLimiter.uninterruptibleSleepNs;
+
+import io.openmessaging.benchmark.driver.BenchmarkProducer;
+import io.openmessaging.benchmark.utils.UniformRateLimiter;
+import java.util.Optional;
+import java.util.function.Supplier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class MessageProducer {
+
+ private final WorkerStats stats;
+ private UniformRateLimiter rateLimiter;
+ private Supplier nanoClock;
+
+ MessageProducer(UniformRateLimiter rateLimiter, WorkerStats stats) {
+ this(System::nanoTime, rateLimiter, stats);
+ }
+
+ MessageProducer(Supplier nanoClock, UniformRateLimiter rateLimiter, WorkerStats stats) {
+ this.nanoClock = nanoClock;
+ this.rateLimiter = rateLimiter;
+ this.stats = stats;
+ }
+
+ public void sendMessage(BenchmarkProducer producer, Optional key, byte[] payload) {
+ final long intendedSendTime = rateLimiter.acquire();
+ uninterruptibleSleepNs(intendedSendTime);
+ final long sendTime = nanoClock.get();
+ producer
+ .sendAsync(key, payload)
+ .thenRun(() -> success(payload.length, intendedSendTime, sendTime))
+ .exceptionally(this::failure);
+ }
+
+ private void success(long payloadLength, long intendedSendTime, long sendTime) {
+ long nowNs = nanoClock.get();
+ stats.recordProducerSuccess(payloadLength, intendedSendTime, sendTime, nowNs);
+ }
+
+ private Void failure(Throwable t) {
+ stats.recordProducerFailure();
+ log.warn("Write error on message", t);
+ return null;
+ }
+
+ private static final Logger log = LoggerFactory.getLogger(MessageProducer.class);
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/Worker.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/Worker.java
new file mode 100644
index 0000000..f88205b
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/Worker.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker;
+
+
+import io.openmessaging.benchmark.worker.commands.ConsumerAssignment;
+import io.openmessaging.benchmark.worker.commands.CountersStats;
+import io.openmessaging.benchmark.worker.commands.CumulativeLatencies;
+import io.openmessaging.benchmark.worker.commands.PeriodStats;
+import io.openmessaging.benchmark.worker.commands.ProducerWorkAssignment;
+import io.openmessaging.benchmark.worker.commands.TopicsInfo;
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+public interface Worker extends AutoCloseable {
+
+ void initializeDriver(File configurationFile) throws IOException;
+
+ List createTopics(TopicsInfo topicsInfo) throws IOException;
+
+ void createProducers(List topics) throws IOException;
+
+ void createConsumers(ConsumerAssignment consumerAssignment) throws IOException;
+
+ void probeProducers() throws IOException;
+
+ void startLoad(ProducerWorkAssignment producerWorkAssignment) throws IOException;
+
+ void adjustPublishRate(double publishRate) throws IOException;
+
+ void pauseConsumers() throws IOException;
+
+ void resumeConsumers() throws IOException;
+
+ void pauseProducers() throws IOException;
+
+ void resumeProducers() throws IOException;
+
+ CountersStats getCountersStats() throws IOException;
+
+ PeriodStats getPeriodStats() throws IOException;
+
+ CumulativeLatencies getCumulativeLatencies() throws IOException;
+
+ void resetStats() throws IOException;
+
+ void stopAll();
+
+ String id();
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/WorkerHandler.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/WorkerHandler.java
new file mode 100644
index 0000000..dd1e624
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/WorkerHandler.java
@@ -0,0 +1,168 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker;
+
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.io.Files;
+import io.javalin.Context;
+import io.javalin.Javalin;
+import io.openmessaging.benchmark.worker.commands.ConsumerAssignment;
+import io.openmessaging.benchmark.worker.commands.ProducerWorkAssignment;
+import io.openmessaging.benchmark.worker.commands.TopicsInfo;
+import io.openmessaging.benchmark.worker.jackson.ObjectMappers;
+import java.io.File;
+import java.util.List;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("unchecked")
+public class WorkerHandler {
+
+ public static final String INITIALIZE_DRIVER = "/initialize-driver";
+ public static final String CREATE_TOPICS = "/create-topics";
+ public static final String CREATE_PRODUCERS = "/create-producers";
+ public static final String PROBE_PRODUCERS = "/probe-producers";
+ public static final String CREATE_CONSUMERS = "/create-consumers";
+ public static final String PAUSE_CONSUMERS = "/pause-consumers";
+ public static final String RESUME_CONSUMERS = "/resume-consumers";
+ public static final String PAUSE_PRODUCERS = "/pause-producers";
+ public static final String RESUME_PRODUCERS = "/resume-producers";
+ public static final String START_LOAD = "/start-load";
+ public static final String ADJUST_PUBLISH_RATE = "/adjust-publish-rate";
+ public static final String STOP_ALL = "/stop-all";
+ public static final String PERIOD_STATS = "/period-stats";
+ public static final String CUMULATIVE_LATENCIES = "/cumulative-latencies";
+ public static final String COUNTERS_STATS = "/counters-stats";
+ public static final String RESET_STATS = "/reset-stats";
+ private final Worker localWorker;
+
+ public WorkerHandler(Javalin app, StatsLogger statsLogger) {
+ this.localWorker = new LocalWorker(statsLogger);
+
+ app.post(INITIALIZE_DRIVER, this::handleInitializeDriver);
+ app.post(CREATE_TOPICS, this::handleCreateTopics);
+ app.post(CREATE_PRODUCERS, this::handleCreateProducers);
+ app.post(PROBE_PRODUCERS, this::handleProbeProducers);
+ app.post(CREATE_CONSUMERS, this::handleCreateConsumers);
+ app.post(PAUSE_CONSUMERS, this::handlePauseConsumers);
+ app.post(RESUME_CONSUMERS, this::handleResumeConsumers);
+ app.post(PAUSE_PRODUCERS, this::handlePauseProducers);
+ app.post(RESUME_PRODUCERS, this::handleResumeProducers);
+ app.post(START_LOAD, this::handleStartLoad);
+ app.post(ADJUST_PUBLISH_RATE, this::handleAdjustPublishRate);
+ app.post(STOP_ALL, this::handleStopAll);
+ app.get(PERIOD_STATS, this::handlePeriodStats);
+ app.get(CUMULATIVE_LATENCIES, this::handleCumulativeLatencies);
+ app.get(COUNTERS_STATS, this::handleCountersStats);
+ app.post(RESET_STATS, this::handleResetStats);
+ }
+
+ private void handleInitializeDriver(Context ctx) throws Exception {
+ // Save config to temp file
+ File tempFile = File.createTempFile("driver-configuration", "conf");
+ Files.write(ctx.bodyAsBytes(), tempFile);
+
+ localWorker.initializeDriver(tempFile);
+ tempFile.delete();
+ }
+
+ private void handleCreateTopics(Context ctx) throws Exception {
+ TopicsInfo topicsInfo = mapper.readValue(ctx.body(), TopicsInfo.class);
+ log.info("Received create topics request for topics: {}", ctx.body());
+ List topics = localWorker.createTopics(topicsInfo);
+ ctx.result(writer.writeValueAsString(topics));
+ }
+
+ private void handleCreateProducers(Context ctx) throws Exception {
+ List topics = (List) mapper.readValue(ctx.body(), List.class);
+ log.info("Received create producers request for topics: {}", topics);
+ localWorker.createProducers(topics);
+ }
+
+ private void handleProbeProducers(Context ctx) throws Exception {
+ localWorker.probeProducers();
+ }
+
+ private void handleCreateConsumers(Context ctx) throws Exception {
+ ConsumerAssignment consumerAssignment = mapper.readValue(ctx.body(), ConsumerAssignment.class);
+
+ log.info(
+ "Received create consumers request for topics: {}", consumerAssignment.topicsSubscriptions);
+ localWorker.createConsumers(consumerAssignment);
+ }
+
+ private void handlePauseConsumers(Context ctx) throws Exception {
+ localWorker.pauseConsumers();
+ }
+
+ private void handleResumeConsumers(Context ctx) throws Exception {
+ localWorker.resumeConsumers();
+ }
+
+ private void handlePauseProducers(Context ctx) throws Exception {
+ localWorker.pauseProducers();
+ }
+
+ private void handleResumeProducers(Context ctx) throws Exception {
+ localWorker.resumeProducers();
+ }
+
+ private void handleStartLoad(Context ctx) throws Exception {
+ ProducerWorkAssignment producerWorkAssignment =
+ mapper.readValue(ctx.body(), ProducerWorkAssignment.class);
+
+ log.info(
+ "Start load publish-rate: {} msg/s -- payload-size: {}",
+ producerWorkAssignment.publishRate,
+ producerWorkAssignment.payloadData.get(0).length);
+
+ localWorker.startLoad(producerWorkAssignment);
+ }
+
+ private void handleAdjustPublishRate(Context ctx) throws Exception {
+ Double publishRate = mapper.readValue(ctx.body(), Double.class);
+ log.info("Adjust publish-rate: {} msg/s", publishRate);
+ localWorker.adjustPublishRate(publishRate);
+ }
+
+ private void handleStopAll(Context ctx) throws Exception {
+ log.info("Stop All");
+ localWorker.stopAll();
+ }
+
+ private void handlePeriodStats(Context ctx) throws Exception {
+ ctx.result(writer.writeValueAsString(localWorker.getPeriodStats()));
+ }
+
+ private void handleCumulativeLatencies(Context ctx) throws Exception {
+ ctx.result(writer.writeValueAsString(localWorker.getCumulativeLatencies()));
+ }
+
+ private void handleCountersStats(Context ctx) throws Exception {
+ ctx.result(writer.writeValueAsString(localWorker.getCountersStats()));
+ }
+
+ private void handleResetStats(Context ctx) throws Exception {
+ log.info("Reset stats");
+ localWorker.resetStats();
+ }
+
+ private static final Logger log = LoggerFactory.getLogger(WorkerHandler.class);
+
+ private static final ObjectMapper mapper = ObjectMappers.DEFAULT.mapper();
+ private static final ObjectWriter writer = ObjectMappers.DEFAULT.writer();
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/WorkerStats.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/WorkerStats.java
new file mode 100644
index 0000000..68b9d53
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/WorkerStats.java
@@ -0,0 +1,186 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker;
+
+
+import io.openmessaging.benchmark.worker.commands.CountersStats;
+import io.openmessaging.benchmark.worker.commands.CumulativeLatencies;
+import io.openmessaging.benchmark.worker.commands.PeriodStats;
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAdder;
+import org.HdrHistogram.Recorder;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+public class WorkerStats {
+
+ private final StatsLogger statsLogger;
+
+ private final OpStatsLogger publishDelayLatencyStats;
+
+ private final Recorder endToEndLatencyRecorder = new Recorder(TimeUnit.HOURS.toMicros(12), 5);
+ private final Recorder endToEndCumulativeLatencyRecorder =
+ new Recorder(TimeUnit.HOURS.toMicros(12), 5);
+ private final OpStatsLogger endToEndLatencyStats;
+
+ private final LongAdder messagesSent = new LongAdder();
+ private final LongAdder messageSendErrors = new LongAdder();
+ private final LongAdder bytesSent = new LongAdder();
+ private final Counter messageSendErrorCounter;
+ private final Counter messagesSentCounter;
+ private final Counter bytesSentCounter;
+
+ private final LongAdder messagesReceived = new LongAdder();
+ private final LongAdder bytesReceived = new LongAdder();
+ private final Counter messagesReceivedCounter;
+ private final Counter bytesReceivedCounter;
+
+ private final LongAdder totalMessagesSent = new LongAdder();
+ private final LongAdder totalMessageSendErrors = new LongAdder();
+ private final LongAdder totalMessagesReceived = new LongAdder();
+
+ private static final long highestTrackableValue = TimeUnit.SECONDS.toMicros(60);
+ private final Recorder publishLatencyRecorder = new Recorder(highestTrackableValue, 5);
+ private final Recorder cumulativePublishLatencyRecorder = new Recorder(highestTrackableValue, 5);
+ private final OpStatsLogger publishLatencyStats;
+
+ private final Recorder publishDelayLatencyRecorder = new Recorder(highestTrackableValue, 5);
+ private final Recorder cumulativePublishDelayLatencyRecorder =
+ new Recorder(highestTrackableValue, 5);
+
+ WorkerStats(StatsLogger statsLogger) {
+ this.statsLogger = statsLogger;
+
+ StatsLogger producerStatsLogger = statsLogger.scope("producer");
+ this.messagesSentCounter = producerStatsLogger.getCounter("messages_sent");
+ this.messageSendErrorCounter = producerStatsLogger.getCounter("message_send_errors");
+ this.bytesSentCounter = producerStatsLogger.getCounter("bytes_sent");
+ this.publishDelayLatencyStats = producerStatsLogger.getOpStatsLogger("producer_delay_latency");
+ this.publishLatencyStats = producerStatsLogger.getOpStatsLogger("produce_latency");
+
+ StatsLogger consumerStatsLogger = statsLogger.scope("consumer");
+ this.messagesReceivedCounter = consumerStatsLogger.getCounter("messages_recv");
+ this.bytesReceivedCounter = consumerStatsLogger.getCounter("bytes_recv");
+ this.endToEndLatencyStats = consumerStatsLogger.getOpStatsLogger("e2e_latency");
+ }
+
+ public StatsLogger getStatsLogger() {
+ return statsLogger;
+ }
+
+ public void recordMessageSent() {
+ totalMessagesSent.increment();
+ }
+
+ public void recordMessageReceived(long payloadLength, long endToEndLatencyMicros) {
+ messagesReceived.increment();
+ totalMessagesReceived.increment();
+ messagesReceivedCounter.inc();
+ bytesReceived.add(payloadLength);
+ bytesReceivedCounter.add(payloadLength);
+
+ if (endToEndLatencyMicros > 0) {
+ endToEndCumulativeLatencyRecorder.recordValue(endToEndLatencyMicros);
+ endToEndLatencyRecorder.recordValue(endToEndLatencyMicros);
+ endToEndLatencyStats.registerSuccessfulEvent(endToEndLatencyMicros, TimeUnit.MICROSECONDS);
+ }
+ }
+
+ public PeriodStats toPeriodStats() {
+ PeriodStats stats = new PeriodStats();
+
+ stats.messagesSent = messagesSent.sumThenReset();
+ stats.messageSendErrors = messageSendErrors.sumThenReset();
+ stats.bytesSent = bytesSent.sumThenReset();
+
+ stats.messagesReceived = messagesReceived.sumThenReset();
+ stats.bytesReceived = bytesReceived.sumThenReset();
+
+ stats.totalMessagesSent = totalMessagesSent.sum();
+ stats.totalMessageSendErrors = totalMessageSendErrors.sum();
+ stats.totalMessagesReceived = totalMessagesReceived.sum();
+
+ stats.publishLatency = publishLatencyRecorder.getIntervalHistogram();
+ stats.publishDelayLatency = publishDelayLatencyRecorder.getIntervalHistogram();
+ stats.endToEndLatency = endToEndLatencyRecorder.getIntervalHistogram();
+ return stats;
+ }
+
+ public CumulativeLatencies toCumulativeLatencies() {
+ CumulativeLatencies latencies = new CumulativeLatencies();
+ latencies.publishLatency = cumulativePublishLatencyRecorder.getIntervalHistogram();
+ latencies.publishDelayLatency = cumulativePublishDelayLatencyRecorder.getIntervalHistogram();
+ latencies.endToEndLatency = endToEndCumulativeLatencyRecorder.getIntervalHistogram();
+ return latencies;
+ }
+
+ public CountersStats toCountersStats() throws IOException {
+ CountersStats stats = new CountersStats();
+ stats.messagesSent = totalMessagesSent.sum();
+ stats.messageSendErrors = totalMessageSendErrors.sum();
+ stats.messagesReceived = totalMessagesReceived.sum();
+ return stats;
+ }
+
+ public void resetLatencies() {
+ publishLatencyRecorder.reset();
+ cumulativePublishLatencyRecorder.reset();
+ publishDelayLatencyRecorder.reset();
+ cumulativePublishDelayLatencyRecorder.reset();
+ endToEndLatencyRecorder.reset();
+ endToEndCumulativeLatencyRecorder.reset();
+ }
+
+ public void reset() {
+ resetLatencies();
+
+ messagesSent.reset();
+ messageSendErrors.reset();
+ bytesSent.reset();
+ messagesReceived.reset();
+ bytesReceived.reset();
+ totalMessagesSent.reset();
+ totalMessagesReceived.reset();
+ }
+
+ public void recordProducerFailure() {
+ messageSendErrors.increment();
+ messageSendErrorCounter.inc();
+ totalMessageSendErrors.increment();
+ }
+
+ public void recordProducerSuccess(
+ long payloadLength, long intendedSendTimeNs, long sendTimeNs, long nowNs) {
+ messagesSent.increment();
+ totalMessagesSent.increment();
+ messagesSentCounter.inc();
+ bytesSent.add(payloadLength);
+ bytesSentCounter.add(payloadLength);
+
+ final long latencyMicros =
+ Math.min(highestTrackableValue, TimeUnit.NANOSECONDS.toMicros(nowNs - sendTimeNs));
+ publishLatencyRecorder.recordValue(latencyMicros);
+ cumulativePublishLatencyRecorder.recordValue(latencyMicros);
+ publishLatencyStats.registerSuccessfulEvent(latencyMicros, TimeUnit.MICROSECONDS);
+
+ final long sendDelayMicros =
+ Math.min(
+ highestTrackableValue, TimeUnit.NANOSECONDS.toMicros(sendTimeNs - intendedSendTimeNs));
+ publishDelayLatencyRecorder.recordValue(sendDelayMicros);
+ cumulativePublishDelayLatencyRecorder.recordValue(sendDelayMicros);
+ publishDelayLatencyStats.registerSuccessfulEvent(sendDelayMicros, TimeUnit.MICROSECONDS);
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/ConsumerAssignment.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/ConsumerAssignment.java
new file mode 100644
index 0000000..9d97642
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/ConsumerAssignment.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker.commands;
+
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class ConsumerAssignment {
+ public List topicsSubscriptions = new ArrayList<>();
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/CountersStats.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/CountersStats.java
new file mode 100644
index 0000000..6ddc593
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/CountersStats.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker.commands;
+
+public class CountersStats {
+ public long messagesSent;
+ public long messagesReceived;
+ public long messageSendErrors;
+
+ public CountersStats plus(CountersStats toAdd) {
+ CountersStats result = new CountersStats();
+ result.messagesSent += this.messagesSent;
+ result.messagesReceived += this.messagesReceived;
+ result.messageSendErrors += this.messageSendErrors;
+
+ result.messagesSent += toAdd.messagesSent;
+ result.messagesReceived += toAdd.messagesReceived;
+ result.messageSendErrors += toAdd.messageSendErrors;
+ return result;
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/CumulativeLatencies.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/CumulativeLatencies.java
new file mode 100644
index 0000000..e9afea6
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/CumulativeLatencies.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker.commands;
+
+import static java.util.concurrent.TimeUnit.HOURS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+import org.HdrHistogram.Histogram;
+
+public class CumulativeLatencies {
+
+ public Histogram publishLatency = new Histogram(SECONDS.toMicros(60), 5);
+ public Histogram publishDelayLatency = new Histogram(SECONDS.toMicros(60), 5);
+ public Histogram endToEndLatency = new Histogram(HOURS.toMicros(12), 5);
+
+ public CumulativeLatencies plus(CumulativeLatencies toAdd) {
+ CumulativeLatencies result = new CumulativeLatencies();
+
+ result.publishLatency.add(this.publishLatency);
+ result.publishDelayLatency.add(this.publishDelayLatency);
+ result.endToEndLatency.add(this.endToEndLatency);
+
+ result.publishLatency.add(toAdd.publishLatency);
+ result.publishDelayLatency.add(toAdd.publishDelayLatency);
+ result.endToEndLatency.add(toAdd.endToEndLatency);
+
+ return result;
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/PeriodStats.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/PeriodStats.java
new file mode 100644
index 0000000..3f06f69
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/PeriodStats.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker.commands;
+
+import static java.util.concurrent.TimeUnit.HOURS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+import org.HdrHistogram.Histogram;
+
+public class PeriodStats {
+ public long messagesSent = 0;
+ public long messageSendErrors = 0;
+ public long bytesSent = 0;
+
+ public long messagesReceived = 0;
+ public long bytesReceived = 0;
+
+ public long totalMessagesSent = 0;
+ public long totalMessageSendErrors = 0;
+ public long totalMessagesReceived = 0;
+
+ public Histogram publishLatency = new Histogram(SECONDS.toMicros(60), 5);
+ public Histogram publishDelayLatency = new Histogram(SECONDS.toMicros(60), 5);
+ public Histogram endToEndLatency = new Histogram(HOURS.toMicros(12), 5);
+
+ public PeriodStats plus(PeriodStats toAdd) {
+ PeriodStats result = new PeriodStats();
+
+ result.messagesSent += this.messagesSent;
+ result.messageSendErrors += this.messageSendErrors;
+ result.bytesSent += this.bytesSent;
+ result.messagesReceived += this.messagesReceived;
+ result.bytesReceived += this.bytesReceived;
+ result.totalMessagesSent += this.totalMessagesSent;
+ result.totalMessageSendErrors += this.totalMessageSendErrors;
+ result.totalMessagesReceived += this.totalMessagesReceived;
+ result.publishLatency.add(this.publishLatency);
+ result.publishDelayLatency.add(this.publishDelayLatency);
+ result.endToEndLatency.add(this.endToEndLatency);
+
+ result.messagesSent += toAdd.messagesSent;
+ result.messageSendErrors += toAdd.messageSendErrors;
+ result.bytesSent += toAdd.bytesSent;
+ result.messagesReceived += toAdd.messagesReceived;
+ result.bytesReceived += toAdd.bytesReceived;
+ result.totalMessagesSent += toAdd.totalMessagesSent;
+ result.totalMessageSendErrors += toAdd.totalMessageSendErrors;
+ result.totalMessagesReceived += toAdd.totalMessagesReceived;
+ result.publishLatency.add(toAdd.publishLatency);
+ result.publishDelayLatency.add(toAdd.publishDelayLatency);
+ result.endToEndLatency.add(toAdd.endToEndLatency);
+
+ return result;
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/ProducerWorkAssignment.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/ProducerWorkAssignment.java
new file mode 100644
index 0000000..9506ef7
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/ProducerWorkAssignment.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker.commands;
+
+
+import io.openmessaging.benchmark.utils.distributor.KeyDistributorType;
+import java.util.List;
+
+public class ProducerWorkAssignment {
+
+ public List payloadData;
+
+ public double publishRate;
+
+ public KeyDistributorType keyDistributorType;
+
+ public ProducerWorkAssignment withPublishRate(double publishRate) {
+ ProducerWorkAssignment copy = new ProducerWorkAssignment();
+ copy.keyDistributorType = this.keyDistributorType;
+ copy.payloadData = this.payloadData;
+ copy.publishRate = publishRate;
+ return copy;
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/TopicSubscription.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/TopicSubscription.java
new file mode 100644
index 0000000..688fb94
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/TopicSubscription.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker.commands;
+
+public class TopicSubscription {
+ public String topic;
+ public int subId;
+ public String subscription;
+ public int partitionsPerTopic;
+ public int partitionsPerSubscription;
+
+ public TopicSubscription() {}
+
+ public TopicSubscription(
+ String topic,
+ int subId,
+ String subscription,
+ int partitionsPerTopic,
+ int partitionsPerSubscription) {
+ this.topic = topic;
+ this.subId = subId;
+ this.subscription = subscription;
+ this.partitionsPerTopic = partitionsPerTopic;
+ this.partitionsPerSubscription = partitionsPerSubscription;
+ }
+
+ @Override
+ public String toString() {
+ return "TopicSubscription{"
+ + "topic='"
+ + topic
+ + '\''
+ + ", subId="
+ + subId
+ + '\''
+ + ", subscription='"
+ + subscription
+ + '\''
+ + ", partitionsPerTopic="
+ + partitionsPerTopic
+ + ", partitionsPerSubscription="
+ + partitionsPerSubscription
+ + '}';
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/TopicsInfo.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/TopicsInfo.java
new file mode 100644
index 0000000..90545c7
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/TopicsInfo.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker.commands;
+
+public class TopicsInfo {
+ public int numberOfTopics;
+ public int numberOfPartitionsPerTopic;
+
+ public TopicsInfo() {}
+
+ public TopicsInfo(int numberOfTopics, int numberOfPartitionsPerTopic) {
+ this.numberOfTopics = numberOfTopics;
+ this.numberOfPartitionsPerTopic = numberOfPartitionsPerTopic;
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/jackson/HistogramDeserializer.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/jackson/HistogramDeserializer.java
new file mode 100644
index 0000000..6aa562b
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/jackson/HistogramDeserializer.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker.jackson;
+
+
+import com.fasterxml.jackson.core.JacksonException;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
+import com.fasterxml.jackson.databind.util.ByteBufferBackedOutputStream;
+import io.netty.buffer.ByteBufUtil;
+import io.netty.buffer.Unpooled;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import org.HdrHistogram.Histogram;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HistogramDeserializer extends StdDeserializer {
+
+ private final ThreadLocal threadBuffer =
+ ThreadLocal.withInitial(() -> ByteBuffer.allocate(8 * 1024 * 1024));
+
+ public HistogramDeserializer() {
+ super(Histogram.class);
+ }
+
+ @Override
+ public Histogram deserialize(JsonParser jsonParser, DeserializationContext deserializationContext)
+ throws IOException, JacksonException {
+ ByteBuffer buffer = threadBuffer.get();
+ buffer.clear();
+ try (OutputStream os = new ByteBufferBackedOutputStream(buffer)) {
+ jsonParser.readBinaryValue(os);
+ buffer.flip();
+ // Long.MIN_VALUE used so that Histogram will defer to the value encoded in the histogram
+ // value. This assumes that it is acceptable for the deserialized value we create to
+ // share the same parameters of the source histogram that was serialized.
+ return Histogram.decodeFromCompressedByteBuffer(buffer, Long.MIN_VALUE);
+ } catch (Exception e) {
+ log.error(
+ "Failed to decode publish delay latency: {}",
+ ByteBufUtil.prettyHexDump(Unpooled.wrappedBuffer(buffer)));
+ throw new RuntimeException(e);
+ }
+ }
+
+ private static final Logger log = LoggerFactory.getLogger(HistogramDeserializer.class);
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/jackson/HistogramSerializer.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/jackson/HistogramSerializer.java
new file mode 100644
index 0000000..c1c253f
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/jackson/HistogramSerializer.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker.jackson;
+
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.HdrHistogram.Histogram;
+
+public class HistogramSerializer extends StdSerializer {
+
+ private final ThreadLocal threadBuffer =
+ ThreadLocal.withInitial(() -> ByteBuffer.allocate(8 * 1024 * 1024));
+
+ public HistogramSerializer() {
+ super(Histogram.class);
+ }
+
+ static byte[] toByteArray(ByteBuffer buffer) {
+ byte[] encodedBuffer = new byte[buffer.remaining()];
+ buffer.get(encodedBuffer);
+ return encodedBuffer;
+ }
+
+ static ByteBuffer serializeHistogram(Histogram histo, ByteBuffer buffer) {
+ buffer.clear();
+ while (true) {
+ final int outBytes = histo.encodeIntoCompressedByteBuffer(buffer);
+ Preconditions.checkState(outBytes == buffer.position());
+ final int capacity = buffer.capacity();
+ if (outBytes < capacity) {
+ // encoding succesful
+ break;
+ }
+ // We filled the entire buffer, an indication that the buffer was not
+ // large enough, so we double the buffer and try again.
+ // See: https://github.com/HdrHistogram/HdrHistogram/issues/201
+ buffer = ByteBuffer.allocate(capacity * 2);
+ }
+ buffer.flip();
+ return buffer;
+ }
+
+ @Override
+ public void serialize(
+ Histogram histogram, JsonGenerator jsonGenerator, SerializerProvider serializerProvider)
+ throws IOException {
+ ByteBuffer buffer = threadBuffer.get();
+ ByteBuffer newBuffer = serializeHistogram(histogram, buffer);
+ if (newBuffer != buffer) {
+ threadBuffer.set(newBuffer);
+ }
+ jsonGenerator.writeBinary(toByteArray(newBuffer));
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/jackson/ObjectMappers.java b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/jackson/ObjectMappers.java
new file mode 100644
index 0000000..680a5dc
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/jackson/ObjectMappers.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker.jackson;
+
+
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import org.HdrHistogram.Histogram;
+
+public enum ObjectMappers {
+ DEFAULT;
+
+ private static final ObjectMapper mapper =
+ new ObjectMapper().configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+ static {
+ mapper.enable(DeserializationFeature.READ_UNKNOWN_ENUM_VALUES_USING_DEFAULT_VALUE);
+ SimpleModule module = new SimpleModule();
+ module.addSerializer(Histogram.class, new HistogramSerializer());
+ module.addDeserializer(Histogram.class, new HistogramDeserializer());
+ mapper.registerModule(module);
+ }
+
+ private static final ObjectWriter writer = mapper.writerWithDefaultPrettyPrinter();
+
+ public ObjectMapper mapper() {
+ return mapper;
+ }
+
+ public ObjectWriter writer() {
+ return writer;
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/main/resources/log4j2.yaml b/openmessaging-benchmark/benchmark-framework/src/main/resources/log4j2.yaml
new file mode 100644
index 0000000..3083a28
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/main/resources/log4j2.yaml
@@ -0,0 +1,42 @@
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+Configuration:
+ status: INFO
+ name: messaging-benchmark
+
+ Appenders:
+ Console:
+ name: Console
+ target: SYSTEM_OUT
+ PatternLayout:
+ Pattern: "%d{HH:mm:ss.SSS} [%t] %-4level %c{1} - %msg%n"
+ RollingFile:
+ name: RollingFile
+ fileName: benchmark-worker.log
+ filePattern: benchmark-worker.log.%d{yyyy-MM-dd-hh-mm-ss}.gz
+ PatternLayout:
+ Pattern: "%d{HH:mm:ss.SSS} [%t] %-4level %c{1} - %msg%n"
+ Policies:
+ SizeBasedTriggeringPolicy:
+ size: 100MB
+ DefaultRollOverStrategy:
+ max: 10
+ Loggers:
+ Root:
+ level: info
+ additivity: false
+ AppenderRef:
+ - ref: Console
+ - ref: RollingFile
diff --git a/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/RateControllerTest.java b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/RateControllerTest.java
new file mode 100644
index 0000000..6f014f0
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/RateControllerTest.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.jupiter.api.Test;
+
+class RateControllerTest {
+ private final RateController rateController = new RateController();
+ private double rate = 10_000;
+ private long periodNanos = SECONDS.toNanos(1);
+
+ @Test
+ void receiveBacklog() {
+ assertThat(rateController.getRampingFactor()).isEqualTo(1);
+
+ // no backlog
+ rate = rateController.nextRate(rate, periodNanos, 10_000, 10_000);
+ assertThat(rate).isEqualTo(20_000);
+ assertThat(rateController.getRampingFactor()).isEqualTo(1);
+
+ // receive backlog
+ rate = rateController.nextRate(rate, periodNanos, 20_000, 15_000);
+ assertThat(rate).isEqualTo(5_000);
+ assertThat(rateController.getRampingFactor()).isEqualTo(0.5);
+ }
+
+ @Test
+ void publishBacklog() {
+ assertThat(rateController.getRampingFactor()).isEqualTo(1);
+
+ // no backlog
+ rate = rateController.nextRate(rate, periodNanos, 10_000, 10_000);
+ assertThat(rate).isEqualTo(20_000);
+ assertThat(rateController.getRampingFactor()).isEqualTo(1);
+
+ // publish backlog
+ rate = rateController.nextRate(rate, periodNanos, 15_000, 20_000);
+ assertThat(rate).isEqualTo(5_000);
+ assertThat(rateController.getRampingFactor()).isEqualTo(0.5);
+ }
+
+ @Test
+ void rampUp() {
+ assertThat(rateController.getRampingFactor()).isEqualTo(1);
+
+ // receive backlog
+ rate = rateController.nextRate(rate, periodNanos, 10_000, 5_000);
+ assertThat(rate).isEqualTo(5_000);
+ assertThat(rateController.getRampingFactor()).isEqualTo(0.5);
+
+ // no backlog
+ rate = rateController.nextRate(rate, periodNanos, 20_000, 20_000);
+ assertThat(rate).isEqualTo(10_000);
+ assertThat(rateController.getRampingFactor()).isEqualTo(1);
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/utils/EnvTest.java b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/utils/EnvTest.java
new file mode 100644
index 0000000..cf62b85
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/utils/EnvTest.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.utils;
+
+import static com.github.stefanbirkner.systemlambda.SystemLambda.withEnvironmentVariable;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.jupiter.api.Test;
+
+class EnvTest {
+ private static final String ENV_KEY = "KEY";
+
+ @Test
+ void envLong() throws Exception {
+ withEnvironmentVariable(ENV_KEY, "2")
+ .execute(
+ () -> {
+ assertThat(Env.getLong(ENV_KEY, 1)).isEqualTo(2);
+ });
+ }
+
+ @Test
+ void envLongDefault() throws Exception {
+ withEnvironmentVariable(ENV_KEY, null)
+ .execute(
+ () -> {
+ assertThat(Env.getLong(ENV_KEY, 1)).isEqualTo(1);
+ });
+ }
+
+ @Test
+ void envDouble() throws Exception {
+ withEnvironmentVariable(ENV_KEY, "2.34")
+ .execute(
+ () -> {
+ assertThat(Env.getDouble(ENV_KEY, 1.23)).isEqualTo(2.34);
+ });
+ }
+
+ @Test
+ void envDoubleDefault() throws Exception {
+ withEnvironmentVariable(ENV_KEY, null)
+ .execute(
+ () -> {
+ assertThat(Env.getDouble(ENV_KEY, 1.23)).isEqualTo(1.23);
+ });
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/utils/ListPartitionTest.java b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/utils/ListPartitionTest.java
new file mode 100644
index 0000000..a7472df
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/utils/ListPartitionTest.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.utils;
+
+import static java.util.Arrays.asList;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.List;
+import org.junit.jupiter.api.Test;
+
+class ListPartitionTest {
+
+ @Test
+ void partitionList() {
+ List list = asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
+ List> lists = ListPartition.partitionList(list, 3);
+ assertThat(lists)
+ .satisfies(
+ s -> {
+ assertThat(s).hasSize(3);
+ assertThat(s.get(0)).isEqualTo(asList(1, 4, 7, 10));
+ assertThat(s.get(1)).isEqualTo(asList(2, 5, 8));
+ assertThat(s.get(2)).isEqualTo(asList(3, 6, 9));
+ });
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/utils/PaddingDecimalFormatTest.java b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/utils/PaddingDecimalFormatTest.java
new file mode 100644
index 0000000..be1b9a5
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/utils/PaddingDecimalFormatTest.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.utils;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.jupiter.api.Test;
+
+class PaddingDecimalFormatTest {
+
+ @Test
+ void format() {
+ PaddingDecimalFormat format = new PaddingDecimalFormat("0.0", 7);
+ assertThat(format.format(1L)).isEqualTo(" 1.0");
+ assertThat(format.format(1000L)).isEqualTo(" 1000.0");
+ assertThat(format.format(10000000L)).isEqualTo("10000000.0");
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/utils/TimerTest.java b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/utils/TimerTest.java
new file mode 100644
index 0000000..87004e3
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/utils/TimerTest.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.utils;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.function.Supplier;
+import org.junit.jupiter.api.Test;
+
+class TimerTest {
+
+ @Test
+ void elapsedMillis() {
+ Supplier mockClock = mock(Supplier.class);
+ when(mockClock.get()).thenReturn(MILLISECONDS.toNanos(1), MILLISECONDS.toNanos(3));
+ Timer timer = new Timer(mockClock);
+ assertThat(timer.elapsedMillis()).isEqualTo(2.0d);
+ }
+
+ @Test
+ void elapsedSeconds() {
+ Supplier mockClock = mock(Supplier.class);
+ when(mockClock.get()).thenReturn(SECONDS.toNanos(1), SECONDS.toNanos(3));
+ Timer timer = new Timer(mockClock);
+ assertThat(timer.elapsedSeconds()).isEqualTo(2.0d);
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/utils/UniformRateLimiterTest.java b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/utils/UniformRateLimiterTest.java
new file mode 100644
index 0000000..a9b9046
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/utils/UniformRateLimiterTest.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.utils;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatCode;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.function.Supplier;
+import org.junit.jupiter.api.Test;
+
+class UniformRateLimiterTest {
+
+ @Test
+ void getOpsPerSec() {
+ assertThat(new UniformRateLimiter(1000).getOpsPerSec()).isEqualTo(1000.0d);
+ }
+
+ @Test
+ void getIntervalNs() {
+ assertThat(new UniformRateLimiter(1000).getIntervalNs()).isEqualTo(SECONDS.toNanos(1) / 1000);
+ }
+
+ @Test
+ void acquireSlowSingleThread() {
+ Supplier mockClock = mock(Supplier.class);
+ when(mockClock.get()).thenReturn(SECONDS.toNanos(2));
+ UniformRateLimiter rateLimiter = new UniformRateLimiter(1000, mockClock);
+ assertThat(rateLimiter.acquire()).isEqualTo(2000000000L);
+ assertThat(rateLimiter.acquire()).isEqualTo(2001000000L);
+ assertThat(rateLimiter.acquire()).isEqualTo(2002000000L);
+ }
+
+ @Test
+ void uninterruptibleSleepNs() {
+ long start = System.nanoTime();
+ long expectedEnd = start + MILLISECONDS.toNanos(100);
+ UniformRateLimiter.uninterruptibleSleepNs(expectedEnd);
+ long end = System.nanoTime();
+ assertThat(end).isGreaterThan(expectedEnd);
+ }
+
+ @Test
+ void cinitExceptions() {
+ assertThatCode(() -> new UniformRateLimiter(Double.NaN))
+ .isInstanceOf(IllegalArgumentException.class);
+ assertThatCode(() -> new UniformRateLimiter(1.0d / 0.0d))
+ .isInstanceOf(IllegalArgumentException.class);
+ assertThatCode(() -> new UniformRateLimiter(-0.1)).isInstanceOf(IllegalArgumentException.class);
+ assertThatCode(() -> new UniformRateLimiter(0.0)).isInstanceOf(IllegalArgumentException.class);
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/worker/DistributedWorkersEnsembleTest.java b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/worker/DistributedWorkersEnsembleTest.java
new file mode 100644
index 0000000..c6f4805
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/worker/DistributedWorkersEnsembleTest.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.List;
+import java.util.stream.Stream;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+class DistributedWorkersEnsembleTest {
+
+ @ParameterizedTest
+ @MethodSource("producerCountExpectations")
+ void getNumberOfProducerWorkers(int workerCount, boolean extraConsumers, int expected) {
+ List workers = mock(List.class);
+ when(workers.size()).thenReturn(workerCount);
+ assertThat(DistributedWorkersEnsemble.getNumberOfProducerWorkers(workers, extraConsumers))
+ .isEqualTo(expected);
+ }
+
+ private static Stream producerCountExpectations() {
+ return Stream.of(
+ Arguments.of(2, true, 1),
+ Arguments.of(3, true, 1),
+ Arguments.of(4, true, 2),
+ Arguments.of(5, true, 2),
+ Arguments.of(6, true, 2),
+ Arguments.of(7, true, 3),
+ Arguments.of(8, true, 3),
+ Arguments.of(9, true, 3),
+ Arguments.of(2, false, 1),
+ Arguments.of(3, false, 1),
+ Arguments.of(4, false, 2),
+ Arguments.of(5, false, 2),
+ Arguments.of(6, false, 3),
+ Arguments.of(7, false, 3),
+ Arguments.of(8, false, 4),
+ Arguments.of(9, false, 4));
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/worker/commands/CountersStatsTest.java b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/worker/commands/CountersStatsTest.java
new file mode 100644
index 0000000..699304d
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/worker/commands/CountersStatsTest.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker.commands;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.jupiter.api.Test;
+
+class CountersStatsTest {
+
+ @Test
+ void plus() {
+ CountersStats one = new CountersStats();
+ one.messagesSent = 1;
+ one.messageSendErrors = 10;
+ one.messagesReceived = 100;
+ CountersStats two = new CountersStats();
+ two.messagesSent = 2;
+ two.messageSendErrors = 20;
+ two.messagesReceived = 200;
+
+ CountersStats result = one.plus(two);
+ assertThat(result)
+ .satisfies(
+ r -> {
+ assertThat(r.messagesSent).isEqualTo(3);
+ assertThat(r.messageSendErrors).isEqualTo(30);
+ assertThat(r.messagesReceived).isEqualTo(300);
+ });
+ }
+
+ @Test
+ void zeroPlus() {
+ CountersStats zero = new CountersStats();
+ CountersStats two = new CountersStats();
+ two.messagesSent = 2;
+ two.messageSendErrors = 20;
+ two.messagesReceived = 200;
+
+ CountersStats result = zero.plus(two);
+ assertThat(result)
+ .satisfies(
+ r -> {
+ assertThat(r.messagesSent).isEqualTo(2);
+ assertThat(r.messageSendErrors).isEqualTo(20);
+ assertThat(r.messagesReceived).isEqualTo(200);
+ });
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/worker/commands/CumulativeLatenciesTest.java b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/worker/commands/CumulativeLatenciesTest.java
new file mode 100644
index 0000000..5919739
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/worker/commands/CumulativeLatenciesTest.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker.commands;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.jupiter.api.Test;
+
+class CumulativeLatenciesTest {
+
+ @Test
+ void zeroPlus() {
+ CumulativeLatencies one = new CumulativeLatencies();
+ CumulativeLatencies two = new CumulativeLatencies();
+ CumulativeLatencies result = one.plus(two);
+ assertThat(result)
+ .satisfies(
+ r -> {
+ assertThat(r.publishLatency).isEqualTo(two.publishLatency);
+ assertThat(r.publishDelayLatency).isEqualTo(two.publishDelayLatency);
+ assertThat(r.endToEndLatency).isEqualTo(two.endToEndLatency);
+ });
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/worker/commands/PeriodStatsTest.java b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/worker/commands/PeriodStatsTest.java
new file mode 100644
index 0000000..01c3e04
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/worker/commands/PeriodStatsTest.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker.commands;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.jupiter.api.Test;
+
+class PeriodStatsTest {
+
+ @Test
+ void plus() {
+ PeriodStats one = new PeriodStats();
+ one.messagesSent = 1;
+ one.messageSendErrors = 2;
+ one.bytesSent = 3;
+ one.messagesReceived = 4;
+ one.bytesReceived = 5;
+ one.totalMessagesSent = 6;
+ one.totalMessageSendErrors = 7;
+ one.totalMessagesReceived = 8;
+ PeriodStats two = new PeriodStats();
+ two.messagesSent = 10;
+ two.messageSendErrors = 20;
+ two.bytesSent = 30;
+ two.messagesReceived = 40;
+ two.bytesReceived = 50;
+ two.totalMessagesSent = 60;
+ two.totalMessageSendErrors = 70;
+ two.totalMessagesReceived = 80;
+ PeriodStats result = one.plus(two);
+ assertThat(result)
+ .satisfies(
+ r -> {
+ assertThat(r.messagesSent).isEqualTo(11);
+ assertThat(r.messageSendErrors).isEqualTo(22);
+ assertThat(r.bytesSent).isEqualTo(33);
+ assertThat(r.messagesReceived).isEqualTo(44);
+ assertThat(r.bytesReceived).isEqualTo(55);
+ assertThat(r.totalMessagesSent).isEqualTo(66);
+ assertThat(r.totalMessageSendErrors).isEqualTo(77);
+ assertThat(r.totalMessagesReceived).isEqualTo(88);
+
+ two.publishLatency.add(one.publishLatency);
+ two.publishDelayLatency.add(one.publishDelayLatency);
+ two.endToEndLatency.add(one.endToEndLatency);
+
+ assertThat(r.publishLatency).isEqualTo(two.publishLatency);
+ assertThat(r.publishDelayLatency).isEqualTo(two.publishDelayLatency);
+ assertThat(r.endToEndLatency).isEqualTo(two.endToEndLatency);
+ });
+ }
+
+ @Test
+ void zeroPlus() {
+ PeriodStats one = new PeriodStats();
+ PeriodStats two = new PeriodStats();
+ two.messagesSent = 10;
+ two.messageSendErrors = 20;
+ two.bytesSent = 30;
+ two.messagesReceived = 40;
+ two.bytesReceived = 50;
+ two.totalMessagesSent = 60;
+ two.totalMessageSendErrors = 70;
+ two.totalMessagesReceived = 80;
+ PeriodStats result = one.plus(two);
+ assertThat(result)
+ .satisfies(
+ r -> {
+ assertThat(r.messagesSent).isEqualTo(10);
+ assertThat(r.messageSendErrors).isEqualTo(20);
+ assertThat(r.bytesSent).isEqualTo(30);
+ assertThat(r.messagesReceived).isEqualTo(40);
+ assertThat(r.bytesReceived).isEqualTo(50);
+ assertThat(r.totalMessagesSent).isEqualTo(60);
+ assertThat(r.totalMessageSendErrors).isEqualTo(70);
+ assertThat(r.totalMessagesReceived).isEqualTo(80);
+
+ assertThat(r.publishLatency).isEqualTo(two.publishLatency);
+ assertThat(r.publishDelayLatency).isEqualTo(two.publishDelayLatency);
+ assertThat(r.endToEndLatency).isEqualTo(two.endToEndLatency);
+ });
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/worker/jackson/HistogramSerDeTest.java b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/worker/jackson/HistogramSerDeTest.java
new file mode 100644
index 0000000..9769eac
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/test/java/io/openmessaging/benchmark/worker/jackson/HistogramSerDeTest.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.worker.jackson;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Random;
+import org.HdrHistogram.Histogram;
+import org.junit.jupiter.api.Test;
+
+class HistogramSerDeTest {
+
+ @Test
+ void deserialize() throws IOException {
+ Histogram value = new Histogram(100_000, 3);
+ value.recordValue(1);
+ value.recordValue(100);
+ value.recordValue(10_000);
+
+ ObjectMapper mapper = new ObjectMapper();
+ SimpleModule module = new SimpleModule();
+ module.addSerializer(Histogram.class, new HistogramSerializer());
+ module.addDeserializer(Histogram.class, new HistogramDeserializer());
+ mapper.registerModule(module);
+
+ byte[] serialized = mapper.writeValueAsBytes(value);
+ Histogram deserialized = mapper.readValue(serialized, Histogram.class);
+
+ assertThat(deserialized).isEqualTo(value);
+ }
+
+ /**
+ * Create a random histogram and insert the given number of samples.
+ *
+ * @param samples the number of samples to record into the histogram
+ * @return a Histogram with the given number of samples
+ */
+ private Histogram randomHisto(int samples) {
+ Random r = new Random(0xBADBEEF);
+ Histogram h = new org.HdrHistogram.Histogram(5);
+ for (int i = 0; i < samples; i++) {
+ h.recordValue(r.nextInt(10000000));
+ }
+
+ return h;
+ }
+
+ byte[] serializeRandomHisto(int samples, int initialBufferSize) throws Exception {
+ ByteBuffer inbuffer = ByteBuffer.allocate(initialBufferSize);
+ Histogram inHisto = randomHisto(samples);
+ byte[] serialBytes =
+ HistogramSerializer.toByteArray(HistogramSerializer.serializeHistogram(inHisto, inbuffer));
+
+ // check roundtrip
+ Histogram outHisto =
+ Histogram.decodeFromCompressedByteBuffer(ByteBuffer.wrap(serialBytes), Long.MIN_VALUE);
+ assertThat(inHisto).isEqualTo(outHisto);
+
+ return serialBytes;
+ }
+
+ @Test
+ public void testHistogram() throws Exception {
+
+ // in the worker it's 8 MB but it takes a while to make a histogram that big
+ final int bufSize = 1002;
+
+ int samples = 300;
+
+ // we do an exponential search to fit the crossover point where we need to grow the buffer
+ while (true) {
+ byte[] serialBytes = serializeRandomHisto(samples, bufSize);
+ // System.out.println("Samples: " + samples + ", histogram size: " + serialBytes.length);
+ if (serialBytes.length >= bufSize) {
+ break;
+ }
+ samples *= 1.05;
+ }
+
+ // then walk backwards across the point linearly with increment of 1 to check the boundary
+ // carefully
+ while (true) {
+ samples--;
+ byte[] serialBytes = serializeRandomHisto(samples, bufSize);
+ // System.out.println("Samples: " + samples + ", histogram size: " + serialBytes.length);
+ if (serialBytes.length < bufSize - 10) {
+ break;
+ }
+ }
+ }
+}
diff --git a/openmessaging-benchmark/benchmark-framework/src/test/resources/log4j2.yaml b/openmessaging-benchmark/benchmark-framework/src/test/resources/log4j2.yaml
new file mode 100644
index 0000000..7ed3f31
--- /dev/null
+++ b/openmessaging-benchmark/benchmark-framework/src/test/resources/log4j2.yaml
@@ -0,0 +1,33 @@
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+Configuration:
+ status: INFO
+ name: messaging-benchmark
+
+ Appenders:
+ Console:
+ name: Console
+ target: SYSTEM_OUT
+ PatternLayout:
+ Pattern: "%d{HH:mm:ss.SSS} [%t] %-4level %c{1} - %msg%n"
+ Loggers:
+ Logger:
+ - name: io.openmessaging
+ level: debug
+ Root:
+ level: info
+ additivity: false
+ AppenderRef:
+ - ref: Console
diff --git a/openmessaging-benchmark/bin/benchmark b/openmessaging-benchmark/bin/benchmark
new file mode 100755
index 0000000..9954f63
--- /dev/null
+++ b/openmessaging-benchmark/bin/benchmark
@@ -0,0 +1,30 @@
+#!/bin/bash
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+if [ -d "./lib" ]; then
+ CLASSPATH=$CLASSPATH:lib/*
+else
+ CLASSPATH=benchmark-framework/target/classes:`cat benchmark-framework/target/classpath.txt`
+fi
+
+if [ -z "$HEAP_OPTS" ]
+then
+ HEAP_OPTS="-Xms4G -Xmx4G"
+fi
+
+JVM_MEM="${HEAP_OPTS} -XX:+UseG1GC"
+JVM_GC_LOG=" -XX:+PrintGCDetails -XX:+PrintGCApplicationStoppedTime -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=5 -XX:GCLogFileSize=64m -Xloggc:/dev/shm/benchmark-client-gc_%p.log"
+
+java -server -cp $CLASSPATH $JVM_MEM io.openmessaging.benchmark.Benchmark $*
diff --git a/openmessaging-benchmark/bin/benchmark-local b/openmessaging-benchmark/bin/benchmark-local
new file mode 100755
index 0000000..74ab495
--- /dev/null
+++ b/openmessaging-benchmark/bin/benchmark-local
@@ -0,0 +1,46 @@
+#!/bin/bash
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+if [ -z "$HEAP_OPTS" ]
+then
+ HEAP_OPTS="-Xms4G -Xmx4G"
+fi
+
+JVM_MEM="${HEAP_OPTS} -XX:+UseG1GC"
+JVM_OPENS="--add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.lang=ALL-UNNAMED"
+JVM_GC_LOG=" -XX:+PrintGCDetails -XX:+PrintGCApplicationStoppedTime -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=5 -XX:GCLogFileSize=64m -Xloggc:/dev/shm/benchmark-client-gc_%p.log"
+
+# Load JVM options from jvm.config file if it exists
+JVM_OPTS=""
+if [ -f "jvm.config" ]; then
+ while IFS= read -r line; do
+ # Skip empty lines and comments
+ if [[ -n "$line" && ! "$line" =~ ^[[:space:]]*# ]]; then
+ JVM_OPTS="$JVM_OPTS $line"
+ fi
+ done < jvm.config
+fi
+
+# Build classpath: framework fat jar + all driver jars and their dependencies
+CLASSPATH="benchmark-framework/target/benchmark-framework-0.0.1-SNAPSHOT-jar-with-dependencies.jar"
+for dir in driver-*/target; do
+ if [ -d "$dir" ]; then
+ for jar in "$dir"/*.jar "$dir"/dependency/*.jar; do
+ [ -f "$jar" ] && CLASSPATH="$CLASSPATH:$jar"
+ done
+ fi
+done
+
+java -server $JVM_OPTS $JVM_MEM $JVM_OPENS -cp "$CLASSPATH" io.openmessaging.benchmark.Benchmark $*
diff --git a/openmessaging-benchmark/bin/benchmark-worker b/openmessaging-benchmark/bin/benchmark-worker
new file mode 100755
index 0000000..ce81e0c
--- /dev/null
+++ b/openmessaging-benchmark/bin/benchmark-worker
@@ -0,0 +1,30 @@
+#!/bin/bash
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+if [ -d "./lib" ]; then
+ CLASSPATH=$CLASSPATH:lib/*
+else
+ CLASSPATH=benchmark-framework/target/classes:`cat benchmark-framework/target/classpath.txt`
+fi
+
+if [ -z "$HEAP_OPTS" ]
+then
+ HEAP_OPTS="-Xms4G -Xmx4G"
+fi
+
+JVM_MEM="${HEAP_OPTS} -XX:+UseG1GC"
+JVM_GC_LOG=" -XX:+PrintGCDetails -XX:+PrintGCApplicationStoppedTime -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=5 -XX:GCLogFileSize=64m -Xloggc:/dev/shm/benchmark-client-gc_%p.log"
+
+exec java -server -cp $CLASSPATH $JVM_MEM io.openmessaging.benchmark.worker.BenchmarkWorker $*
diff --git a/openmessaging-benchmark/bin/benchmark-worker-local b/openmessaging-benchmark/bin/benchmark-worker-local
new file mode 100755
index 0000000..84b99e9
--- /dev/null
+++ b/openmessaging-benchmark/bin/benchmark-worker-local
@@ -0,0 +1,35 @@
+#!/bin/bash
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+if [ -z "$HEAP_OPTS" ]
+then
+ HEAP_OPTS="-Xms4G -Xmx4G"
+fi
+
+JVM_MEM="${HEAP_OPTS} -XX:+UseG1GC"
+JVM_GC_LOG=" -XX:+PrintGCDetails -XX:+PrintGCApplicationStoppedTime -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=5 -XX:GCLogFileSize=64m -Xloggc:/dev/shm/benchmark-client-gc_%p.log"
+
+# Load JVM options from jvm.config file if it exists
+JVM_OPTS=""
+if [ -f "jvm.config" ]; then
+ while IFS= read -r line; do
+ # Skip empty lines and comments
+ if [[ -n "$line" && ! "$line" =~ ^[[:space:]]*# ]]; then
+ JVM_OPTS="$JVM_OPTS $line"
+ fi
+ done < jvm.config
+fi
+
+exec java -server $JVM_OPTS $JVM_MEM -cp benchmark-framework/target/benchmark-framework-0.0.1-SNAPSHOT-jar-with-dependencies.jar io.openmessaging.benchmark.worker.BenchmarkWorker $*
diff --git a/openmessaging-benchmark/driver-api/pom.xml b/openmessaging-benchmark/driver-api/pom.xml
new file mode 100644
index 0000000..77a1803
--- /dev/null
+++ b/openmessaging-benchmark/driver-api/pom.xml
@@ -0,0 +1,59 @@
+
+
+
+ 4.0.0
+
+ io.openmessaging.benchmark
+ messaging-benchmark
+ 0.0.1-SNAPSHOT
+
+
+ driver-api
+
+
+ ${project.basedir}/..
+
+
+
+
+ com.fasterxml.jackson.core
+ jackson-annotations
+
+
+ com.fasterxml.jackson.dataformat
+ jackson-dataformat-yaml
+
+
+ com.fasterxml.jackson.jaxrs
+ jackson-jaxrs-base
+
+
+ com.fasterxml.jackson.jaxrs
+ jackson-jaxrs-json-provider
+
+
+ org.apache.bookkeeper.stats
+ bookkeeper-stats-api
+ ${bookkeeper.version}
+
+
+ org.projectlombok
+ lombok
+
+
+
diff --git a/openmessaging-benchmark/driver-api/src/main/java/io/openmessaging/benchmark/driver/BenchmarkConsumer.java b/openmessaging-benchmark/driver-api/src/main/java/io/openmessaging/benchmark/driver/BenchmarkConsumer.java
new file mode 100644
index 0000000..a8e745e
--- /dev/null
+++ b/openmessaging-benchmark/driver-api/src/main/java/io/openmessaging/benchmark/driver/BenchmarkConsumer.java
@@ -0,0 +1,16 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.driver;
+
+public interface BenchmarkConsumer extends AutoCloseable {}
diff --git a/openmessaging-benchmark/driver-api/src/main/java/io/openmessaging/benchmark/driver/BenchmarkDriver.java b/openmessaging-benchmark/driver-api/src/main/java/io/openmessaging/benchmark/driver/BenchmarkDriver.java
new file mode 100644
index 0000000..a762261
--- /dev/null
+++ b/openmessaging-benchmark/driver-api/src/main/java/io/openmessaging/benchmark/driver/BenchmarkDriver.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.driver;
+
+import static java.util.stream.Collectors.toList;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import lombok.Value;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/** Base driver interface. */
+public interface BenchmarkDriver extends AutoCloseable {
+ /**
+ * Driver implementation can use this method to initialize the client libraries, with the provided
+ * configuration file.
+ *
+ *
The format of the configuration file is specific to the driver implementation.
+ *
+ * @param configurationFile
+ * @param statsLogger stats logger to collect stats from benchmark driver
+ * @throws IOException
+ */
+ void initialize(File configurationFile, StatsLogger statsLogger)
+ throws IOException, InterruptedException;
+
+ /**
+ * Get a driver specific prefix to be used in creating multiple topic names.
+ *
+ * @return the topic name prefix
+ */
+ String getTopicNamePrefix();
+
+ /**
+ * Create a new topic with a given number of partitions.
+ *
+ * @param topic
+ * @param partitions
+ * @return a future the completes when the topic is created
+ */
+ CompletableFuture createTopic(String topic, int partitions);
+
+ /**
+ * Create a list of new topics with the given number of partitions.
+ *
+ * @param topicInfos
+ * @return a future the completes when the topics are created
+ */
+ default CompletableFuture createTopics(List topicInfos) {
+ @SuppressWarnings("unchecked")
+ CompletableFuture[] futures =
+ topicInfos.stream()
+ .map(topicInfo -> createTopic(topicInfo.getTopic(), topicInfo.getPartitions()))
+ .toArray(CompletableFuture[]::new);
+ return CompletableFuture.allOf(futures);
+ }
+
+ /**
+ * Create a producer for a given topic.
+ *
+ * @param topic
+ * @return a producer future
+ */
+ CompletableFuture createProducer(String topic);
+
+ /**
+ * Create a producers for a given topic.
+ *
+ * @param producers
+ * @return a producers future
+ */
+ default CompletableFuture> createProducers(List producers) {
+ List> futures =
+ producers.stream().map(ci -> createProducer(ci.getTopic())).collect(toList());
+ return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]))
+ .thenApply(v -> futures.stream().map(CompletableFuture::join).collect(toList()));
+ }
+
+ /**
+ * Create a benchmark consumer relative to one particular topic and subscription.
+ *
+ *
It is responsibility of the driver implementation to invoke the consumerCallback
+ * each time a message is received.
+ *
+ * @param id
+ * @param partitionsPerTopic
+ * @param partitionsPerSubscription
+ * @param topic
+ * @param subscriptionName
+ * @param consumerCallback
+ * @return a consumer future
+ */
+ CompletableFuture createConsumer(
+ int id,
+ int partitionsPerTopic,
+ int partitionsPerSubscription,
+ String topic,
+ String subscriptionName,
+ ConsumerCallback consumerCallback);
+
+ /**
+ * Create a consumers for a given topic.
+ *
+ * @param consumers
+ * @return a consumers future
+ */
+ default CompletableFuture> createConsumers(List consumers) {
+ List> futures =
+ consumers.stream()
+ .map(
+ ci ->
+ createConsumer(
+ ci.getId(),
+ ci.getPartitionsPerTopic(),
+ ci.getPartitionsPerSubscription(),
+ ci.getTopic(),
+ ci.getSubscriptionName(),
+ ci.getConsumerCallback()))
+ .collect(toList());
+ return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]))
+ .thenApply(v -> futures.stream().map(CompletableFuture::join).collect(toList()));
+ }
+
+ @Value
+ class TopicInfo {
+ String topic;
+ int partitions;
+ }
+
+ @Value
+ class ProducerInfo {
+ int id;
+ String topic;
+ }
+
+ @Value
+ class ConsumerInfo {
+ int id;
+ int partitionsPerTopic;
+ int partitionsPerSubscription;
+ String topic;
+ String subscriptionName;
+ ConsumerCallback consumerCallback;
+ }
+}
diff --git a/openmessaging-benchmark/driver-api/src/main/java/io/openmessaging/benchmark/driver/BenchmarkProducer.java b/openmessaging-benchmark/driver-api/src/main/java/io/openmessaging/benchmark/driver/BenchmarkProducer.java
new file mode 100644
index 0000000..1662cf1
--- /dev/null
+++ b/openmessaging-benchmark/driver-api/src/main/java/io/openmessaging/benchmark/driver/BenchmarkProducer.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.driver;
+
+
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+
+public interface BenchmarkProducer extends AutoCloseable {
+
+ /**
+ * Publish a message and return a callback to track the completion of the operation.
+ *
+ * @param key the key associated with this message
+ * @param payload the message payload
+ * @return a future that will be triggered when the message is successfully published
+ */
+ CompletableFuture sendAsync(Optional key, byte[] payload);
+}
diff --git a/openmessaging-benchmark/driver-api/src/main/java/io/openmessaging/benchmark/driver/ConsumerCallback.java b/openmessaging-benchmark/driver-api/src/main/java/io/openmessaging/benchmark/driver/ConsumerCallback.java
new file mode 100644
index 0000000..b68ecf4
--- /dev/null
+++ b/openmessaging-benchmark/driver-api/src/main/java/io/openmessaging/benchmark/driver/ConsumerCallback.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.driver;
+
+
+import java.nio.ByteBuffer;
+
+/** Callback that the driver implementation calls when a message is received. */
+public interface ConsumerCallback {
+ /**
+ * Driver should invoke this method (or the ByteBuffer variant) once for each message received.
+ *
+ * @param payload the received message payload
+ * @param publishTimestamp the publish timestamp in milliseconds
+ */
+ void messageReceived(byte[] payload, long publishTimestamp);
+
+ /**
+ * Driver should invoke this method (or the byte[] variant) once for each message received.
+ *
+ * @param payload the received message payload
+ * @param publishTimestamp the publish timestamp in milliseconds
+ */
+ void messageReceived(ByteBuffer payload, long publishTimestamp);
+}
diff --git a/openmessaging-benchmark/driver-api/src/main/java/io/openmessaging/benchmark/driver/ResourceCreator.java b/openmessaging-benchmark/driver-api/src/main/java/io/openmessaging/benchmark/driver/ResourceCreator.java
new file mode 100644
index 0000000..3bdfe53
--- /dev/null
+++ b/openmessaging-benchmark/driver-api/src/main/java/io/openmessaging/benchmark/driver/ResourceCreator.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.driver;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static java.util.stream.Collectors.toMap;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+import lombok.RequiredArgsConstructor;
+import lombok.SneakyThrows;
+import lombok.Value;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+@RequiredArgsConstructor
+public class ResourceCreator {
+ private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor();
+ private final String name;
+ private final int maxBatchSize;
+ private final long interBatchDelayMs;
+ private final Function, Map>> invokeBatchFn;
+ private final Function, CreationResult> complete;
+
+ public CompletableFuture> create(List resources) {
+ return CompletableFuture.completedFuture(createBlocking(resources));
+ }
+
+ private List createBlocking(List resources) {
+ BlockingQueue queue = new ArrayBlockingQueue<>(resources.size(), true, resources);
+ List batch = new ArrayList<>();
+ List created = new ArrayList<>();
+ AtomicInteger succeeded = new AtomicInteger();
+
+ ScheduledFuture> loggingFuture =
+ executor.scheduleAtFixedRate(
+ () -> log.info("Created {}s {}/{}", name, succeeded.get(), resources.size()),
+ 10,
+ 10,
+ SECONDS);
+
+ try {
+ while (succeeded.get() < resources.size()) {
+ int batchSize = queue.drainTo(batch, maxBatchSize);
+ if (batchSize > 0) {
+ executeBatch(batch)
+ .forEach(
+ (resource, result) -> {
+ if (result.success) {
+ created.add(result.created);
+ succeeded.incrementAndGet();
+ } else {
+ //noinspection ResultOfMethodCallIgnored
+ queue.offer(resource);
+ }
+ });
+ batch.clear();
+ }
+ }
+ } finally {
+ loggingFuture.cancel(true);
+ }
+ return created;
+ }
+
+ @SneakyThrows
+ private Map> executeBatch(List batch) {
+ log.debug("Executing batch, size: {}", batch.size());
+ Thread.sleep(interBatchDelayMs);
+ return invokeBatchFn.apply(batch).entrySet().stream()
+ .collect(toMap(Map.Entry::getKey, e -> complete.apply(e.getValue())));
+ }
+
+ @Value
+ public static class CreationResult {
+ C created;
+ boolean success;
+ }
+}
diff --git a/openmessaging-benchmark/driver-fluss/config/fluss.yaml b/openmessaging-benchmark/driver-fluss/config/fluss.yaml
new file mode 100644
index 0000000..619fdaa
--- /dev/null
+++ b/openmessaging-benchmark/driver-fluss/config/fluss.yaml
@@ -0,0 +1,40 @@
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+name: Fluss
+driverClass: io.openmessaging.benchmark.driver.fluss.FlussBenchmarkDriver
+
+# Schema: first field MUST be "long" for E2E timestamp
+schema: "long-int-int-string-string"
+
+# Fluss connection
+bootstrapServers: localhost:9123
+
+# Log format: ARROW or INDEXED
+logFormat: ARROW
+
+# Writer configuration
+writerAcks: all
+writerBatchSize: 1mb
+writerBufferMemory: 32mb
+writerBatchTimeoutMs: 100
+
+# Consumer configuration
+projectFields: 0/1/2
+prefetchNum: 4
+fetchMaxBytes: 16mb
+isCheckCrc: false
+
+# Network
+clientNettyThreads: 1
diff --git a/openmessaging-benchmark/driver-fluss/pom.xml b/openmessaging-benchmark/driver-fluss/pom.xml
new file mode 100644
index 0000000..a703bb1
--- /dev/null
+++ b/openmessaging-benchmark/driver-fluss/pom.xml
@@ -0,0 +1,67 @@
+
+
+
+ 4.0.0
+
+ io.openmessaging.benchmark
+ messaging-benchmark
+ 0.0.1-SNAPSHOT
+
+
+ driver-fluss
+
+ 0.9.0-incubating
+ ${project.basedir}/..
+
+
+
+
+ ${project.groupId}
+ driver-api
+ ${project.version}
+
+
+ org.apache.fluss
+ fluss-client
+ ${fluss.version}
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
+ 3.3.0
+
+
+ copy-dependencies
+
+ copy-dependencies
+
+ package
+
+ ${project.build.directory}/dependency
+ runtime
+
+
+
+
+
+
+
diff --git a/openmessaging-benchmark/driver-fluss/src/main/java/io/openmessaging/benchmark/driver/fluss/FlussBenchmarkConsumer.java b/openmessaging-benchmark/driver-fluss/src/main/java/io/openmessaging/benchmark/driver/fluss/FlussBenchmarkConsumer.java
new file mode 100644
index 0000000..6b0cf9a
--- /dev/null
+++ b/openmessaging-benchmark/driver-fluss/src/main/java/io/openmessaging/benchmark/driver/fluss/FlussBenchmarkConsumer.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.driver.fluss;
+
+import static io.openmessaging.benchmark.driver.fluss.FlussBenchmarkDriver.DEFAULT_DATABASE_NAME;
+
+import io.openmessaging.benchmark.driver.BenchmarkConsumer;
+import io.openmessaging.benchmark.driver.ConsumerCallback;
+import java.time.Duration;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import org.apache.fluss.client.Connection;
+import org.apache.fluss.client.ConnectionFactory;
+import org.apache.fluss.client.table.Table;
+import org.apache.fluss.client.table.scanner.Scan;
+import org.apache.fluss.client.table.scanner.log.LogScanner;
+import org.apache.fluss.client.table.scanner.log.ScanRecords;
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.metadata.TablePath;
+import org.apache.fluss.row.BinaryString;
+import org.apache.fluss.row.InternalRow;
+import org.apache.fluss.types.DataType;
+import org.apache.fluss.types.DataTypeRoot;
+import org.apache.fluss.types.RowType;
+import org.apache.fluss.utils.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Fluss benchmark consumer. */
+public class FlussBenchmarkConsumer implements BenchmarkConsumer {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlussBenchmarkConsumer.class);
+
+ private Connection connection;
+ private final Table table;
+ private final LogScanner logScanner;
+ private final int fieldLength;
+ private final InternalRow.FieldGetter[] flussFieldGetters;
+ private final RowType projectRowType;
+ private final ConsumerCallback callback;
+ private final ExecutorService executor;
+ private final Future> consumerTask;
+ private volatile boolean closing = false;
+
+ public FlussBenchmarkConsumer(
+ Configuration conf,
+ String topic,
+ int[] projectFields,
+ List subscriptionBuckets,
+ ConsumerCallback callback) {
+ this(conf, topic, projectFields, subscriptionBuckets, callback, 100L);
+ }
+
+ public FlussBenchmarkConsumer(
+ Configuration conf,
+ String topic,
+ int[] projectFields,
+ List subscriptionBuckets,
+ ConsumerCallback callback,
+ long pollTimeoutMs) {
+ this.connection = ConnectionFactory.createConnection(conf);
+ this.callback = callback;
+ this.table = connection.getTable(new TablePath(DEFAULT_DATABASE_NAME, topic));
+ RowType originRowType = table.getTableInfo().getRowType();
+ this.logScanner = createLogScanner(table, projectFields, subscriptionBuckets);
+ if (projectFields.length == 0) {
+ this.projectRowType = new RowType(originRowType.getFields());
+ this.fieldLength = originRowType.getFieldCount();
+ } else {
+ this.projectRowType = originRowType.project(projectFields);
+ this.fieldLength = projectRowType.getFieldCount();
+ }
+ this.flussFieldGetters = new InternalRow.FieldGetter[fieldLength];
+ for (int i = 0; i < fieldLength; i++) {
+ flussFieldGetters[i] = InternalRow.createFieldGetter(projectRowType.getTypeAt(i), i);
+ }
+
+ this.executor = Executors.newSingleThreadExecutor();
+ this.consumerTask =
+ this.executor.submit(
+ () -> {
+ while (!closing) {
+ try {
+ ScanRecords records = logScanner.poll(Duration.ofMillis(pollTimeoutMs));
+ records.forEach(record -> processRecord(record.getRow()));
+ } catch (Exception e) {
+ LOG.error("exception occur while consuming message", e);
+ }
+ }
+ });
+ }
+
+ private void processRecord(InternalRow row) {
+ if (fieldLength <= 0) {
+ LOG.warn("No fields available in the row to process");
+ return;
+ }
+ long timeStamp = (long) flussFieldGetters[0].getFieldOrNull(row);
+ // Include all fields (including field 0) in the size calculation
+ // to match the producer's messageSize-based byte reporting.
+ int sizeInBytes = objectSizes(null, projectRowType.getTypeAt(0));
+ for (int i = 1; i < fieldLength; i++) {
+ Object value = flussFieldGetters[i].getFieldOrNull(row);
+ sizeInBytes += objectSizes(value, projectRowType.getTypeAt(i));
+ }
+ callback.messageReceived(new byte[sizeInBytes], timeStamp);
+ }
+
+ private LogScanner createLogScanner(
+ Table table, int[] projectFields, List subscriptionBuckets) {
+ LogScanner logScanner;
+ if (projectFields.length == 0) {
+ logScanner = table.newScan().createLogScanner();
+ } else {
+ // logScanner = new LogScan().withProjectedFields(projectFields);
+ Scan projectScan = table.newScan().project(projectFields);
+ logScanner = projectScan.createLogScanner();
+ }
+ for (int subscriptionBucket : subscriptionBuckets) {
+ logScanner.subscribeFromBeginning(subscriptionBucket);
+ }
+
+ return logScanner;
+ }
+
+ private int objectSizes(Object obj, DataType type) {
+ DataTypeRoot typeRoot = type.getTypeRoot();
+ if (typeRoot == DataTypeRoot.INTEGER) {
+ return 4;
+ } else if (typeRoot == DataTypeRoot.BIGINT) {
+ return 8;
+ } else if (typeRoot == DataTypeRoot.STRING) {
+ if (obj instanceof BinaryString) {
+ return ((BinaryString) obj).getSizeInBytes();
+ } else if (obj instanceof byte[]) {
+ return ((byte[]) obj).length;
+ } else {
+ LOG.warn("Unexpected object type for STRING field: {}", obj.getClass().getName());
+ return String.valueOf(obj).getBytes(java.nio.charset.StandardCharsets.UTF_8).length;
+ }
+ } else {
+ throw new UnsupportedOperationException("Unsupported type: " + typeRoot);
+ }
+ }
+
+ @Override
+ public void close() throws Exception {
+ closing = true;
+ executor.shutdown();
+ consumerTask.get();
+ logScanner.close();
+
+ IOUtils.closeQuietly(table);
+
+ try {
+ if (connection != null) {
+ connection.close();
+ }
+ } catch (Exception e) {
+ LOG.warn("Exception occurs while closing Fluss Connection.", e);
+ }
+ connection = null;
+ }
+}
diff --git a/openmessaging-benchmark/driver-fluss/src/main/java/io/openmessaging/benchmark/driver/fluss/FlussBenchmarkDriver.java b/openmessaging-benchmark/driver-fluss/src/main/java/io/openmessaging/benchmark/driver/fluss/FlussBenchmarkDriver.java
new file mode 100644
index 0000000..0d91de4
--- /dev/null
+++ b/openmessaging-benchmark/driver-fluss/src/main/java/io/openmessaging/benchmark/driver/fluss/FlussBenchmarkDriver.java
@@ -0,0 +1,253 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.driver.fluss;
+
+import static org.apache.fluss.utils.function.ThrowingConsumer.unchecked;
+
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+import io.openmessaging.benchmark.driver.BenchmarkConsumer;
+import io.openmessaging.benchmark.driver.BenchmarkDriver;
+import io.openmessaging.benchmark.driver.BenchmarkProducer;
+import io.openmessaging.benchmark.driver.ConsumerCallback;
+import java.io.File;
+import java.io.IOException;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.fluss.client.Connection;
+import org.apache.fluss.client.ConnectionFactory;
+import org.apache.fluss.client.admin.Admin;
+import org.apache.fluss.config.ConfigOptions;
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.config.MemorySize;
+import org.apache.fluss.metadata.DatabaseDescriptor;
+import org.apache.fluss.metadata.LogFormat;
+import org.apache.fluss.metadata.Schema;
+import org.apache.fluss.metadata.TableDescriptor;
+import org.apache.fluss.metadata.TablePath;
+import org.apache.fluss.types.DataTypes;
+import org.apache.fluss.types.RowType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Fluss benchmark driver. */
+public class FlussBenchmarkDriver implements BenchmarkDriver {
+ private static final Logger LOG = LoggerFactory.getLogger(FlussBenchmarkDriver.class);
+
+ public static final String DEFAULT_DATABASE_NAME = "benchmarkDb";
+ private static final ObjectWriter writer = new ObjectMapper().writerWithDefaultPrettyPrinter();
+ private static final ObjectMapper mapper =
+ new ObjectMapper(new YAMLFactory())
+ .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+ private FlussConfig config;
+ private Configuration flussConfiguration;
+ private Connection connection;
+ private Admin flussAdmin;
+ private int[] projectFields;
+ private LogFormat logFormat;
+ private Schema schema;
+ private RowType rowType;
+ private String[] schemaTypes;
+
+ private final List producers = Collections.synchronizedList(new ArrayList<>());
+ private final List consumers = Collections.synchronizedList(new ArrayList<>());
+
+ @Override
+ public void initialize(File configurationFile, StatsLogger statsLogger) throws IOException {
+ this.config = mapper.readValue(configurationFile, FlussConfig.class);
+ LOG.info("Fluss driver conf: {}", writer.writeValueAsString(config));
+
+ this.schemaTypes = config.schema.split("-");
+ if (schemaTypes.length == 0 || !schemaTypes[0].equals("long")) {
+ throw new IllegalArgumentException(
+ "Schema must start with 'long' for E2E timestamp. Got: " + config.schema);
+ }
+
+ this.schema = buildSchema(schemaTypes);
+ this.rowType = schema.getRowType();
+ this.flussConfiguration = buildFlussConfiguration(config);
+ this.connection = ConnectionFactory.createConnection(flussConfiguration);
+ this.flussAdmin = connection.getAdmin();
+ }
+
+ @Override
+ public String getTopicNamePrefix() {
+ return "Fluss-Benchmark";
+ }
+
+ @Override
+ public CompletableFuture createTopic(String topic, int partitions) {
+ try {
+ flussAdmin.createDatabase(DEFAULT_DATABASE_NAME, DatabaseDescriptor.EMPTY, true).get();
+ } catch (Exception e) {
+ throw new RuntimeException("Exception occurs while creating Fluss database.", e);
+ }
+
+ TableDescriptor tableDescriptor =
+ TableDescriptor.builder()
+ .schema(schema)
+ .logFormat(logFormat)
+ .distributedBy(partitions)
+ .build();
+ return flussAdmin.createTable(
+ new TablePath(DEFAULT_DATABASE_NAME, topic), tableDescriptor, false);
+ }
+
+ @Override
+ public CompletableFuture createProducer(String topic) {
+ try {
+ BenchmarkProducer benchmarkProducer =
+ new FlussBenchmarkProducer(flussConfiguration, topic, schemaTypes, rowType);
+ producers.add(benchmarkProducer);
+ return CompletableFuture.completedFuture(benchmarkProducer);
+ } catch (Throwable t) {
+ CompletableFuture future = new CompletableFuture<>();
+ future.completeExceptionally(t);
+ return future;
+ }
+ }
+
+ @Override
+ public CompletableFuture createConsumer(
+ int id,
+ int partitionsPerTopic,
+ int partitionsPerSubscription,
+ String topic,
+ String subscriptionName,
+ ConsumerCallback consumerCallback) {
+ try {
+ List subscriptionBuckets =
+ calculateSubscriptionBuckets(id, partitionsPerTopic, partitionsPerSubscription);
+ FlussBenchmarkConsumer consumer =
+ new FlussBenchmarkConsumer(
+ flussConfiguration, topic, projectFields, subscriptionBuckets, consumerCallback);
+ consumers.add(consumer);
+ return CompletableFuture.completedFuture(consumer);
+ } catch (Throwable t) {
+ CompletableFuture future = new CompletableFuture<>();
+ future.completeExceptionally(t);
+ return future;
+ }
+ }
+
+ @Override
+ public void close() {
+ producers.forEach(unchecked(BenchmarkProducer::close));
+ consumers.forEach(unchecked(BenchmarkConsumer::close));
+
+ try {
+ if (connection != null) {
+ connection.close();
+ }
+ } catch (Exception e) {
+ LOG.warn("Exception occurs while closing Fluss Connection.", e);
+ }
+ connection = null;
+ }
+
+ private static Schema buildSchema(String[] types) {
+ Schema.Builder builder = Schema.newBuilder();
+ for (int i = 0; i < types.length; i++) {
+ String colName = "col" + i;
+ switch (types[i]) {
+ case "int":
+ builder.column(colName, DataTypes.INT());
+ break;
+ case "long":
+ builder.column(colName, DataTypes.BIGINT());
+ break;
+ case "string":
+ builder.column(colName, DataTypes.STRING());
+ break;
+ default:
+ throw new IllegalArgumentException("Unsupported schema type: " + types[i]);
+ }
+ }
+ return builder.build();
+ }
+
+ private Configuration buildFlussConfiguration(FlussConfig config) {
+ Configuration conf = new Configuration();
+ conf.setString(ConfigOptions.BOOTSTRAP_SERVERS.key(), config.bootstrapServers);
+ conf.setString(ConfigOptions.CLIENT_WRITER_ACKS.key(), config.writerAcks);
+ conf.set(ConfigOptions.CLIENT_WRITER_BATCH_SIZE, MemorySize.parse(config.writerBatchSize));
+ conf.set(
+ ConfigOptions.CLIENT_WRITER_BUFFER_MEMORY_SIZE,
+ MemorySize.parse(config.writerBufferMemory));
+ conf.set(
+ ConfigOptions.CLIENT_WRITER_BATCH_TIMEOUT, Duration.ofMillis(config.writerBatchTimeoutMs));
+ conf.setInt(ConfigOptions.NETTY_CLIENT_NUM_NETWORK_THREADS, config.clientNettyThreads);
+ conf.set(
+ ConfigOptions.CLIENT_SCANNER_LOG_FETCH_MAX_BYTES, MemorySize.parse(config.fetchMaxBytes));
+ conf.set(ConfigOptions.CLIENT_SCANNER_REMOTE_LOG_PREFETCH_NUM, config.prefetchNum);
+ conf.setBoolean(ConfigOptions.CLIENT_SCANNER_LOG_CHECK_CRC, config.isCheckCrc);
+
+ if (config.logFormat.equals(LogFormat.INDEXED.toString())) {
+ this.logFormat = LogFormat.INDEXED;
+ } else {
+ this.logFormat = LogFormat.ARROW;
+ }
+
+ String projectFieldString = config.projectFields;
+ if (projectFieldString.equals("all")) {
+ projectFields = new int[0];
+ } else {
+ String[] fields = projectFieldString.split("/");
+ projectFields = new int[fields.length];
+ for (int i = 0; i < fields.length; i++) {
+ projectFields[i] = Integer.parseInt(fields[i]);
+ }
+ }
+
+ return conf;
+ }
+
+ private List calculateSubscriptionBuckets(
+ int id, int partitionsPerTopic, int subscriptionsPerTopic) {
+ List buckets = new ArrayList<>();
+
+ int partitionsPerSubscription = partitionsPerTopic / subscriptionsPerTopic;
+ int extraPartitions = partitionsPerTopic % subscriptionsPerTopic;
+
+ int startBucket;
+ if (id < extraPartitions) {
+ startBucket = id * (partitionsPerSubscription + 1);
+ } else {
+ startBucket = id * partitionsPerSubscription + extraPartitions;
+ }
+
+ int endBucket = startBucket + partitionsPerSubscription - 1;
+ if (id < extraPartitions) {
+ endBucket += 1;
+ }
+
+ for (int i = startBucket; i <= endBucket; i++) {
+ buckets.add(i);
+ }
+ LOG.info(
+ "id: {}, the subscriptionBuckets is {}, total buckets: {}, total consumers: {}",
+ id,
+ buckets,
+ partitionsPerTopic,
+ subscriptionsPerTopic);
+ return buckets;
+ }
+}
diff --git a/openmessaging-benchmark/driver-fluss/src/main/java/io/openmessaging/benchmark/driver/fluss/FlussBenchmarkProducer.java b/openmessaging-benchmark/driver-fluss/src/main/java/io/openmessaging/benchmark/driver/fluss/FlussBenchmarkProducer.java
new file mode 100644
index 0000000..80dddbc
--- /dev/null
+++ b/openmessaging-benchmark/driver-fluss/src/main/java/io/openmessaging/benchmark/driver/fluss/FlussBenchmarkProducer.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.driver.fluss;
+
+import static io.openmessaging.benchmark.driver.fluss.FlussBenchmarkDriver.DEFAULT_DATABASE_NAME;
+
+import io.openmessaging.benchmark.driver.BenchmarkProducer;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.fluss.client.Connection;
+import org.apache.fluss.client.ConnectionFactory;
+import org.apache.fluss.client.table.Table;
+import org.apache.fluss.client.table.writer.AppendResult;
+import org.apache.fluss.client.table.writer.AppendWriter;
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.metadata.TablePath;
+import org.apache.fluss.row.BinaryString;
+import org.apache.fluss.row.InternalRow;
+import org.apache.fluss.row.indexed.IndexedRow;
+import org.apache.fluss.row.indexed.IndexedRowWriter;
+import org.apache.fluss.types.DataType;
+import org.apache.fluss.types.RowType;
+import org.apache.fluss.utils.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Fluss benchmark producer that self-constructs IndexedRow from schema definition. */
+public class FlussBenchmarkProducer implements BenchmarkProducer {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlussBenchmarkProducer.class);
+
+ private final AppendWriter appendWriter;
+ private final String[] schemaTypes;
+ private final DataType[] fieldTypes;
+ private Connection connection;
+ private final Table table;
+
+ /** Fixed byte size for non-string fields, computed from schema. */
+ private final int fixedBytes;
+
+ /** Number of string fields in the schema. */
+ private final int stringCount;
+
+ /** Per-string byte size, lazily computed on first sendAsync call. */
+ private volatile int perStringByteSize = -1;
+
+ public FlussBenchmarkProducer(
+ Configuration conf, String topic, String[] schemaTypes, RowType rowType) {
+ this.schemaTypes = schemaTypes.clone();
+ this.connection = ConnectionFactory.createConnection(conf);
+ this.table = connection.getTable(new TablePath(DEFAULT_DATABASE_NAME, topic));
+ this.appendWriter = table.newAppend().createWriter();
+ this.fieldTypes = rowType.getChildren().toArray(new DataType[0]);
+
+ int fixed = 0;
+ int strings = 0;
+ for (String type : schemaTypes) {
+ switch (type) {
+ case "int":
+ fixed += 4;
+ break;
+ case "long":
+ fixed += 8;
+ break;
+ case "string":
+ strings++;
+ break;
+ default:
+ throw new IllegalArgumentException("Unsupported schema type: " + type);
+ }
+ }
+ this.fixedBytes = fixed;
+ this.stringCount = strings;
+ }
+
+ @Override
+ public CompletableFuture sendAsync(Optional key, byte[] payload) {
+ int targetSize = payload.length;
+
+ // Skip perStringByteSize initialization for probe messages (small payloads).
+ // Only initialize on the first real workload message.
+ if (targetSize < fixedBytes) {
+ InternalRow row = buildRow();
+ CompletableFuture result = appendWriter.append(row);
+ return result.thenApply(appendResult -> null);
+ }
+
+ if (perStringByteSize < 0) {
+ synchronized (this) {
+ if (perStringByteSize < 0) {
+ if (stringCount > 0) {
+ perStringByteSize = (targetSize - fixedBytes) / stringCount;
+ } else {
+ perStringByteSize = 0;
+ }
+ LOG.info(
+ "Computed perStringByteSize={}, targetSize={}, fixedBytes={}, stringCount={}",
+ perStringByteSize,
+ targetSize,
+ fixedBytes,
+ stringCount);
+ }
+ }
+ }
+
+ InternalRow row = buildRow();
+ CompletableFuture result = appendWriter.append(row);
+ return result.thenApply(appendResult -> null);
+ }
+
+ @Override
+ public void close() throws Exception {
+ IOUtils.closeQuietly(table);
+
+ try {
+ if (connection != null) {
+ connection.close();
+ }
+ } catch (Exception e) {
+ LOG.warn("Exception occurs while closing Fluss Connection.", e);
+ }
+ connection = null;
+ }
+
+ private InternalRow buildRow() {
+ IndexedRowWriter rowWriter = new IndexedRowWriter(fieldTypes);
+ for (int i = 0; i < schemaTypes.length; i++) {
+ switch (schemaTypes[i]) {
+ case "long":
+ if (i == 0) {
+ rowWriter.writeLong(System.currentTimeMillis());
+ } else {
+ rowWriter.writeLong(ThreadLocalRandom.current().nextLong());
+ }
+ break;
+ case "int":
+ rowWriter.writeInt(ThreadLocalRandom.current().nextInt());
+ break;
+ case "string":
+ rowWriter.writeString(randomBinaryString(perStringByteSize));
+ break;
+ default:
+ throw new IllegalArgumentException("Unsupported schema type: " + schemaTypes[i]);
+ }
+ }
+ rowWriter.complete();
+ IndexedRow row = new IndexedRow(fieldTypes);
+ row.pointTo(rowWriter.segment(), 0, rowWriter.position());
+ return row;
+ }
+
+ private static BinaryString randomBinaryString(int byteSize) {
+ if (byteSize <= 0) {
+ return BinaryString.EMPTY_UTF8;
+ }
+ byte[] bytes = new byte[byteSize];
+ ThreadLocalRandom.current().nextBytes(bytes);
+ // Use ASCII printable characters to avoid multi-byte encoding surprises
+ for (int i = 0; i < bytes.length; i++) {
+ bytes[i] = (byte) (33 + (bytes[i] & 0x7F) % 94);
+ }
+ return BinaryString.fromBytes(bytes);
+ }
+}
diff --git a/openmessaging-benchmark/driver-fluss/src/main/java/io/openmessaging/benchmark/driver/fluss/FlussConfig.java b/openmessaging-benchmark/driver-fluss/src/main/java/io/openmessaging/benchmark/driver/fluss/FlussConfig.java
new file mode 100644
index 0000000..d3c64e3
--- /dev/null
+++ b/openmessaging-benchmark/driver-fluss/src/main/java/io/openmessaging/benchmark/driver/fluss/FlussConfig.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.openmessaging.benchmark.driver.fluss;
+
+/** Fluss benchmark config. All Fluss-specific configuration lives here. */
+public class FlussConfig {
+
+ /** Fluss coordinator bootstrap servers. */
+ public String bootstrapServers;
+
+ /**
+ * Schema definition, dash-separated type names. First field MUST be "long" (used for E2E
+ * timestamp). Example: "long-int-int-string-string"
+ */
+ public String schema = "long-int-int-string-string";
+
+ /** Log format: "ARROW" or "INDEXED". */
+ public String logFormat = "ARROW";
+
+ /** Writer acknowledgment mode. */
+ public String writerAcks = "all";
+
+ /** Writer batch size (Fluss MemorySize string, e.g. "1mb"). */
+ public String writerBatchSize = "1mb";
+
+ /** Writer buffer memory (Fluss MemorySize string, e.g. "32mb"). */
+ public String writerBufferMemory = "32mb";
+
+ /** Writer batch timeout in milliseconds. */
+ public int writerBatchTimeoutMs = 0;
+
+ /** Number of Netty client threads. */
+ public int clientNettyThreads = 1;
+
+ /** Max bytes per fetch (Fluss MemorySize string, e.g. "16mb"). */
+ public String fetchMaxBytes = "16mb";
+
+ /** Project fields: "all" or slash-separated indices like "0/2/4". */
+ public String projectFields = "all";
+
+ /** Number of remote log segments to prefetch. */
+ public int prefetchNum = 2;
+
+ /** Whether to check CRC on log records. */
+ public boolean isCheckCrc = false;
+}
diff --git a/openmessaging-benchmark/etc/APACHE-2.txt b/openmessaging-benchmark/etc/APACHE-2.txt
new file mode 100644
index 0000000..86a8bee
--- /dev/null
+++ b/openmessaging-benchmark/etc/APACHE-2.txt
@@ -0,0 +1,11 @@
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
\ No newline at end of file
diff --git a/openmessaging-benchmark/etc/checkstyle.xml b/openmessaging-benchmark/etc/checkstyle.xml
new file mode 100644
index 0000000..2d95c2b
--- /dev/null
+++ b/openmessaging-benchmark/etc/checkstyle.xml
@@ -0,0 +1,217 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/openmessaging-benchmark/etc/findbugsExclude.xml b/openmessaging-benchmark/etc/findbugsExclude.xml
new file mode 100644
index 0000000..9126fc5
--- /dev/null
+++ b/openmessaging-benchmark/etc/findbugsExclude.xml
@@ -0,0 +1,137 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/openmessaging-benchmark/package/pom.xml b/openmessaging-benchmark/package/pom.xml
new file mode 100644
index 0000000..95d5ca8
--- /dev/null
+++ b/openmessaging-benchmark/package/pom.xml
@@ -0,0 +1,75 @@
+
+
+
+ 4.0.0
+
+
+ io.openmessaging.benchmark
+ messaging-benchmark
+ 0.0.1-SNAPSHOT
+
+
+ package
+ pom
+ Packaging
+
+
+ ${project.basedir}/..
+
+
+
+
+ ${project.groupId}
+ benchmark-framework
+ ${project.version}
+
+
+ ${project.groupId}
+ driver-fluss
+ ${project.version}
+
+
+
+
+
+
+ maven-assembly-plugin
+ 3.3.0
+
+
+ distro-assembly
+
+ single
+
+ package
+
+ true
+ posix
+ openmessaging-benchmark-${project.version}
+
+ src/assemble/bin.xml
+ src/assemble/src.xml
+
+
+
+
+
+
+
+
+
diff --git a/openmessaging-benchmark/package/src/assemble/bin.xml b/openmessaging-benchmark/package/src/assemble/bin.xml
new file mode 100644
index 0000000..9f8acdc
--- /dev/null
+++ b/openmessaging-benchmark/package/src/assemble/bin.xml
@@ -0,0 +1,86 @@
+
+
+ bin
+
+ tar.gz
+
+ true
+
+
+ true
+
+
+
+
+
+ ${basedir}/../workloads
+
+
+ ${basedir}/../payload
+
+
+ ${basedir}/../bin
+ 755
+
+
+
+ ${basedir}/../driver-fluss/config
+ driver-fluss/config
+
+
+
+
+
+ ${basedir}/../../LICENSE
+ LICENSE
+ .
+ 644
+
+
+
+
+ lib
+ false
+ compile
+ false
+
+ ${artifact.groupId}-${artifact.artifactId}-${artifact.version}${dashClassifier?}.${artifact.extension}
+
+
+
+ io.netty:netty-common
+ io.netty:netty-buffer
+ io.netty:netty-codec-http
+ io.netty:netty-codec
+ io.netty:netty-transport
+ io.netty:netty-handler
+ io.netty:netty-transport-native-epoll
+ io.netty:netty-transport-native-kqueue
+ io.netty:netty-codec-http
+ io.netty:netty-codec-dns
+ io.netty:netty-resolver
+ io.netty:netty-resolver-dns
+ io.netty:netty-transport-native-unix-common
+ ch.qos.logback:*
+
+
+
+
diff --git a/openmessaging-benchmark/package/src/assemble/src.xml b/openmessaging-benchmark/package/src/assemble/src.xml
new file mode 100644
index 0000000..757f105
--- /dev/null
+++ b/openmessaging-benchmark/package/src/assemble/src.xml
@@ -0,0 +1,120 @@
+
+
+ src
+
+ tar.gz
+
+ true
+
+
+
+ true
+
+
+ false
+
+
+
+
+ .git/**
+ **/.gitignore
+ **/.svn
+ **/*.iws
+ **/*.ipr
+ **/*.iml
+ **/*.cbp
+ **/*.pyc
+ **/.classpath
+ **/.project
+ **/.settings
+ **/target/**
+ **/.terraform/**
+ **/terraform.tfstate
+ **/terraform.tfstate.backup
+ **/*.retry
+ **/.settings/**
+
+
+
+
+
+
+
+
+
diff --git a/openmessaging-benchmark/payload/payload-100b.data b/openmessaging-benchmark/payload/payload-100b.data
new file mode 100644
index 0000000..3883beb
--- /dev/null
+++ b/openmessaging-benchmark/payload/payload-100b.data
@@ -0,0 +1 @@
+fbb0cb87a24408039b4f07c89929b5f619a682ecf61a1d97d495e38f876729ddc5a74c04d2fe9382b5eb85379d0f3027fece
\ No newline at end of file
diff --git a/openmessaging-benchmark/payload/payload-1Kb.data b/openmessaging-benchmark/payload/payload-1Kb.data
new file mode 100644
index 0000000..8490219
--- /dev/null
+++ b/openmessaging-benchmark/payload/payload-1Kb.data
@@ -0,0 +1 @@
+6b8d0ca6d616a2e39d674b73c9b11f9a3262a4f03444451f01f27003e84dc3469e61ddcfad9c2bc7c591988add0132b934e5ec0e7b396c2ef3ae8b229090650a422072555dee7685c030eef99cda045c368ce4b119b864b3665056109984ebcf3061e10f56940618f59f7b4b0099d1ea6aae11390a42c46151939ad5f866ae0d2aaedb3cdc4bac917d4419ce1440a6b4cd68b5609baf787fe28eb349bddef61bd9a6d304c57cd20f7ec9eab1cbc8cd6d07fe16e3bf134c87cfd71879ed047f0fd1951223e9b3d006ca0da4c10dc388b3a32a7db3a4516e9d6bcc178b8a0a7fdb21e454ce30f745cc97c7bd921d90db592e76fb349990146fa63bc579e897f6c4902203be20857a0794970d58db4365736f7ab14762c24095b270ecd8b1e42501b2282774e3cebb98d6d46fbcc5817ab65cf0cda02d1679d6ccd23bdec3de734211e657f4104cd34c962fd4b6e45fb34b4141fccfcdb70312b7a3d6f0e4b79077756e7d8d9ded9d586f9039f4f9e55801b2273297dc618a10f72ca3540d4345dec9996fb7d36dabaf9733629d3f47ed3748a6236daacf9c9271f8701dfc0730413bba3b73df235377963d21e685027aafd123fea1b1a7f4e91cfaf16672658e2f7b6ad86d97fd8dc240a94f87bf0ba025b08573e403e4ab0bc1372357c2461a8e5e190c96eb5d1df2f95a414e7cfeeb7f31667b4fe0bea6385d15faadc6cc83a3
\ No newline at end of file
diff --git a/openmessaging-benchmark/payload/payload-200b.data b/openmessaging-benchmark/payload/payload-200b.data
new file mode 100644
index 0000000..2928bb7
--- /dev/null
+++ b/openmessaging-benchmark/payload/payload-200b.data
@@ -0,0 +1 @@
+6b8d0ca6d616a2e39d674b73c9b11f9a3262a4f03444451f01f27003e84dc3469e61ddcfad9c2bc7c591988add0132b934e5ec0e7b396c2ef3ae8b229090650a422072555dee7685c030eef99cda045c368ce4b119b864b3665056109984ebcf3061e10f
\ No newline at end of file
diff --git a/openmessaging-benchmark/payload/payload-2Kb.data b/openmessaging-benchmark/payload/payload-2Kb.data
new file mode 100644
index 0000000..6b0d1e4
--- /dev/null
+++ b/openmessaging-benchmark/payload/payload-2Kb.data
@@ -0,0 +1 @@
+rtsDpjDRfh7fcULeEhdjYMUoFgdnDgu9zY2NLcxux9ILpOGlkLZJlAOU0FzOYmZbYOmMaH7ziv3ZuVPxUEWLWfjMidsCWDNZ70SeAvbgh0mzRZDHUYVxFuIgml8x07O8j4m7dzpkY3Ip6hixuEJuiakOG9DV7knwcNChzvdAKAOSO4Pip3ity8pEOtmCe11Uef1QARufTTnDREAOKJ4WFHywJ6rwCZKuirSm6DK3UdSKk4gtKCNWM33IWHhDDHvlhFOrrRJT68czmlw6U4QOqamdz0rS5LYEBJeiqoxGNf3AOtm1ZOInHUbVuCC87dq5qv4gnJ49LTHlJcHDS71C5V24pIDEKMqzJIplCS1TJ8vewlyPTnP4CZcQul4xWiI33blfzirHJ9zxWRNiGdCN7ovzEBvZX8dcqVt0vX0uNGKg8ZOxa68KHvv2G9nfdGLZIFxInp9Lcmd6h7pEd2JVJZwCs4aLRi7JlQKCSG61bljC2WW8ObMhTNlixhHyvFkgxn5unhuvrzbGB0lF6bwUeIM4lnrJrrlwHSBOr7bQ2R4mw72whEmMV0tsIag76lYYdkSwMMwgOEzB7zCSVlNZubmFIME419Z5if3lSijROYso6zZQVwuRNbAB6pynuyNdhl222ddw9wglppsNrwNRf1vsBjlHn8Tq7q6JYYeOEI8xnVfPnmeRl3Te10aplU3EmIURuvk7z8nvYNVqhBqkAtWFKaNNjPjGC4dIM5LQwFvkl4VEJg9kHafsW1fTKsDiRLLmqrlV1MzztmVGQCEhF66a94MS89VAiLmY2BscYepN0gsaU6d8iKxUGLv4g48jTziPLTiOfAlc6C8aEvLkiZ17EfBrR7DBcavkYiAZS4UAkwq48H2zV0Ey0c4JtAqa17GGDXSrrMNQlsCg3TSMKODehw9eSJ2SYh0EcWjh9NNingSAEv07Yf3uv0VAMAvFkIJFmpP2dpVpooX6IPryCkxstQUM1tIp8HgNWv4gk4HRmxAkk8NkWCIFmGZblBFwXZY0AUgcpCSmKGWphB21G19SWE53sYNyoyg4BLU3yihKaOhTjA4lj6wdHmmXJi0ZVUE4A8SQWqXeNCdndGUnEb7KqldQmhJJ9IpcOMrEI1BcbVcev2ZorGzQif64HGGJHFPL9K1GWZOT0lULwJr53Xnxg3r6AbmuXrwiLwcGDnGPC9eXQC5M5lvzTmnggmy06aNGvSMc6vnaLlJnEpct8MfrA6VZputKajNYq1fGqHq1qYcfNGo1mzJOycFC34P8aPwH09fGtFPM3GQ0nYzCmJoePMPUCAHPR5H6zJdw4XNY94aanmJYgePB71kwWxoUTmcPoz8iggk0yf95JEvGi91Kz7wbbUSdriASIC68i4TJcqwXH4GcrNoV3opnziiKz37bmK7r5kb0HapS4WnvP23CT6kQQKZVDOYqnTuweOe2RYVTD1I2EPdPICopMXTiDnCQjORl3Ddmah1IcS3qi4NFWT0Eo5xSvWkAlNgolRCAmRn2rKPYENiNw2zIY4G7r5lpidYiaXeTaCJOjZQTfxNIzStezivJvJ58KvagTKyWEKxA7pbLHj2LdM0kNWN9Ug9oLBqBYbawkxS3G4cSb4FPdGCSe3h1xTr0JcA8tc1XGouCejy0OjsENOhL9TN68t7rdYmvGmz0Syjnhfr4rrHHkbRSa7MN41bDO0MlLsGs9T8ifod7FFjGn7cD025G3x2tVGGYqxHm1MvlW0X3lqneoCpQdjkPqNCNa4xzypOfrcQqZb8fN9OH1uXP4PotpR8yIIPpad6rRQrmrGKtmCWjPQGe6QMak56CxmRp25b8EmbkWDWQnDcsqVZ3NAoaCqgYdmEawAyjTXVTOYjdwp7BwyBTCnUyL92lZxN1cbLZ7elfsXz5IKygtpObworD8OzAD4XQXR7dJDJoDEnfVzBhFsYk4d6qqTuhtJV3XCJ6ajlyhAjsqLk6tjt936wcvYvudRkzJTqwX56EGhQnm5l3uAH2IffVBqtPwXu1gGbR3UDaW1aZcUYNNywmPzGm
\ No newline at end of file
diff --git a/openmessaging-benchmark/payload/payload-400b.data b/openmessaging-benchmark/payload/payload-400b.data
new file mode 100644
index 0000000..1447cfd
--- /dev/null
+++ b/openmessaging-benchmark/payload/payload-400b.data
@@ -0,0 +1 @@
+6b8d0ca6d616a2e39d674b73c9b11f9a3262a4f03444451f01f27003e84dc3469e61ddcfad9c2bc7c591988add0132b934e5ec0e7b396c2ef3ae8b229090650a422072555dee7685c030eef99cda045c368ce4b119b864b3665056109984ebcf3061e10f56940618f59f7b4b0099d1ea6aae11390a42c46151939ad5f866ae0d2aaedb3cdc4bac917d4419ce1440a6b4cd68b5609baf787fe28eb349bddef61bd9a6d304c57cd20f7ec9eab1cbc8cd6d07fe16e3bf134c87cfd71879ed047f0fd1951223e9b3d006
\ No newline at end of file
diff --git a/openmessaging-benchmark/payload/payload-4Kb.data b/openmessaging-benchmark/payload/payload-4Kb.data
new file mode 100644
index 0000000..7a0bcdf
--- /dev/null
+++ b/openmessaging-benchmark/payload/payload-4Kb.data
@@ -0,0 +1 @@
+rUuY9ThnS1cvr8DZ1LZtH30WgDaEPOZHYleg7mF1zkQ04DQ9EEX1cyWIlKPNPKMBdPvGtQqBsQ85bZ2c2bqxXa0XSReS3o11ZZXX1jxCTLMrWL8BlicjPo9oiFXKG9roK3aW1dNIhHdzbdvOOCRcAEPFcIq4wIqR8kCa5vspSbnEiIO8v9SII8qFmyIuQgPi97A7YXIgBBr3oGoDh3YPWmLxDYn7N6Bqh2iPx3X9eUBqlMpjNqBaEoweDCnXn3cTvzl6WxKfli5aaXk5UvwADymw1hxeJtYAY2dd7fnGx1UEpP1i5g3G9KDRNHEWr06MM7804boGlN36aNmE5uH5y5RkH4oy9rAlEHXQKxNb0w0ogh4c15tslZ7LRPV91RymAACgAk9jFWGhLMd8OvzYnVUsb16ChK39NOsNwu8Mgfa83dPwiIANd6ZmHdetYaxFzr5bnCdNowgr8H8ynUqZ1yINa3GCFROQLDeXi6c6byvDCk0MGuzMcuCA4qSdMn6ktx6KgVyW7QjIzojoNQ6CSLOVxN1DxbgWNGR9IsP30bxCDVlZIkYGxYWSg2XT6Efun5RYu2Elk4zvdJkyeSAEV3H4jADsZRzXSQCEE1NoYcBko6YmTzVTGMldS3HrUhUmXXCgIHqarBji2kvOJUzDJkpxd7kRCHbHDHlYjGXGDc1j7e425t8WdNG4e1IR7g96Os1hCXuEgXZVzRFZWxRSQgRmDg3r2lehJMSG0Lg2Gh9jTTBrvAogQDuMdkHYLhU1Zfjf9oatvj1xVwgOfnICocfyfWICmrxyN8845zYL5buAlNBc1JVs3uOHWj0r0257Y4XAven5Xq9HsJiFuVALVt4LOFaljWS8Zj1Fc0ZiVMWF99pzCYA29o65ZAbwB8TecK7K4cII8byB0a9ojUgkCikuasvzxPGEWEzIBsOIvUd1E0agUb3FYs3XwT0LPN8tilNdMmposEPGUzIifP57nvSkhVk4ueu8wZCVOkSul9xV0rW1HFtThvDEuWJYLs1SqA6L7bQyVZg2CYK5u9IuoyhpCs396dy7QU9bUWJiyRUsqWJxKrFfyeMv1WRXCjfnC6aP0GPXNfaTeqzB7lGlmJI9ljYLb8OWd03bTwbUMDwbgPIpp6CKgwnWCVxWp6ralgv8TNc5mWFwl5yC2aTiFaoMD0rzWjxzYFPwRUdhi2Rs6k0Olc4jdLfQI63s2Nc3g8l0FznZqkU9qxUd07whX5d5nnibQB6c5UH4TpG8g55WkJqEnkOsZL6YuSxW48csD3pb0fuec5IfjbS62WvxbXcRvBROekLpvMTUZIkNaBllyu6p3ZNd3Y75YaQD5lLmg7TAcFyblpyjsmKZNlYEaMksJVcX6t9H3Iman3Nyfv4k4I2B12yuo5B1QuLfuCP1kGLPNW9QeM19Yp8Lyoc4y3g3krhNcXavBbMAF4AmpXR5RMq8GQ3BU3v7uxboLlaEEyua6L8WGJRw1vegxmF4kdZQO1yjdUE08cWA1bcX4SiYuUsnGOWLxHTW3PEggezUrR6BOrouJ5cmUllAbh1vfbmpbfGqqzTuQvKbEnys3sFSOn0B959uebiGX5sqXXpzWdLNuctZw4Jq7c6T92VJS3mOWzf2SUNtNrktKSmpnXDew7cW6mveOY4csSaVkDOkiCPLl09Qt6fuwog9GuhvXZbYYC4wGXbzhTCMQV65iTx4KqDOGwomepdAcodkgcU7wcGW017Uivx00FotMCnIcklxBVEQo6OgpH5OKCCvLpCuhpoWbPD3kegesQA0vOnh220qPEj8kG1wSlcqvDw4dp11Hj5MPn4jvVfOY69X7VFLejurS6onYEMkxijAbvQeNhaW4NG025TvxU3qTIPyov2u8CmC7vRTZcHdPYkiPO4454Xh2Z2LOzaDAeshkPErR818ZkkQD8TDOd8xDxst8ZocXGJIez4S0YP5GnJ26I8pwytdtmakHJPpLyWEWHQq8C2WiLve6pLLWMonRPOKQFiAmHwleeAA1f2OzhvofsasW8b6LbihhWo4MNK4hOh0sCn26crzUyLcPTxj3KXleGxc5iklrPua3CHe5N1kJAqcKremMGxhmPd2TTn3ELLPExlfvQ8dxoYlMszIzqD14w7mZCMYZD5t1ckO5Q7KM9UvYlF9P49i9515x8oe7h7sIpAmqY1qrB2nlUkn2JS153yvKspRGnCgHbMnVitV3Pz7r5lwsqqaJqru2GTBHfD8VSvfMXO99gqPH2n89ewj2lqzmMlyzY9HD0yQ3eOALPFA0irPWvK7sCpwTIBEmwSueKyO32LdJastJCcyht0ng8uIRrL7o6VcLmBL4p46reu3a2Ha7KW7ysMNwMyOLqnn9lVsNXitzC41mEvwYUebt2lmX9G9ubTSCMKJVslvAhG8NKxYJ6OKD4uIoAML9D9IW23cOkmJnH2H8clJ2KBzjoAPEZX92BsJoGbHsMpcrp8jtSRBDBImDJ7jTUf3BnRfNJYt1wxtYLjg7MbmcvanSuuVS9mG2um3zwABjU3lOHmLpZZj5RC28gYz2bCFSeo4SJewWWQIDX5ipq0Kjl2ffBT6YfVejWpa3b4vCeNTQTfmUMs6IjSAJjrYAKlEObyWikEEX21sDyGunTclLyYBUgMiNPg32dfGpmYPKVD2DFOrWXvk6uDwVgYVtNQj3FO0aGzxfP4wELJPlMmt6yI44EjtT2dGlKOoBhpeAq1uNpeP8Y3RvgnozPSpuICARcm68AxFgItWHRXVl1RghDrNpelGmziMzlDv8CF7g61XpbxpcUERVa5KyErzETRKf4iaWJEv4cqROZmIQafLoKk6YEJgCbmlU3l65CVitj8I7Jz2JhRAnqJGeMkQFurvy4bKDKxMVyb77XthyVaiC280YCDjEG4x96K6ccGCS8xJz2q2wZ7mP6ZaStcgnl3sD9H1TBQpo3DudH9KKr0VeGq2NxslDSEaWIgjbZgBTjRVK2eovs61uGPoyqRbdyE1LdRiALnV9YtWmCtg1GNxHj0xkvdlgoFK2OlKIRVmb6vwEFc6GIIH0P2xBjS1RgiIFnLIybrGKOq3V2Lu2orpbmXvO1cQfuMYfGhY87xjR8wzSXnHDTrQUTKM71YoyRqey2fZgWXHpsWRm1keNsjGZ9WYMxzHkAUVnKkYPJ45NduxcLdetPJBDcvUqp34YTHG6bi8rpCAdGShKHhHEWCOurfm1xEWuRiVRRWt8I8WiDW4ZN1P7DjKZ9KkbWLGPsa6F3Ntp02SKXOYEnMx629MrgmAHxyZVbRiS1Fs2IiUyERqEy1rJXqARmHg6ADIKKoDQIvYKPyWccvXjipk2zoAs6yDp3WTpjwzEadZhO6ftGxoUEJCaXb6qqAF379QYo006Sqd975J9E0VxemATolZOR6bu84hTjVYdD23SX6BIsm6BtoCoYhKt0AeuHd53ogQkUbYdqLN9wKBIL4gOzWQMyu6cCEqyUgnJOlhRJnRM6w3dREZyllLC79urQpFF6QxaAB59JHePLdNpPNvTOqLracHSPSRc96mecn0aXCMfw2GC1rC2zqPFPHWa03QkaSXKjwmuwglwkDlt3mXvZKAXQHBrTt0bpgoWr3kXxnDrv0cWGTiw2mGQczfE9R4ir5ADWoQTomtTydmmKbomUbZQrmQGkWOcpz48cbk0WFnXfpoRml2Iys1FmejC5gAwNTx3alIYiwIqwHDCp2k6Q87kqVOQsMMmT9xidZzinPYmVKIGKVqWEYGO5mfwVX2mTWqIdGCvG3SMf8iYmvZd5VXOn4uGK53rteuxFs0fKEpTi92tSTvYa6UIi0tyasgqyWQc4jUxxhHCFaWJdtqcrnrrIgcBuESozVSLuBgZ4rdHOtXocnU0NwmZeEfiJ1ZBhNzpG5B4LVd3kfFENK3Ca4MWoPTGI09GqopBaIB4BLLBwcKaKkgkdkYeUObtsXxQ6Pjkh1HcRlZqTEj5kffoUBnLzBbPINKIOUkUA61uLOH7ZDlQRn9YAMC4W5xsK8qNAKm30vjy5mKuw0j1m486eVdiN2F4ybWkdB5oXKWwuHO6UPHz631DooV1lAzFcJeCOC7G5T2
\ No newline at end of file
diff --git a/openmessaging-benchmark/pom.xml b/openmessaging-benchmark/pom.xml
new file mode 100644
index 0000000..e60e2bf
--- /dev/null
+++ b/openmessaging-benchmark/pom.xml
@@ -0,0 +1,462 @@
+
+
+
+ 4.0.0
+
+ io.openmessaging.benchmark
+ messaging-benchmark
+ 0.0.1-SNAPSHOT
+ pom
+ Messaging Benchmark
+
+ 2017
+
+
+ OpenMessaging Project
+ https://openmessaging.cloud
+
+
+
+
+ Apache License, Version 2.0
+ https://www.apache.org/licenses/LICENSE-2.0.txt
+ repo
+
+
+
+
+ driver-api
+ benchmark-framework
+ driver-fluss
+ package
+
+
+
+ UTF-8
+ UTF-8
+
+ 3.23.1
+ 4.14.4
+ 10.3.3
+ 3.12.0
+ 2.20.0
+ 1.18.24
+ 2.13.2
+ 1.48
+ 5.9.0
+ 4.8.0
+ 4.1.65.Final
+ 1.7.36
+
+ 3.2.0
+ 0.8.8
+ 4.1
+ 3.10.1
+ 3.1.0
+ 3.0.0-M7
+ 4.7.2.0
+ 2.25.0
+
+
+ ${project.basedir}
+
+
+
+
+
+ com.beust
+ jcommander
+ ${jcommander.version}
+
+
+ com.fasterxml.jackson.core
+ jackson-annotations
+ ${jackson.version}
+
+
+ com.fasterxml.jackson.dataformat
+ jackson-dataformat-yaml
+ ${jackson.version}
+
+
+ com.fasterxml.jackson.jaxrs
+ jackson-jaxrs-base
+ ${jackson.version}
+
+
+ com.fasterxml.jackson.jaxrs
+ jackson-jaxrs-json-provider
+ ${jackson.version}
+
+
+ com.google.guava
+ guava
+ 29.0-jre
+
+
+ io.netty
+ netty-all
+ ${netty.version}
+
+
+ org.apache.commons
+ commons-lang3
+ ${commons.lang3.version}
+
+
+ org.apache.logging.log4j
+ log4j-core
+ ${log4j.version}
+
+
+ org.apache.logging.log4j
+ log4j-slf4j-impl
+ ${log4j.version}
+
+
+ org.projectlombok
+ lombok
+ ${lombok.version}
+ provided
+
+
+ org.assertj
+ assertj-core
+ ${assertj.version}
+ test
+
+
+ org.junit.jupiter
+ junit-jupiter
+ ${junit.jupiter.version}
+ test
+
+
+ org.mockito
+ mockito-junit-jupiter
+ ${mockito.junit.jupiter.version}
+ test
+
+
+
+
+
+
+
+
+ com.diffplug.spotless
+ spotless-maven-plugin
+ ${spotless.plugin.version}
+
+
+
+ false
+ true
+ 4
+ scope,groupId,artifactId
+ groupId,artifactId
+ groupId,artifactId
+
+
+
+
+
+
+
+ true
+ 2
+
+
+ true
+ 4
+
+
+
+
+ **/*.md
+
+
+
+
+
+
+ etc/**/*.xml
+ .github/workflows/**/*.yml
+ **/doc/**/*.puml
+
+
+
+
+ true
+ 4
+
+
+
+
+
+
+ check
+
+ check
+
+
+
+
+
+
+ com.github.spotbugs
+ spotbugs-maven-plugin
+ ${spotbugs.plugin.version}
+
+ ${omb.root.dir}/etc/findbugsExclude.xml
+
+
+
+
+ check
+
+
+
+
+
+
+ com.mycila
+ license-maven-plugin
+ ${license.plugin.version}
+
+ ${omb.root.dir}/etc/APACHE-2.txt
+
+ LICENSE
+ NOTICE
+ DISCLAIMER
+ README.md
+ .asf.yaml
+ payload/**
+ **/*.pyc
+ **/.pydevproject
+ .github/**
+
+
+ SCRIPT_STYLE
+ SCRIPT_STYLE
+ SCRIPT_STYLE
+ SCRIPT_STYLE
+ XML_STYLE
+ SCRIPT_STYLE
+ APOSTROPHE_STYLE
+
+
+
+
+ com.mycila
+ license-maven-plugin-git
+ ${license.plugin.version}
+
+
+
+
+
+ check
+
+ process-sources
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-checkstyle-plugin
+ ${checkstyle.plugin.version}
+
+ true
+ true
+ true
+ ${omb.root.dir}/etc/checkstyle.xml
+ true
+
+
+
+ com.puppycrawl.tools
+ checkstyle
+ ${checkstyle.version}
+
+
+
+
+
+ check
+
+ validate
+
+
+
+
+ org.apache.maven.plugins
+ maven-enforcer-plugin
+ ${maven.enforcer.plugin.version}
+
+
+ enforce-maven-version
+
+ enforce
+
+
+
+
+ [3.8.6,)
+
+
+ true
+
+
+
+
+
+ maven-failsafe-plugin
+ ${maven.surefire.plugin.version}
+
+
+
+ integration-test
+ verify
+
+
+
+
+
+ maven-surefire-plugin
+ ${maven.surefire.plugin.version}
+
+
+ --add-opens java.base/java.util=ALL-UNNAMED --add-opens java.base/java.lang=ALL-UNNAMED
+
+
+
+
+ org.jacoco
+ jacoco-maven-plugin
+ ${jacoco.plugin.version}
+
+
+
+ prepare-agent
+
+
+
+ report
+
+ report
+
+ verify
+
+
+ check
+
+ check
+
+ verify
+
+
+
+ BUNDLE
+
+
+ CLASS
+ COVEREDRATIO
+ 0.00
+
+
+ METHOD
+ COVEREDRATIO
+ 0.00
+
+
+ LINE
+ COVEREDRATIO
+ 0.00
+
+
+ BRANCH
+ COVEREDRATIO
+ 0.00
+
+
+
+
+
+
+
+
+
+
+
+
+
+ modern-java-compile
+
+ [9,)
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-compiler-plugin
+ 3.8.1
+
+ 8
+ 8
+ 8
+ ${project.build.sourceEncoding}
+ true
+ true
+ true
+
+
+
+
+
+
+
+ jdk-8-compile
+
+ [,8]
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-compiler-plugin
+ 3.8.1
+
+ 8
+ 8
+ ${project.build.sourceEncoding}
+ true
+ true
+ true
+
+
+
+
+
+
+
+
diff --git a/openmessaging-benchmark/workloads/1-topic-1-partition-1kb.yaml b/openmessaging-benchmark/workloads/1-topic-1-partition-1kb.yaml
new file mode 100644
index 0000000..62f5ff0
--- /dev/null
+++ b/openmessaging-benchmark/workloads/1-topic-1-partition-1kb.yaml
@@ -0,0 +1,33 @@
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+name: 1 topic / 1 partition / 1KB
+
+topics: 1
+partitionsPerTopic: 1
+
+messageSize: 128
+useRandomizedPayloads: true
+randomBytesRatio: 0.5
+randomizedPayloadPoolSize: 1000
+
+subscriptionsPerTopic: 1
+producersPerTopic: 1
+consumerPerSubscription: 1
+
+producerRate: 50000
+
+consumerBacklogSizeGB: 0
+testDurationMinutes: 1
+warmupDurationMinutes: 0
diff --git a/openmessaging-benchmark/workloads/1-topic-16-partitions-1kb.yaml b/openmessaging-benchmark/workloads/1-topic-16-partitions-1kb.yaml
new file mode 100644
index 0000000..fc7e46e
--- /dev/null
+++ b/openmessaging-benchmark/workloads/1-topic-16-partitions-1kb.yaml
@@ -0,0 +1,33 @@
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+name: 1 topic / 16 partitions / 1KB
+
+topics: 1
+partitionsPerTopic: 16
+
+messageSize: 1024
+useRandomizedPayloads: true
+randomBytesRatio: 0.5
+randomizedPayloadPoolSize: 1000
+
+subscriptionsPerTopic: 1
+producersPerTopic: 4
+consumerPerSubscription: 4
+
+producerRate: 200000
+
+consumerBacklogSizeGB: 0
+testDurationMinutes: 5
+warmupDurationMinutes: 1