├── .github ├── stale.yml └── workflows │ └── pr-build-and-test.yml ├── .gitignore ├── LICENSE ├── README.md ├── benchmark-framework ├── pom.xml └── src │ ├── main │ ├── java │ │ └── io │ │ │ └── openmessaging │ │ │ └── benchmark │ │ │ ├── Benchmark.java │ │ │ ├── DriverConfiguration.java │ │ │ ├── RateController.java │ │ │ ├── ResultsToCsv.java │ │ │ ├── TestResult.java │ │ │ ├── Workers.java │ │ │ ├── Workload.java │ │ │ ├── WorkloadGenerator.java │ │ │ ├── utils │ │ │ ├── Env.java │ │ │ ├── ListPartition.java │ │ │ ├── PaddingDecimalFormat.java │ │ │ ├── RandomGenerator.java │ │ │ ├── Timer.java │ │ │ ├── UniformRateLimiter.java │ │ │ ├── distributor │ │ │ │ ├── KeyDistributor.java │ │ │ │ ├── KeyDistributorType.java │ │ │ │ ├── KeyRoundRobin.java │ │ │ │ ├── NoKeyDistributor.java │ │ │ │ └── RandomNano.java │ │ │ └── payload │ │ │ │ ├── FilePayloadReader.java │ │ │ │ ├── PayloadException.java │ │ │ │ └── PayloadReader.java │ │ │ └── worker │ │ │ ├── BenchmarkWorker.java │ │ │ ├── DistributedWorkersEnsemble.java │ │ │ ├── HttpWorkerClient.java │ │ │ ├── LocalWorker.java │ │ │ ├── MessageProducer.java │ │ │ ├── Worker.java │ │ │ ├── WorkerHandler.java │ │ │ ├── WorkerStats.java │ │ │ ├── commands │ │ │ ├── ConsumerAssignment.java │ │ │ ├── CountersStats.java │ │ │ ├── CumulativeLatencies.java │ │ │ ├── PeriodStats.java │ │ │ ├── ProducerWorkAssignment.java │ │ │ ├── TopicSubscription.java │ │ │ └── TopicsInfo.java │ │ │ └── jackson │ │ │ ├── HistogramDeserializer.java │ │ │ ├── HistogramSerializer.java │ │ │ └── ObjectMappers.java │ └── resources │ │ └── log4j2.yaml │ └── test │ ├── java │ └── io │ │ └── openmessaging │ │ └── benchmark │ │ ├── RateControllerTest.java │ │ ├── utils │ │ ├── EnvTest.java │ │ ├── ListPartitionTest.java │ │ ├── PaddingDecimalFormatTest.java │ │ ├── TimerTest.java │ │ └── UniformRateLimiterTest.java │ │ └── worker │ │ ├── DistributedWorkersEnsembleTest.java │ │ ├── commands │ │ ├── CountersStatsTest.java │ │ ├── CumulativeLatenciesTest.java │ │ └── PeriodStatsTest.java │ │ └── jackson │ │ └── HistogramSerDeTest.java │ └── resources │ └── log4j2.yaml ├── bin ├── benchmark ├── benchmark-worker └── create_charts.py ├── deployment └── kubernetes │ └── helm │ ├── README.md │ └── benchmark │ ├── Chart.yaml │ ├── templates │ ├── _helpers.tpl │ ├── benchmark-worker.yaml │ └── benchmark.yaml │ └── values.yaml ├── docker ├── Dockerfile ├── Dockerfile.build ├── README.md └── pom.xml ├── driver-api ├── pom.xml └── src │ └── main │ └── java │ └── io │ └── openmessaging │ └── benchmark │ └── driver │ ├── BenchmarkConsumer.java │ ├── BenchmarkDriver.java │ ├── BenchmarkProducer.java │ ├── ConsumerCallback.java │ └── ResourceCreator.java ├── driver-artemis ├── artemis.yaml ├── pom.xml └── src │ └── main │ └── java │ └── io │ └── openmessaging │ └── benchmark │ └── driver │ └── artemis │ ├── ArtemisBenchmarkConsumer.java │ ├── ArtemisBenchmarkDriver.java │ ├── ArtemisBenchmarkProducer.java │ └── ArtemisConfig.java ├── driver-bookkeeper ├── README.md ├── bookkeeper.yaml ├── deploy │ ├── ansible.cfg │ ├── deploy.yaml │ ├── provision-pulsar-aws.tf │ ├── templates │ │ ├── benchmark-worker.service │ │ ├── bookkeeper.conf │ │ ├── bookkeeper.service │ │ ├── myid │ │ ├── prometheus.service │ │ ├── prometheus.yml │ │ ├── pulsar-dashboard.service │ │ ├── pulsar_env.sh │ │ ├── workers.yaml │ │ ├── zoo.cfg │ │ └── zookeeper.service │ └── terraform.tfvars ├── pom.xml └── src │ └── main │ └── java │ └── io │ └── openmessaging │ └── benchmark │ └── driver │ └── bookkeeper │ ├── Config.java │ ├── DlogBenchmarkConsumer.java │ ├── DlogBenchmarkDriver.java │ ├── DlogBenchmarkProducer.java │ └── stats │ ├── CounterAdaptor.java │ ├── GaugeAdaptor.java │ ├── OpStatsLoggerAdaptor.java │ └── StatsLoggerAdaptor.java ├── driver-jms ├── README.md ├── artemis-jms-transactions.yaml ├── artemis-jms.yaml ├── kafka-jms.yaml ├── package-kafka.sh ├── package-pulsar.sh ├── pom.xml ├── pulsar-jms-selectors.yaml ├── pulsar-jms-transactions.yaml ├── pulsar-jms.yaml └── src │ └── main │ └── java │ └── io │ └── openmessaging │ └── benchmark │ └── driver │ └── jms │ ├── JMSBenchmarkConsumer.java │ ├── JMSBenchmarkDriver.java │ ├── JMSBenchmarkProducer.java │ ├── JMSBenchmarkTransactionProducer.java │ └── config │ └── JMSConfig.java ├── driver-kafka ├── README.md ├── deploy │ ├── README.md │ ├── hdd-deployment │ │ ├── alicloud │ │ │ ├── ansible.cfg │ │ │ ├── deploy.yaml │ │ │ ├── provision-kafka-alicloud.tf │ │ │ └── terraform.tfvars │ │ ├── ansible.cfg │ │ ├── deploy.yaml │ │ ├── provision-kafka-aws.tf │ │ ├── templates │ │ │ ├── benchmark-worker.service │ │ │ ├── chrony.conf │ │ │ ├── kafka.service │ │ │ ├── myid │ │ │ ├── server.properties │ │ │ ├── workers.yaml │ │ │ ├── zookeeper.properties │ │ │ └── zookeeper.service │ │ └── terraform.tfvars │ └── ssd-deployment │ │ ├── alicloud │ │ ├── ansible.cfg │ │ ├── deploy.yaml │ │ ├── provision-kafka-alicloud.tf │ │ └── terraform.tfvars │ │ ├── ansible.cfg │ │ ├── deploy.yaml │ │ ├── provision-kafka-aws.tf │ │ ├── templates │ │ ├── benchmark-worker.service │ │ ├── chrony.conf │ │ ├── kafka.service │ │ ├── myid │ │ ├── server.properties │ │ ├── workers.yaml │ │ ├── zookeeper.properties │ │ └── zookeeper.service │ │ └── terraform.tfvars ├── kafka-big-batches-gzip.yaml ├── kafka-big-batches-lz4.yaml ├── kafka-big-batches-snappy.yaml ├── kafka-big-batches-zstd.yaml ├── kafka-big-batches.yaml ├── kafka-compression-gzip.yaml ├── kafka-compression-lz4.yaml ├── kafka-compression-snappy.yaml ├── kafka-compression-zstd.yaml ├── kafka-exactly-once.yaml ├── kafka-latency.yaml ├── kafka-no-linger.yaml ├── kafka-sync.yaml ├── kafka-throughput.yaml ├── pom.xml └── src │ ├── main │ └── java │ │ └── io │ │ └── openmessaging │ │ └── benchmark │ │ └── driver │ │ └── kafka │ │ ├── Config.java │ │ ├── KafkaBenchmarkConsumer.java │ │ ├── KafkaBenchmarkDriver.java │ │ ├── KafkaBenchmarkProducer.java │ │ └── KafkaTopicCreator.java │ └── test │ ├── java │ └── io │ │ └── openmessaging │ │ └── benchmark │ │ └── driver │ │ └── kafka │ │ └── KafkaTopicCreatorTest.java │ └── resources │ └── log4j2.yaml ├── driver-kop ├── kafka_to_kafka.yaml ├── kafka_to_pulsar.yaml ├── pom.xml ├── pulsar_to_kafka.yaml └── src │ ├── main │ └── java │ │ └── io │ │ └── openmessaging │ │ └── benchmark │ │ └── driver │ │ └── kop │ │ ├── KopBenchmarkDriver.java │ │ └── config │ │ ├── ClientType.java │ │ ├── Config.java │ │ └── PulsarConfig.java │ └── test │ ├── java │ └── io │ │ └── openmessaging │ │ └── benchmark │ │ └── driver │ │ └── kop │ │ └── KopBenchmarkDriverTest.java │ └── resources │ ├── kop_custom.yaml │ ├── kop_required.yaml │ └── kop_wrong_kafka_config.yaml ├── driver-nats-streaming ├── README.md ├── deploy │ ├── ali │ │ ├── ansible.cfg │ │ ├── provision-nats-streaming-ali.tf │ │ └── terraform.tfvars │ ├── ansible.cfg │ ├── deploy.yaml │ ├── provision-nats-streaming-aws.tf │ ├── templates │ │ ├── benchmark-worker.service │ │ ├── chrony.conf │ │ ├── cluster.conf │ │ ├── grafana-dashboards.yml │ │ ├── grafana-datasource.yml │ │ ├── nats-streaming-dashboard.service │ │ ├── nats-streaming-metrics.service │ │ ├── nats-streaming-server.service │ │ ├── prometheus.service │ │ ├── prometheus.yml │ │ └── workers.yaml │ └── terraform.tfvars ├── nats-streaming.yaml ├── pom.xml └── src │ └── main │ └── java │ └── io │ └── openmessaging │ └── benchmark │ └── driver │ └── natsStreaming │ ├── NatsStreamingBenchmarkConsumer.java │ ├── NatsStreamingBenchmarkDriver.java │ ├── NatsStreamingBenchmarkProducer.java │ └── NatsStreamingClientConfig.java ├── driver-nats ├── README.md ├── deploy │ ├── alicloud │ │ ├── ansible.cfg │ │ ├── deploy.yaml │ │ ├── provision-nats-ali.tf │ │ └── terraform.tfvars │ ├── ansible.cfg │ ├── deploy-client-jars.yaml │ ├── deploy.yaml │ ├── provision-nats-aws.tf │ ├── restart-brokers.yaml │ ├── restart-workers.yaml │ ├── templates │ │ ├── benchmark-worker.service │ │ ├── chrony.conf │ │ ├── cluster.conf │ │ ├── grafana-dashboards.yml │ │ ├── grafana-datasource.yml │ │ ├── nats-dashboard.service │ │ ├── nats-metrics.service │ │ ├── nats-server.service │ │ ├── nats.yaml │ │ ├── prometheus.service │ │ ├── prometheus.yml │ │ └── workers.yaml │ └── terraform.tfvars ├── nats.yaml ├── pom.xml └── src │ └── main │ └── java │ └── io │ └── openmessaging │ └── benchmark │ └── driver │ └── nats │ ├── NatsBenchmarkConsumer.java │ ├── NatsBenchmarkDriver.java │ ├── NatsBenchmarkProducer.java │ └── NatsConfig.java ├── driver-nsq ├── deploy │ ├── ansible.cfg │ ├── deploy.yaml │ ├── provision-rabbitmq-aws.tf │ ├── templates │ │ ├── benchmark-worker.service │ │ ├── nsqd-server.service │ │ ├── nsqlookupd-server.service │ │ └── workers.yaml │ └── terraform.tfvars ├── nsq.yaml ├── pom.xml └── src │ └── main │ └── java │ └── io │ └── openmessaging │ └── benchmark │ └── driver │ └── nsq │ ├── NsqBenchmarkConsumer.java │ ├── NsqBenchmarkDriver.java │ ├── NsqBenchmarkProducer.java │ └── NsqConfig.java ├── driver-pravega ├── README.md ├── build-docker.sh ├── deploy-k8s-components.sh ├── deploy │ ├── ansible.cfg │ ├── collect_logs_and_metrics.yaml │ ├── deploy.yaml │ ├── open_saved_metrics │ │ ├── docker-compose.yml │ │ └── open_saved_metrics.sh │ ├── provision-pravega-aws.tf │ ├── templates │ │ ├── benchmark-worker.service │ │ ├── bk_server.conf │ │ ├── bkenv.sh │ │ ├── bookkeeper.service │ │ ├── chrony.conf │ │ ├── common.sh │ │ ├── config.properties │ │ ├── controller.config.properties │ │ ├── dashboards │ │ │ ├── alerts_dashboard.json │ │ │ ├── controller_dashboard.json │ │ │ ├── operation_dashboard.json │ │ │ ├── scope_dashboard.json │ │ │ ├── segmentstore_dashboard.json │ │ │ └── stream_dashboard.json │ │ ├── influxdb.service │ │ ├── logback.xml │ │ ├── myid │ │ ├── pravega-controller.service │ │ ├── pravega-dashboard.service │ │ ├── pravega-segmentstore.service │ │ ├── prometheus.service │ │ ├── prometheus.yml │ │ ├── workers.yaml │ │ ├── zoo.cfg │ │ └── zookeeper.service │ ├── terraform.tfvars │ └── vars.yaml ├── doc │ ├── build_pravega.md │ ├── metrics_and_logs.md │ ├── run_in_k8s.md │ └── troubleshooting.md ├── driver-bash.sh ├── pom.xml ├── pravega-exactly-once.yaml ├── pravega.yaml ├── src │ └── main │ │ └── java │ │ └── io │ │ └── openmessaging │ │ └── benchmark │ │ └── driver │ │ └── pravega │ │ ├── PravegaBenchmarkConsumer.java │ │ ├── PravegaBenchmarkDriver.java │ │ ├── PravegaBenchmarkProducer.java │ │ ├── PravegaBenchmarkTransactionProducer.java │ │ └── config │ │ ├── PravegaClientConfig.java │ │ ├── PravegaConfig.java │ │ └── PravegaWriterConfig.java └── uninstall.sh ├── driver-pulsar ├── README.md ├── deploy │ └── ssd │ │ ├── alicloud │ │ ├── ansible.cfg │ │ ├── deploy.yaml │ │ ├── provision-pulsar-alicloud.tf │ │ └── terraform.tfvars │ │ ├── ansible.cfg │ │ ├── deploy-client-jars.yaml │ │ ├── deploy.yaml │ │ ├── extra_vars.yaml │ │ ├── provision-pulsar-aws.tf │ │ ├── restart-brokers.yaml │ │ ├── restart-workers.yaml │ │ ├── templates │ │ ├── benchmark-worker.service │ │ ├── bkenv.sh │ │ ├── bookkeeper-skip-journal.conf │ │ ├── bookkeeper.conf │ │ ├── bookkeeper.service │ │ ├── broker.conf │ │ ├── chrony.conf │ │ ├── client.conf │ │ ├── kop.conf │ │ ├── mop.conf │ │ ├── myid │ │ ├── prometheus.service │ │ ├── prometheus.yml │ │ ├── pulsar-dashboard.service │ │ ├── pulsar.service │ │ ├── pulsar_env.sh │ │ ├── workers.yaml │ │ ├── zoo.cfg │ │ └── zookeeper.service │ │ └── terraform.tfvars ├── pom.xml ├── pulsar-effectively-once.yaml ├── pulsar.yaml └── src │ └── main │ └── java │ └── io │ └── openmessaging │ └── benchmark │ └── driver │ └── pulsar │ ├── PulsarBenchmarkConsumer.java │ ├── PulsarBenchmarkDriver.java │ ├── PulsarBenchmarkProducer.java │ └── config │ ├── PulsarClientConfig.java │ ├── PulsarConfig.java │ ├── PulsarConsumerConfig.java │ └── PulsarProducerConfig.java ├── driver-rabbitmq ├── README.md ├── deploy │ ├── alicloud │ │ ├── ansible.cfg │ │ ├── deploy.yaml │ │ ├── provision-pulsar-alicloud.tf │ │ └── terraform.tfvars │ ├── ansible.cfg │ ├── deploy-client-jars.yaml │ ├── deploy.yaml │ ├── erlang.cookie │ ├── provision-rabbitmq-aws.tf │ ├── restart-brokers.yaml │ ├── restart-workers.yaml │ ├── templates │ │ ├── benchmark-worker.service │ │ ├── chrony.conf │ │ ├── grafana-datasource.yml │ │ ├── prometheus.service │ │ ├── prometheus.yml │ │ ├── rabbitmq-classic.yaml │ │ ├── rabbitmq-dashboard.service │ │ ├── rabbitmq-env.conf │ │ ├── rabbitmq-quorum.yaml │ │ ├── rabbitmq-server.service │ │ ├── rabbitmq.conf │ │ └── workers.yaml │ └── terraform.tfvars ├── pom.xml ├── rabbitmq.yaml └── src │ ├── main │ └── java │ │ └── io │ │ └── openmessaging │ │ └── benchmark │ │ └── driver │ │ └── rabbitmq │ │ ├── RabbitMqBenchmarkConsumer.java │ │ ├── RabbitMqBenchmarkDriver.java │ │ ├── RabbitMqBenchmarkProducer.java │ │ └── RabbitMqConfig.java │ └── test │ └── java │ └── io │ └── openmessaging │ └── benchmark │ └── driver │ └── rabbitmq │ ├── QueueTypeTest.java │ └── RabbitMqConfigTest.java ├── driver-redis ├── pom.xml ├── redis.yaml └── src │ └── main │ └── java │ └── io │ └── openmessaging │ └── benchmark │ └── driver │ └── redis │ ├── RedisBenchmarkConsumer.java │ ├── RedisBenchmarkDriver.java │ ├── RedisBenchmarkProducer.java │ └── client │ └── RedisClientConfig.java ├── driver-rocketmq ├── deploy │ ├── ansible.cfg │ ├── deploy.yaml │ ├── provision-rocketmq-alicloud.tf │ ├── templates │ │ ├── benchmark-worker.service │ │ ├── rmq-broker.properties │ │ ├── rmq-broker.service │ │ ├── rmq-namesrv.properties │ │ ├── rmq-namesrv.service │ │ └── workers.yaml │ └── terraform.tfvars ├── pom.xml ├── rocketmq.yaml └── src │ └── main │ └── java │ └── io │ └── openmessaging │ └── benchmark │ └── driver │ └── rocketmq │ ├── RocketMQBenchmarkConsumer.java │ ├── RocketMQBenchmarkDriver.java │ ├── RocketMQBenchmarkProducer.java │ └── client │ └── RocketMQClientConfig.java ├── etc ├── APACHE-2.txt ├── checkstyle.xml └── findbugsExclude.xml ├── package ├── pom.xml └── src │ └── assemble │ ├── bin.xml │ └── src.xml ├── payload ├── payload-100b.data ├── payload-1Kb.data ├── payload-200b.data ├── payload-2Kb.data ├── payload-400b.data └── payload-4Kb.data ├── pom.xml ├── style └── copyright │ └── Apache.xml ├── tool ├── README.md ├── pom.xml └── src │ ├── main │ ├── java │ │ └── io │ │ │ └── openmessaging │ │ │ └── benchmark │ │ │ └── tool │ │ │ └── workload │ │ │ ├── WorkloadGenerationTool.java │ │ │ ├── WorkloadGenerator.java │ │ │ ├── WorkloadNameFormat.java │ │ │ └── WorkloadSetTemplate.java │ └── resources │ │ └── log4j2.yaml │ └── test │ ├── java │ └── io │ │ └── openmessaging │ │ └── benchmark │ │ └── tool │ │ └── workload │ │ └── WorkloadNameFormatTest.java │ └── resources │ └── template.yaml └── workloads ├── 1-topic-1-partition-100b.yaml ├── 1-topic-1-partition-1kb.yaml ├── 1-topic-100-partitions-1kb-4p-4c-1000k.yaml ├── 1-topic-100-partitions-1kb-4p-4c-2000k.yaml ├── 1-topic-100-partitions-1kb-4p-4c-200k-backlog.yaml ├── 1-topic-100-partitions-1kb-4p-4c-200k.yaml ├── 1-topic-100-partitions-1kb-4p-4c-500k.yaml ├── 1-topic-100-partitions-1kb.yaml ├── 1-topic-10000-partitions-1kb-4p-4c-1000k.yaml ├── 1-topic-10000-partitions-1kb-4p-4c-2000k.yaml ├── 1-topic-10000-partitions-1kb-4p-4c-200k.yaml ├── 1-topic-10000-partitions-1kb-4p-4c-500k.yaml ├── 1-topic-16-partition-100b.yaml ├── 1-topic-16-partitions-1kb.yaml ├── 1-topic-3-partition-100b-3producers.yaml ├── 1-topic-6-partition-100b.yaml ├── 100-topic-1kb-4p-4c-2000k.yaml ├── 100-topic-1kb-4p-4c-500k.yaml ├── 100-topics-1-partitions-1kb.yaml ├── 100k-topic-1kb-4p-4c-100k.yaml ├── 100k-topic-1kb-4p-4c-2000k.yaml ├── 100k-topic-1kb-4p-4c-500k.yaml ├── 10k-topic-1kb-4p-4c-100k.yaml ├── 10k-topic-1kb-4p-4c-2000k.yaml ├── 10k-topic-1kb-4p-4c-500k.yaml ├── 1k-topic-1kb-4p-4c-100k.yaml ├── 1k-topic-1kb-4p-4c-2000k.yaml ├── 1k-topic-1kb-4p-4c-500k.yaml ├── 1m-10-topics-1-partition-100b.yaml ├── 1m-10-topics-16-partitions-100b.yaml ├── 1m-10-topics-2-partitions-100b.yaml ├── 1m-10-topics-3-partitions-100b.yaml ├── 1m-10-topics-4-partitions-100b.yaml ├── 1m-10-topics-6-partitions-100b.yaml ├── 1m-10-topics-9-partitions-100b.yaml ├── 1m-topic-1kb-4p-4c-2000k.yaml ├── 1m-topic-1kb-4p-4c-500k.yaml ├── 200k-topic-1kb-4p-4c-100k.yaml ├── 200k-topic-1kb-4p-4c-2000k.yaml ├── 200k-topic-1kb-4p-4c-500k.yaml ├── 300k-topic-1kb-4p-4c-2000k.yaml ├── 300k-topic-1kb-4p-4c-500k.yaml ├── 400k-topic-1kb-4p-4c-2000k.yaml ├── 400k-topic-1kb-4p-4c-500k.yaml ├── 500-topic-1kb-4p-4c-2000k.yaml ├── 500-topic-1kb-4p-4c-500k.yaml ├── 500k-topic-1kb-4p-4c-2000k.yaml ├── 500k-topic-1kb-4p-4c-500k.yaml ├── 50k-topic-1kb-4p-4c-100k.yaml ├── 50k-topic-1kb-4p-4c-2000k.yaml ├── 50k-topic-1kb-4p-4c-500k.yaml ├── 5k-topic-1kb-4p-4c-100k.yaml ├── 5k-topic-1kb-4p-4c-2000k.yaml ├── 5k-topic-1kb-4p-4c-500k.yaml ├── 600k-topic-1kb-4p-4c-2000k.yaml ├── 600k-topic-1kb-4p-4c-600k.yaml ├── 700k-topic-1kb-4p-4c-2000k.yaml ├── 700k-topic-1kb-4p-4c-500k.yaml ├── 800k-topic-1kb-4p-4c-2000k.yaml ├── 800k-topic-1kb-4p-4c-500k.yaml ├── 900k-topic-1kb-4p-4c-2000k.yaml ├── 900k-topic-1kb-4p-4c-500k.yaml ├── backlog-1-topic-1-partition-1kb.yaml ├── backlog-1-topic-16-partitions-1kb.yaml ├── max-rate-1-topic-1-partition-1p-1c-100b.yaml ├── max-rate-1-topic-1-partition-1p-1c-1kb.yaml ├── max-rate-1-topic-1-partition-1p-1c-64kb.yaml ├── max-rate-1-topic-1-partition-4p-1c-1kb.yaml ├── max-rate-1-topic-10-partitions-10p-10c-100b.yaml ├── max-rate-1-topic-10-partitions-10p-10c-1kb.yaml ├── max-rate-1-topic-10-partitions-10p-10c-64kb.yaml ├── max-rate-1-topic-10-partitions-1p-1c-100b.yaml ├── max-rate-1-topic-10-partitions-1p-1c-1kb.yaml ├── max-rate-1-topic-10-partitions-1p-1c-64kb.yaml ├── max-rate-1-topic-100-partitions-100b.yaml ├── max-rate-1-topic-100-partitions-100p-100c-100b.yaml ├── max-rate-1-topic-100-partitions-100p-100c-1kb.yaml ├── max-rate-1-topic-100-partitions-100p-100c-64kb.yaml ├── max-rate-1-topic-100-partitions-1kb.yaml ├── max-rate-1-topic-100-partitions-1p-1c-100b.yaml ├── max-rate-1-topic-100-partitions-1p-1c-1kb.yaml ├── max-rate-1-topic-16-partitions-100b.yaml ├── max-rate-1-topic-16-partitions-1kb.yaml ├── max-rate-1-topic-20-partitions-20p-20c-100b.yaml ├── max-rate-1-topic-20-partitions-20p-20c-1kb.yaml ├── max-rate-1-topic-20-partitions-20p-20c-64kb.yaml ├── max-rate-1-topic-30-partitions-30p-30c-100b.yaml ├── max-rate-1-topic-30-partitions-30p-30c-1kb.yaml ├── max-rate-1-topic-40-partitions-40p-40c-100b.yaml ├── max-rate-1-topic-40-partitions-40p-40c-1kb.yaml ├── max-rate-1-topic-40-partitions-40p-40c-64kb.yaml ├── max-rate-1-topic-50-partitions-50p-50c-100b.yaml ├── max-rate-1-topic-50-partitions-50p-50c-1kb.yaml ├── max-rate-1-topic-60-partitions-60p-60c-100b.yaml ├── max-rate-1-topic-60-partitions-60p-60c-1kb.yaml ├── max-rate-1-topic-60-partitions-60p-60c-64kb.yaml ├── max-rate-1-topic-70-partitions-70p-70c-100b.yaml ├── max-rate-1-topic-70-partitions-70p-70c-1kb.yaml ├── max-rate-1-topic-80-partitions-80p-80c-100b.yaml ├── max-rate-1-topic-80-partitions-80p-80c-1kb.yaml ├── max-rate-1-topic-80-partitions-80p-80c-64kb.yaml ├── max-rate-1-topic-90-partitions-90p-90c-100b.yaml ├── max-rate-1-topic-90-partitions-90p-90c-1kb.yaml ├── max-rate-10-topics-1-partition-100b.yaml ├── max-rate-10-topics-1-partition-1kb.yaml ├── max-rate-10-topics-1-partition-64kb.yaml ├── max-rate-100-topics-1-partition-100b.yaml ├── max-rate-100-topics-1-partition-1kb.yaml ├── max-rate-100-topics-1-partition-64kb.yaml ├── max-rate-20-topics-1-partition-100b.yaml ├── max-rate-20-topics-1-partition-1kb.yaml ├── max-rate-20-topics-1-partition-64kb.yaml ├── max-rate-30-topics-1-partition-100b.yaml ├── max-rate-30-topics-1-partition-1kb.yaml ├── max-rate-40-topics-1-partition-100b.yaml ├── max-rate-40-topics-1-partition-1kb.yaml ├── max-rate-40-topics-1-partition-64kb.yaml ├── max-rate-50-topics-1-partition-100b.yaml ├── max-rate-50-topics-1-partition-1kb.yaml ├── max-rate-60-topics-1-partition-100b.yaml ├── max-rate-60-topics-1-partition-1kb.yaml ├── max-rate-60-topics-1-partition-64kb.yaml ├── max-rate-70-topics-1-partition-100b.yaml ├── max-rate-70-topics-1-partition-1kb.yaml ├── max-rate-80-topics-1-partition-100b.yaml ├── max-rate-80-topics-1-partition-1kb.yaml ├── max-rate-80-topics-1-partition-64kb.yaml ├── max-rate-90-topics-1-partition-100b.yaml ├── max-rate-90-topics-1-partition-1kb.yaml └── simple-workload.yaml /.gitignore: -------------------------------------------------------------------------------- 1 | .classpath 2 | .project 3 | .settings 4 | .pydevproject 5 | target 6 | /.metadata/ 7 | *.pyc 8 | *.svg 9 | core/*.json 10 | *.json 11 | *.retry 12 | *.pem 13 | *.hcl 14 | **/.terraform 15 | **/terraform.tfstate 16 | **/terraform.tfstate.backup 17 | **/*.lock.info 18 | .DS_Store 19 | .factorypath 20 | 21 | # IntelliJ artifacts 22 | .idea/ 23 | *.iml 24 | -------------------------------------------------------------------------------- /benchmark-framework/src/main/java/io/openmessaging/benchmark/DriverConfiguration.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark; 15 | 16 | public class DriverConfiguration { 17 | public String name; 18 | 19 | public String driverClass; 20 | } 21 | -------------------------------------------------------------------------------- /benchmark-framework/src/main/java/io/openmessaging/benchmark/Workers.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark; 15 | 16 | 17 | import java.util.ArrayList; 18 | import java.util.List; 19 | 20 | public class Workers { 21 | public List workers = new ArrayList<>(); 22 | } 23 | -------------------------------------------------------------------------------- /benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/distributor/NoKeyDistributor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark.utils.distributor; 15 | 16 | public class NoKeyDistributor extends KeyDistributor { 17 | 18 | @Override 19 | public String next() { 20 | return null; 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/distributor/RandomNano.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark.utils.distributor; 15 | 16 | 17 | import javax.annotation.concurrent.ThreadSafe; 18 | 19 | @ThreadSafe 20 | public class RandomNano extends KeyDistributor { 21 | 22 | public String next() { 23 | int randomIndex = Math.abs((int) System.nanoTime() % getLength()); 24 | return get(randomIndex); 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/payload/PayloadException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark.utils.payload; 15 | 16 | public class PayloadException extends RuntimeException { 17 | 18 | public PayloadException(String message) { 19 | super(message); 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /benchmark-framework/src/main/java/io/openmessaging/benchmark/utils/payload/PayloadReader.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark.utils.payload; 15 | 16 | public interface PayloadReader { 17 | 18 | byte[] load(String resourceName); 19 | } 20 | -------------------------------------------------------------------------------- /benchmark-framework/src/main/java/io/openmessaging/benchmark/worker/commands/ConsumerAssignment.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark.worker.commands; 15 | 16 | 17 | import java.util.ArrayList; 18 | import java.util.List; 19 | 20 | public class ConsumerAssignment { 21 | public List topicsSubscriptions = new ArrayList<>(); 22 | } 23 | -------------------------------------------------------------------------------- /benchmark-framework/src/test/resources/log4j2.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | Configuration: 16 | status: INFO 17 | name: messaging-benchmark 18 | 19 | Appenders: 20 | Console: 21 | name: Console 22 | target: SYSTEM_OUT 23 | PatternLayout: 24 | Pattern: "%d{HH:mm:ss.SSS} [%t] %-4level %c{1} - %msg%n" 25 | Loggers: 26 | Logger: 27 | - name: io.openmessaging 28 | level: debug 29 | Root: 30 | level: info 31 | additivity: false 32 | AppenderRef: 33 | - ref: Console 34 | -------------------------------------------------------------------------------- /deployment/kubernetes/helm/README.md: -------------------------------------------------------------------------------- 1 | Users can deploy the helm chart: 2 | 3 | ```bash 4 | $ helm install ./benchmark --name benchmark 5 | ``` 6 | 7 | After the chart has started, users can exec into the pod name "benchmark-driver" and run the benchmark from there. 8 | 9 | For example, once inside the "benchmark-driver" pod, users can execute: 10 | 11 | ```bash 12 | bin/benchmark --drivers driver-pulsar/pulsar.yaml --workers $WORKERS workloads/1-topic-16-partitions-1kb.yaml 13 | ``` 14 | 15 | All workers that has configured to startup will be set in the "$WORKERS" env variable 16 | 17 | To tear down the chart: 18 | 19 | ```bash 20 | $ helm delete benchmark --purge 21 | ``` 22 | 23 | -------------------------------------------------------------------------------- /deployment/kubernetes/helm/benchmark/Chart.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | apiVersion: v1 16 | appVersion: "0.0.1" 17 | description: A Helm chart for running the Open Messaging Benchmark in Kubernetes 18 | name: openmessaging-benchmark 19 | version: 0.0.1 20 | sources: 21 | - https://github.com/openmessaging/openmessaging-benchmark 22 | maintainers: 23 | - name: Jerry Peng -------------------------------------------------------------------------------- /deployment/kubernetes/helm/benchmark/values.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | numWorkers: 8 16 | image: openmessaging/openmessaging-benchmark:latest 17 | imagePullPolicy: Always 18 | workload: workloads/1-topic-16-partitions-1kb.yaml 19 | driver: driver-pulsar/pulsar.yaml 20 | 21 | # Resource Configurations 22 | driverMemoryRequest: 512Mi 23 | driverMemoryLimit: 2048Mi 24 | 25 | #workersMemoryRequest: 512Mi 26 | #workersMemoryLimit: 2048Mi -------------------------------------------------------------------------------- /docker/Dockerfile: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | FROM openjdk:8-jdk 16 | 17 | ARG BENCHMARK_TARBALL 18 | 19 | ADD ${BENCHMARK_TARBALL} / 20 | 21 | RUN mv openmessaging-benchmark-* /benchmark 22 | 23 | WORKDIR /benchmark 24 | -------------------------------------------------------------------------------- /docker/Dockerfile.build: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | # Build the Project 16 | FROM maven:latest as build 17 | COPY . /benchmark 18 | WORKDIR /benchmark 19 | RUN mvn install 20 | 21 | # Create the benchmark image 22 | FROM openjdk:8-jdk 23 | COPY --from=build /benchmark/package/target/openmessaging-benchmark-*-SNAPSHOT-bin.tar.gz / 24 | RUN mkdir /benchmark && tar -xzf openmessaging-benchmark-*-SNAPSHOT-bin.tar.gz -C /benchmark --strip=1 25 | RUN rm /openmessaging-benchmark-*-SNAPSHOT-bin.tar.gz 26 | WORKDIR /benchmark -------------------------------------------------------------------------------- /docker/README.md: -------------------------------------------------------------------------------- 1 | # OpenMessaging Benchmark Framework Docker 2 | 3 | You can use either of the Dockerfiles - `./docker/Dockerfile` or `./docker/Dockerfile.build` based on your needs. 4 | 5 | ### `Dockerfile` 6 | 7 | Uses `openjdk-8` and takes `BENCHMARK_TARBALL` as an argument. 8 | While using this Dockerfile, you will need to build the project locally **first**. 9 | 10 | ``` 11 | #> mvn build 12 | #> export BENCHMARK_TARBALL=package/target/openmessaging-benchmark--SNAPSHOT-bin.tar.gz 13 | #> docker build --build-arg BENCHMARK_TARBALL . -f docker/Dockerfile 14 | ``` 15 | 16 | ### `Dockerfile.build` 17 | 18 | Uses the latest version of `maven` in order to build the project, and then use `openjdk-8` as runtime. 19 | This Dockerfile has no dependency (you do not need Mavent to be installed locally). 20 | 21 | ``` 22 | #> docker build . -f docker/Dockerfile.build 23 | ``` 24 | 25 | -------------------------------------------------------------------------------- /driver-api/src/main/java/io/openmessaging/benchmark/driver/BenchmarkConsumer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark.driver; 15 | 16 | public interface BenchmarkConsumer extends AutoCloseable {} 17 | -------------------------------------------------------------------------------- /driver-artemis/artemis.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: Artemis 16 | driverClass: io.openmessaging.benchmark.driver.artemis.ArtemisBenchmarkDriver 17 | 18 | # ActiveMQ Artemis client-specific configuration 19 | brokerAddress: tcp://localhost:61616 20 | -------------------------------------------------------------------------------- /driver-artemis/src/main/java/io/openmessaging/benchmark/driver/artemis/ArtemisConfig.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package io.openmessaging.benchmark.driver.artemis; 16 | 17 | public class ArtemisConfig { 18 | public String brokerAddress; 19 | } 20 | -------------------------------------------------------------------------------- /driver-bookkeeper/README.md: -------------------------------------------------------------------------------- 1 | # Apache BookKeeper benchmarks 2 | 3 | Apache BookKeeper is the storage used by Apache Pulsar. This driver allows benchmarking Apache BookKeeper directly using the OpenMessaging benchmark 4 | framework. For more details see the official pulsar [benchmark documentation](http://openmessaging.cloud/docs/benchmarks/pulsar/). 5 | -------------------------------------------------------------------------------- /driver-bookkeeper/deploy/ansible.cfg: -------------------------------------------------------------------------------- 1 | [defaults] 2 | host_key_checking=false 3 | private_key_file=~/.ssh/bookkeeper_aws 4 | 5 | [privilege_escalation] 6 | become=true 7 | become_method='sudo' 8 | become_user='root' -------------------------------------------------------------------------------- /driver-bookkeeper/deploy/templates/benchmark-worker.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=OMS Benchmark Worker 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/benchmark/bin/benchmark-worker 7 | WorkingDirectory=/opt/benchmark 8 | RestartSec=1s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | 13 | [Install] 14 | WantedBy=multi-user.target 15 | -------------------------------------------------------------------------------- /driver-bookkeeper/deploy/templates/bookkeeper.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Bookkeeper 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/pulsar/bin/pulsar bookie 7 | WorkingDirectory=/opt/pulsar 8 | RestartSec=1s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | 13 | [Install] 14 | WantedBy=multi-user.target 15 | -------------------------------------------------------------------------------- /driver-bookkeeper/deploy/templates/myid: -------------------------------------------------------------------------------- 1 | {{ zid }} 2 | -------------------------------------------------------------------------------- /driver-bookkeeper/deploy/templates/prometheus.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Prometheus server 3 | After=network.target 4 | 5 | [Service] 6 | WorkingDirectory=/opt/prometheus 7 | ExecStart=/opt/prometheus/prometheus \ 8 | --storage.tsdb.path=/opt/prometheus/data \ 9 | --config.file=/opt/prometheus/prometheus.yml 10 | 11 | [Install] 12 | WantedBy=multi-user.target 13 | -------------------------------------------------------------------------------- /driver-bookkeeper/deploy/templates/pulsar-dashboard.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Pulsar Dashboard 3 | After=docker.service 4 | Requires=docker.service 5 | After=prometheus.service 6 | Requires=prometheus.service 7 | 8 | [Service] 9 | WorkingDirectory=/opt/pulsar 10 | ExecStartPre=/usr/bin/docker pull apachepulsar/pulsar-grafana:latest 11 | ExecStart=/usr/bin/docker run --restart=always --name=systemd_pulsar_dashboard -p3000:3000 -e PROMETHEUS_URL=http://{{ hostvars[groups['prometheus'][0]].private_ip }}:9090/ apachepulsar/pulsar-grafana:latest 12 | ExecStop=/usr/bin/docker stop systemd_pulsar_dashboard 13 | ExecStopPost=/usr/bin/docker rm -f systemd_pulsar_dashboard 14 | ExecReload=/usr/bin/docker restart systemd_pulsar_dashboard 15 | 16 | [Install] 17 | WantedBy=multi-user.target 18 | -------------------------------------------------------------------------------- /driver-bookkeeper/deploy/templates/workers.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | 16 | workers: 17 | {% for worker in groups['client'] %} 18 | - http://{{ hostvars[worker].private_ip }}:8080 19 | {% endfor %} 20 | -------------------------------------------------------------------------------- /driver-bookkeeper/deploy/templates/zookeeper.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=ZooKeeper Local 3 | After=network.target 4 | 5 | [Service] 6 | Environment=OPTS=-Dstats_server_port=2182 7 | ExecStart=/opt/pulsar/bin/pulsar zookeeper 8 | WorkingDirectory=/opt/pulsar 9 | RestartSec=1s 10 | Restart=on-failure 11 | Type=simple 12 | LimitNOFILE=300000 13 | 14 | [Install] 15 | WantedBy=multi-user.target 16 | -------------------------------------------------------------------------------- /driver-bookkeeper/deploy/terraform.tfvars: -------------------------------------------------------------------------------- 1 | public_key_path = "~/.ssh/bookkeeper_aws.pub" 2 | region = "us-west-2" 3 | ami = "ami-9fa343e7" // RHEL-7.4 4 | 5 | instance_types = { 6 | "bookkeeper" = "i3en.6xlarge" 7 | "zookeeper" = "t2.small" 8 | "client" = "m5n.8xlarge" 9 | "prometheus" = "t2.small" 10 | } 11 | 12 | num_instances = { 13 | "client" = 4 14 | "bookkeeper" = 3 15 | "zookeeper" = 3 16 | "prometheus" = 1 17 | } 18 | -------------------------------------------------------------------------------- /driver-bookkeeper/src/main/java/io/openmessaging/benchmark/driver/bookkeeper/Config.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark.driver.bookkeeper; 15 | 16 | public class Config { 17 | 18 | public String dlogUri; 19 | 20 | public String dlogConf; 21 | } 22 | -------------------------------------------------------------------------------- /driver-jms/artemis-jms-transactions.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: JMS 16 | driverClass: io.openmessaging.benchmark.driver.jms.JMSBenchmarkDriver 17 | 18 | connectionFactoryClassName: org.apache.activemq.artemis.jms.client.ActiveMQConnectionFactory 19 | connectionFactoryConfigurationParam: 'tcp://localhost:61616' 20 | use20api: true 21 | sendWithTransactions: true 22 | -------------------------------------------------------------------------------- /driver-jms/artemis-jms.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: JMS 16 | driverClass: io.openmessaging.benchmark.driver.jms.JMSBenchmarkDriver 17 | 18 | connectionFactoryClassName: org.apache.activemq.artemis.jms.client.ActiveMQConnectionFactory 19 | connectionFactoryConfigurationParam: 'tcp://localhost:61616' 20 | use20api: true 21 | -------------------------------------------------------------------------------- /driver-jms/package-pulsar.sh: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | cd package/target 16 | tar zxvf openmessaging-benchmark-0.0.1-SNAPSHOT-bin.tar.gz 17 | cd openmessaging-benchmark-0.0.1-SNAPSHOT 18 | rm lib/org.apache.pulsar-* 19 | rm lib/*jackson* 20 | curl https://repo1.maven.org/maven2/com/datastax/oss/pulsar-jms-all/1.2.2/pulsar-jms-all-1.2.2.jar -o lib/pulsar-jms-all-1.2.2.jar 21 | cd .. 22 | rm openmessaging-benchmark-0.0.1-SNAPSHOT-bin.tar.gz 23 | tar zcvf openmessaging-benchmark-0.0.1-SNAPSHOT-bin.tar.gz openmessaging-benchmark-0.0.1-SNAPSHOT 24 | 25 | -------------------------------------------------------------------------------- /driver-kafka/README.md: -------------------------------------------------------------------------------- 1 | # Apache Kafka benchmarks 2 | 3 | For instructions on running the OpenMessaging bencmarks for Kafka, see the [official documentation](http://openmessaging.cloud/docs/benchmarks/kafka). 4 | 5 | NOTE: This is a slightly modified version with two key differences: 6 | 7 | - workloads have extra arguments for sending randomized payloads rather than the same payload over and over. 8 | - there is a new argument that converts all output result json files into a single csv. 9 | 10 | TODO: Document these changes. 11 | -------------------------------------------------------------------------------- /driver-kafka/deploy/hdd-deployment/alicloud/ansible.cfg: -------------------------------------------------------------------------------- 1 | [defaults] 2 | host_key_checking=false 3 | private_key_file=benchmark_message_alicloud.pem 4 | 5 | [privilege_escalation] 6 | become=true 7 | become_method='sudo' 8 | become_user='root' 9 | -------------------------------------------------------------------------------- /driver-kafka/deploy/hdd-deployment/alicloud/terraform.tfvars: -------------------------------------------------------------------------------- 1 | region = "cn-shenzhen" 2 | availability_zone = "cn-shenzhen-b" 3 | private_key_file = "benchmark_message_alicloud.pem" 4 | key_name = "key-pair-from-terraform-benchmark-kafka" 5 | image_id = "centos_7_04_64_20G_alibase_201701015.vhd" 6 | 7 | 8 | 9 | instance_types = { 10 | "kafka" = "ecs.i1.xlarge" 11 | "zookeeper" = "ecs.se1.xlarge" 12 | "client" = "ecs.se1.xlarge" 13 | } 14 | 15 | num_instances = { 16 | "client" = 4 17 | "kafka" = 3 18 | "zookeeper" = 3 19 | } 20 | -------------------------------------------------------------------------------- /driver-kafka/deploy/hdd-deployment/ansible.cfg: -------------------------------------------------------------------------------- 1 | [defaults] 2 | host_key_checking=false 3 | private_key_file=~/.ssh/kafka_aws 4 | 5 | [privilege_escalation] 6 | become=true 7 | become_method='sudo' 8 | become_user='root' -------------------------------------------------------------------------------- /driver-kafka/deploy/hdd-deployment/templates/benchmark-worker.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Benchmark worker 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/benchmark/bin/benchmark-worker 7 | WorkingDirectory=/opt/benchmark 8 | RestartSec=1s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | 13 | [Install] 14 | WantedBy=multi-user.target 15 | -------------------------------------------------------------------------------- /driver-kafka/deploy/hdd-deployment/templates/kafka.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Kafka 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/kafka/bin/kafka-server-start.sh config/server.properties 7 | Environment='KAFKA_HEAP_OPTS=-Xms6g -Xmx6g -XX:MetaspaceSize=96m' 8 | Environment='KAFKA_JVM_PERFORMANCE_OPTS=-server -XX:+UseG1GC -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:G1HeapRegionSize=16M -XX:MinMetaspaceFreeRatio=50 -XX:MaxMetaspaceFreeRatio=80 -Djava.awt.headless=true' 9 | WorkingDirectory=/opt/kafka 10 | RestartSec=1s 11 | Restart=on-failure 12 | Type=simple 13 | LimitNOFILE=300000 14 | 15 | [Install] 16 | WantedBy=multi-user.target 17 | -------------------------------------------------------------------------------- /driver-kafka/deploy/hdd-deployment/templates/myid: -------------------------------------------------------------------------------- 1 | {{ zid }} 2 | -------------------------------------------------------------------------------- /driver-kafka/deploy/hdd-deployment/templates/server.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | 16 | broker.id={{ brokerId }} 17 | 18 | advertised.listeners=PLAINTEXT://{{ privateIp }}:9092 19 | 20 | log.dirs=/mnt/data-1,/mnt/data-2 21 | 22 | zookeeper.connect={{ zookeeperServers }} 23 | 24 | num.replica.fetchers=8 25 | 26 | message.max.bytes=10485760 27 | 28 | replica.fetch.max.bytes=10485760 29 | 30 | num.network.threads=8 -------------------------------------------------------------------------------- /driver-kafka/deploy/hdd-deployment/templates/workers.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | 16 | workers: 17 | {% for worker in groups['client'] %} 18 | - http://{{ hostvars[worker].private_ip }}:8080 19 | {% endfor %} 20 | -------------------------------------------------------------------------------- /driver-kafka/deploy/hdd-deployment/templates/zookeeper.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=ZooKeeper 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/kafka/bin/zookeeper-server-start.sh config/zookeeper.properties 7 | WorkingDirectory=/opt/kafka 8 | RestartSec=1s 9 | Restart=on-failure 10 | Type=simple 11 | 12 | [Install] 13 | WantedBy=multi-user.target 14 | -------------------------------------------------------------------------------- /driver-kafka/deploy/hdd-deployment/terraform.tfvars: -------------------------------------------------------------------------------- 1 | public_key_path = "~/.ssh/kafka_aws.pub" 2 | region = "us-west-2" 3 | az = "us-west-2a" 4 | ami = "ami-9fa343e7" // RHEL-7.4 5 | 6 | instance_types = { 7 | "kafka" = "d2.4xlarge" 8 | "zookeeper" = "t2.small" 9 | "client" = "c5.4xlarge" 10 | } 11 | 12 | num_instances = { 13 | "client" = 4 14 | "kafka" = 3 15 | "zookeeper" = 3 16 | } 17 | -------------------------------------------------------------------------------- /driver-kafka/deploy/ssd-deployment/alicloud/ansible.cfg: -------------------------------------------------------------------------------- 1 | [defaults] 2 | host_key_checking=false 3 | private_key_file=benchmark_message_alicloud.pem 4 | 5 | [privilege_escalation] 6 | become=true 7 | become_method='sudo' 8 | become_user='root' 9 | -------------------------------------------------------------------------------- /driver-kafka/deploy/ssd-deployment/alicloud/terraform.tfvars: -------------------------------------------------------------------------------- 1 | region = "cn-shenzhen" 2 | availability_zone = "cn-shenzhen-b" 3 | private_key_file = "benchmark_message_alicloud.pem" 4 | key_name = "key-pair-from-terraform-benchmark-kafka" 5 | image_id = "centos_7_04_64_20G_alibase_201701015.vhd" 6 | 7 | 8 | 9 | instance_types = { 10 | "kafka" = "ecs.i1.xlarge" 11 | "zookeeper" = "ecs.se1.xlarge" 12 | "client" = "ecs.se1.xlarge" 13 | } 14 | 15 | num_instances = { 16 | "client" = 4 17 | "kafka" = 3 18 | "zookeeper" = 3 19 | } 20 | -------------------------------------------------------------------------------- /driver-kafka/deploy/ssd-deployment/ansible.cfg: -------------------------------------------------------------------------------- 1 | [defaults] 2 | host_key_checking=false 3 | private_key_file=~/.ssh/kafka_aws 4 | 5 | [privilege_escalation] 6 | become=true 7 | become_method='sudo' 8 | become_user='root' 9 | 10 | [ssh_connection] 11 | ssh_args=-o ServerAliveInterval=60 12 | retries=10 13 | -------------------------------------------------------------------------------- /driver-kafka/deploy/ssd-deployment/templates/benchmark-worker.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Benchmark worker 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/benchmark/bin/benchmark-worker 7 | WorkingDirectory=/opt/benchmark 8 | RestartSec=1s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | 13 | [Install] 14 | WantedBy=multi-user.target 15 | -------------------------------------------------------------------------------- /driver-kafka/deploy/ssd-deployment/templates/kafka.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Kafka 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/kafka/bin/kafka-server-start.sh config/server.properties 7 | Environment='KAFKA_HEAP_OPTS=-Xms16g -Xmx16g -XX:MetaspaceSize=96m' 8 | Environment='KAFKA_JVM_PERFORMANCE_OPTS=-server -XX:+UnlockExperimentalVMOptions -XX:+UseZGC -XX:+ParallelRefProcEnabled -XX:+AggressiveOpts -XX:+DoEscapeAnalysis -XX:ParallelGCThreads=12 -XX:ConcGCThreads=12 -XX:+DisableExplicitGC -XX:-ResizePLAB -XX:MinMetaspaceFreeRatio=50 -XX:MaxMetaspaceFreeRatio=80 -Djava.awt.headless=true' 9 | WorkingDirectory=/opt/kafka 10 | RestartSec=1s 11 | Restart=on-failure 12 | Type=simple 13 | LimitNOFILE=300000 14 | 15 | [Install] 16 | WantedBy=multi-user.target 17 | -------------------------------------------------------------------------------- /driver-kafka/deploy/ssd-deployment/templates/myid: -------------------------------------------------------------------------------- 1 | {{ zid }} 2 | -------------------------------------------------------------------------------- /driver-kafka/deploy/ssd-deployment/templates/server.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | 16 | broker.id={{ brokerId }} 17 | 18 | advertised.listeners=PLAINTEXT://{{ privateIp }}:9092 19 | 20 | log.dirs=/mnt/data-1,/mnt/data-2 21 | 22 | zookeeper.connect={{ zookeeperServers }} 23 | 24 | num.replica.fetchers=8 25 | 26 | message.max.bytes=10485760 27 | 28 | replica.fetch.max.bytes=10485760 29 | 30 | num.network.threads=8 -------------------------------------------------------------------------------- /driver-kafka/deploy/ssd-deployment/templates/workers.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | 16 | workers: 17 | {% for worker in groups['client'] %} 18 | - http://{{ hostvars[worker].private_ip }}:8080 19 | {% endfor %} 20 | -------------------------------------------------------------------------------- /driver-kafka/deploy/ssd-deployment/templates/zookeeper.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=ZooKeeper 3 | After=network.target 4 | 5 | [Service] 6 | Environment='KAFKA_HEAP_OPTS=-Xms32g -Xmx32g' 7 | ExecStart=/opt/kafka/bin/zookeeper-server-start.sh config/zookeeper.properties 8 | WorkingDirectory=/opt/kafka 9 | RestartSec=1s 10 | Restart=on-failure 11 | Type=simple 12 | 13 | [Install] 14 | WantedBy=multi-user.target 15 | -------------------------------------------------------------------------------- /driver-kafka/deploy/ssd-deployment/terraform.tfvars: -------------------------------------------------------------------------------- 1 | public_key_path = "~/.ssh/kafka_aws.pub" 2 | region = "us-west-2" 3 | az = "us-west-2a" 4 | ami = "ami-08970fb2e5767e3b8" // RHEL-8 5 | 6 | instance_types = { 7 | "kafka" = "i3en.6xlarge" 8 | "zookeeper" = "i3en.2xlarge" 9 | "client" = "m5n.8xlarge" 10 | } 11 | 12 | num_instances = { 13 | "client" = 4 14 | "kafka" = 3 15 | "zookeeper" = 3 16 | } 17 | -------------------------------------------------------------------------------- /driver-kafka/src/main/java/io/openmessaging/benchmark/driver/kafka/Config.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark.driver.kafka; 15 | 16 | public class Config { 17 | public short replicationFactor; 18 | 19 | public String topicConfig; 20 | 21 | public String commonConfig; 22 | 23 | public String producerConfig; 24 | 25 | public String consumerConfig; 26 | } 27 | -------------------------------------------------------------------------------- /driver-kop/kafka_to_pulsar.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: Kafka producer and Pulsar consumer 16 | driverClass: io.openmessaging.benchmark.driver.kop.KopBenchmarkDriver 17 | 18 | producerType: kafka 19 | consumerType: pulsar 20 | 21 | # Pulsar configs 22 | pulsarConfig: 23 | serviceUrl: pulsar://localhost:6650 24 | batchingMaxPublishDelayMs: 1 25 | batchingMaxBytes: 1048576 26 | 27 | # Kafka configs 28 | kafkaConfig: | 29 | bootstrap.servers=localhost:9092 30 | linger.ms=1 31 | batch.size=1048576 32 | -------------------------------------------------------------------------------- /driver-kop/pulsar_to_kafka.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: Kafka producer and Pulsar consumer 16 | driverClass: io.openmessaging.benchmark.driver.kop.KopBenchmarkDriver 17 | 18 | producerType: pulsar 19 | consumerType: kafka 20 | 21 | # Pulsar configs 22 | pulsarConfig: 23 | serviceUrl: pulsar://localhost:6650 24 | batchingMaxPublishDelayMs: 1 25 | batchingMaxBytes: 1048576 26 | 27 | # Kafka configs 28 | kafkaConfig: | 29 | bootstrap.servers=localhost:9092 30 | linger.ms=1 31 | batch.size=1048576 32 | -------------------------------------------------------------------------------- /driver-kop/src/main/java/io/openmessaging/benchmark/driver/kop/config/ClientType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark.driver.kop.config; 15 | 16 | 17 | import com.fasterxml.jackson.annotation.JsonProperty; 18 | 19 | public enum ClientType { 20 | @JsonProperty("kafka") 21 | KAFKA, 22 | @JsonProperty("pulsar") 23 | PULSAR 24 | } 25 | -------------------------------------------------------------------------------- /driver-kop/src/test/resources/kop_required.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | # The KoP config file with required config items. 16 | 17 | name: Kafka-on-Pulsar 18 | driverClass: io.openmessaging.benchmark.driver.kop.KopBenchmarkDriver 19 | 20 | producerType: pulsar 21 | consumerType: kafka 22 | 23 | # Pulsar configs 24 | pulsarConfig: 25 | serviceUrl: pulsar://localhost:6650 26 | 27 | # Kafka configs 28 | kafkaConfig: | 29 | bootstrap.servers=localhost:9092 30 | -------------------------------------------------------------------------------- /driver-kop/src/test/resources/kop_wrong_kafka_config.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | # The KoP config file whose kafka config is wrong (without required configs) 16 | 17 | name: Kafka-on-Pulsar 18 | driverClass: io.openmessaging.benchmark.driver.kop.KopBenchmarkDriver 19 | 20 | kafkaConfig: 21 | linger.ms=1 22 | -------------------------------------------------------------------------------- /driver-nats-streaming/deploy/ali/ansible.cfg: -------------------------------------------------------------------------------- 1 | [defaults] 2 | host_key_checking=false 3 | private_key_file=alicloud.pem 4 | -------------------------------------------------------------------------------- /driver-nats-streaming/deploy/ali/terraform.tfvars: -------------------------------------------------------------------------------- 1 | region = "cn-shenzhen" 2 | availability_zone = "cn-shenzhen-c" 3 | private_key_file = "alicloud.pem" 4 | key_name = "key-pair-from-terraform-nats-streaming" 5 | image_id = "centos_7_04_64_20G_alibase_201701015.vhd" 6 | 7 | instance_types = { 8 | "nats-streaming-server" = "ecs.se1.4xlarge" #4c16g 9 | "client" = "ecs.n4.4xlarge" 10 | } 11 | 12 | num_instances = { 13 | "nats-streaming-server" = 3 14 | "client" = 2 15 | } -------------------------------------------------------------------------------- /driver-nats-streaming/deploy/ansible.cfg: -------------------------------------------------------------------------------- 1 | [defaults] 2 | host_key_checking=false 3 | private_key_file=~/.ssh/nats_streaming_aws 4 | 5 | [privilege_escalation] 6 | become=true 7 | become_method='sudo' 8 | become_user='root' -------------------------------------------------------------------------------- /driver-nats-streaming/deploy/templates/benchmark-worker.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=benchmark worker 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/benchmark/bin/benchmark-worker 7 | WorkingDirectory=/opt/benchmark 8 | Environment='-Djava.security.egd=file:/dev/./urandom' 9 | RestartSec=1s 10 | Restart=on-failure 11 | Type=simple 12 | LimitNOFILE=300000 13 | 14 | [Install] 15 | WantedBy=multi-user.target 16 | -------------------------------------------------------------------------------- /driver-nats-streaming/deploy/templates/cluster.conf: -------------------------------------------------------------------------------- 1 | 2 | # 3 | # Licensed under the Apache License, Version 2.0 (the "License"); 4 | # you may not use this file except in compliance with the License. 5 | # You may obtain a copy of the License at 6 | # 7 | # http://www.apache.org/licenses/LICENSE-2.0 8 | # 9 | # Unless required by applicable law or agreed to in writing, software 10 | # distributed under the License is distributed on an "AS IS" BASIS, 11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | # See the License for the specific language governing permissions and 13 | # limitations under the License. 14 | # 15 | 16 | # Cluster Seed Node 17 | listen: localhost:4222 18 | http_port: 8222 19 | cluster { 20 | listen: localhost:6222 21 | routes: ["nats://natsHostUrl:6222"] 22 | } 23 | 24 | # NATS Streaming specific configuration 25 | streaming { 26 | id: mycluster 27 | store: file 28 | dir: /mnt/data 29 | cluster { 30 | bootstrap: false 31 | # sync: true 32 | # log_cache_size: 1024 33 | } 34 | } -------------------------------------------------------------------------------- /driver-nats-streaming/deploy/templates/grafana-dashboards.yml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | apiVersion: 1 16 | 17 | providers: 18 | - name: 'nats-streaming' 19 | orgId: 1 20 | folder: '' 21 | type: file 22 | disableDeletion: true 23 | options: 24 | path: /dashboards -------------------------------------------------------------------------------- /driver-nats-streaming/deploy/templates/grafana-datasource.yml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | apiVersion: 1 16 | 17 | datasources: 18 | {% for prom in groups['prometheus'] %} 19 | - name: {{ hostvars[prom].inventory_hostname }} 20 | url: http://{{ hostvars[prom].private_ip }}:9090 21 | type: prometheus 22 | access: proxy 23 | orgId: 1 24 | isDefault: true 25 | version: 1 26 | editable: false 27 | {% endfor %} 28 | -------------------------------------------------------------------------------- /driver-nats-streaming/deploy/templates/nats-streaming-dashboard.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=NATS Streaming Dashboard 3 | After=prometheus.service 4 | Requires=prometheus.service 5 | 6 | [Service] 7 | WorkingDirectory=/opt/grafana 8 | ExecStartPre=/usr/bin/docker pull grafana/grafana:8.3.4 9 | ExecStart=/usr/bin/docker run \ 10 | --restart=always \ 11 | --name=systemd_nats_streaming_dashboard \ 12 | -p3000:3000 \ 13 | -e GF_INSTALL_PLUGINS="flant-statusmap-panel,grafana-piechart-panel" \ 14 | -v /opt/nats-streaming/grafana-datasource.yml:/etc/grafana/provisioning/datasources/prometheus.yaml \ 15 | -v /opt/nats-streaming/dashboards.yml:/etc/grafana/provisioning/dashboards/nats-streaming.yaml \ 16 | -v /repos/prometheus-nats-exporter/walkthrough:/dashboards \ 17 | grafana/grafana:8.3.4 18 | ExecStop=/usr/bin/docker stop systemd_nats_streaming_dashboard 19 | ExecStopPost=/usr/bin/docker rm -f systemd_nats_streaming_dashboard 20 | ExecReload=/usr/bin/docker restart systemd_nats_streaming_dashboard 21 | 22 | [Install] 23 | WantedBy=multi-user.target 24 | -------------------------------------------------------------------------------- /driver-nats-streaming/deploy/templates/nats-streaming-metrics.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=NATS Streaming Metrics 3 | After=nats-streaming-server.service 4 | Requires=nats-streaming-server.service 5 | 6 | [Service] 7 | WorkingDirectory=/opt/nats-streaming-metrics 8 | ExecStartPre=/usr/bin/docker pull natsio/prometheus-nats-exporter:latest 9 | ExecStart=/usr/bin/docker run \ 10 | --restart=always \ 11 | --name=systemd_nats_streaming_metrics \ 12 | -p9090:9090 \ 13 | natsio/prometheus-nats-exporter:latest \ 14 | -channelz \ 15 | -serverz \ 16 | -varz \ 17 | -subz \ 18 | -channelz \ 19 | -connz \ 20 | -p 9090 \ 21 | "http://localhost:8222" 22 | ExecStop=/usr/bin/docker stop systemd_nats_streaming_metrics 23 | ExecStopPost=/usr/bin/docker rm -f systemd_nats_streaming_metrics 24 | ExecReload=/usr/bin/docker restart systemd_nats_streaming_metrics 25 | 26 | [Install] 27 | WantedBy=multi-user.target 28 | -------------------------------------------------------------------------------- /driver-nats-streaming/deploy/templates/nats-streaming-server.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=nats broker 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/nats-streaming/nats-streaming-server-v0.25.2-linux-amd64/nats-streaming-server -c /opt/nats-streaming/cluster.conf 7 | WorkingDirectory=/opt/nats-streaming/ 8 | RestartSec=1s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | 13 | [Install] 14 | WantedBy=multi-user.target -------------------------------------------------------------------------------- /driver-nats-streaming/deploy/templates/prometheus.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Prometheus server 3 | After=network.target 4 | 5 | [Service] 6 | WorkingDirectory=/opt/prometheus 7 | ExecStart=/opt/prometheus/prometheus \ 8 | --web.enable-admin-api \ 9 | --storage.tsdb.path=/opt/prometheus/data \ 10 | --config.file=/opt/prometheus/prometheus.yml 11 | 12 | [Install] 13 | WantedBy=multi-user.target 14 | -------------------------------------------------------------------------------- /driver-nats-streaming/deploy/templates/workers.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | 16 | workers: 17 | {% for worker in groups['client'] %} 18 | - http://{{ hostvars[worker].private_ip }}:8080 19 | {% endfor %} 20 | -------------------------------------------------------------------------------- /driver-nats-streaming/deploy/terraform.tfvars: -------------------------------------------------------------------------------- 1 | public_key_path = "~/.ssh/nats_streaming_aws.pub" 2 | region = "us-west-2" 3 | az = "us-west-2a" 4 | ami = "ami-08970fb2e5767e3b8" // RHEL-8 5 | 6 | instance_types = { 7 | "nats" = "i3en.6xlarge" 8 | "client" = "m5n.8xlarge" 9 | "prometheus" = "t2.large" 10 | } 11 | 12 | num_instances = { 13 | "nats" = 3 14 | "client" = 4 15 | "prometheus" = 1 16 | } 17 | -------------------------------------------------------------------------------- /driver-nats-streaming/nats-streaming.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | 16 | name: Nats Streaming 17 | driverClass: io.openmessaging.benchmark.driver.natsStreaming.NatsStreamingBenchmarkDriver 18 | 19 | # RabbitMq client specific configurations 20 | 21 | natsHostUrl: nats://localhost:4222 22 | clusterId: mycluster 23 | 24 | 25 | -------------------------------------------------------------------------------- /driver-nats-streaming/src/main/java/io/openmessaging/benchmark/driver/natsStreaming/NatsStreamingClientConfig.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark.driver.natsStreaming; 15 | 16 | public class NatsStreamingClientConfig { 17 | public String natsHostUrl; 18 | public String clusterId; 19 | 20 | public int maxPubAcksInFlight = 16384; 21 | 22 | public int maxInFlight = 1024; 23 | } 24 | -------------------------------------------------------------------------------- /driver-nats/deploy/alicloud/ansible.cfg: -------------------------------------------------------------------------------- 1 | [defaults] 2 | host_key_checking=false 3 | private_key_file=alicloud.pem 4 | -------------------------------------------------------------------------------- /driver-nats/deploy/alicloud/terraform.tfvars: -------------------------------------------------------------------------------- 1 | region = "cn-shenzhen" 2 | availability_zone = "cn-shenzhen-c" 3 | private_key_file = "alicloud.pem" 4 | key_name = "key-pair-from-terraform-nats" 5 | image_id = "centos_7_04_64_20G_alibase_201701015.vhd" 6 | 7 | instance_types = { 8 | "nats" = "ecs.se1.4xlarge" #4c16g 9 | "client" = "ecs.n4.4xlarge" 10 | } 11 | 12 | num_instances = { 13 | "nats" = 1 14 | "client" = 2 15 | } -------------------------------------------------------------------------------- /driver-nats/deploy/ansible.cfg: -------------------------------------------------------------------------------- 1 | [defaults] 2 | host_key_checking=false 3 | private_key_file=~/.ssh/nats_aws 4 | 5 | [privilege_escalation] 6 | become=true 7 | become_method='sudo' 8 | become_user='root' -------------------------------------------------------------------------------- /driver-nats/deploy/restart-brokers.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | - name: Restart brokers 16 | hosts: nats 17 | connection: ssh 18 | become: true 19 | tasks: 20 | - name: NATS - Start service 21 | systemd: 22 | state: restarted 23 | name: "nats-server" 24 | -------------------------------------------------------------------------------- /driver-nats/deploy/restart-workers.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | - name: Benchmarking worker restart 16 | hosts: client 17 | connection: ssh 18 | become: true 19 | tasks: 20 | - name: Benchmark - Start service 21 | systemd: 22 | state: restarted 23 | daemon_reload: yes 24 | name: "benchmark-worker" 25 | -------------------------------------------------------------------------------- /driver-nats/deploy/templates/benchmark-worker.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=benchmark worker 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/benchmark/bin/benchmark-worker 7 | WorkingDirectory=/opt/benchmark 8 | Environment='-Djava.security.egd=file:/dev/./urandom' 9 | RestartSec=1s 10 | Restart=on-failure 11 | Type=simple 12 | LimitNOFILE=300000 13 | 14 | [Install] 15 | WantedBy=multi-user.target 16 | -------------------------------------------------------------------------------- /driver-nats/deploy/templates/grafana-dashboards.yml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | apiVersion: 1 16 | 17 | providers: 18 | - name: 'nats' 19 | orgId: 1 20 | folder: '' 21 | type: file 22 | disableDeletion: true 23 | options: 24 | path: /dashboards -------------------------------------------------------------------------------- /driver-nats/deploy/templates/grafana-datasource.yml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | apiVersion: 1 16 | 17 | datasources: 18 | {% for prom in groups['prometheus'] %} 19 | - name: NATS-Prometheus 20 | url: http://{{ hostvars[prom].private_ip }}:9090 21 | type: prometheus 22 | access: proxy 23 | orgId: 1 24 | isDefault: true 25 | version: 1 26 | editable: false 27 | {% endfor %} 28 | -------------------------------------------------------------------------------- /driver-nats/deploy/templates/nats-dashboard.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=NATS Dashboard 3 | After=prometheus.service 4 | Requires=prometheus.service 5 | 6 | [Service] 7 | WorkingDirectory=/opt/grafana 8 | ExecStartPre=/usr/bin/docker pull grafana/grafana:8.3.4 9 | ExecStart=/usr/bin/docker run \ 10 | --restart=always \ 11 | --name=systemd_nats_dashboard \ 12 | -p3000:3000 \ 13 | -e GF_INSTALL_PLUGINS="flant-statusmap-panel,grafana-piechart-panel" \ 14 | -v /opt/nats/grafana-datasource.yml:/etc/grafana/provisioning/datasources/prometheus.yaml \ 15 | -v /opt/nats/dashboards.yml:/etc/grafana/provisioning/dashboards/nats.yaml \ 16 | -v /repos/prometheus-nats-exporter/walkthrough:/dashboards \ 17 | grafana/grafana:8.3.4 18 | ExecStop=/usr/bin/docker stop systemd_nats_dashboard 19 | ExecStopPost=/usr/bin/docker rm -f systemd_nats_dashboard 20 | ExecReload=/usr/bin/docker restart systemd_nats_dashboard 21 | 22 | [Install] 23 | WantedBy=multi-user.target 24 | -------------------------------------------------------------------------------- /driver-nats/deploy/templates/nats-metrics.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=NATS Metrics 3 | After=nats-server.service 4 | Requires=nats-server.service 5 | 6 | [Service] 7 | WorkingDirectory=/opt/nats-metrics 8 | ExecStartPre=/usr/bin/docker pull natsio/prometheus-nats-exporter:latest 9 | ExecStart=/usr/bin/docker run \ 10 | --restart=always \ 11 | --name=systemd_nats_metrics \ 12 | -p9090:9090 \ 13 | natsio/prometheus-nats-exporter:latest \ 14 | -channelz \ 15 | -serverz \ 16 | -varz \ 17 | -subz \ 18 | -channelz \ 19 | -connz \ 20 | -jsz all \ 21 | -p 9090 \ 22 | "http://{{ hostvars[inventory_hostname]['ansible_default_ipv4']['address'] }}:8222" 23 | ExecStop=/usr/bin/docker stop systemd_nats_metrics 24 | ExecStopPost=/usr/bin/docker rm -f systemd_nats_metrics 25 | ExecReload=/usr/bin/docker restart systemd_nats_metrics 26 | 27 | [Install] 28 | WantedBy=multi-user.target 29 | -------------------------------------------------------------------------------- /driver-nats/deploy/templates/nats-server.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=nats broker 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/usr/local/bin/nats-server -c /opt/nats/cluster.conf 7 | WorkingDirectory=/opt/nats/ 8 | RestartSec=1s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | 13 | [Install] 14 | WantedBy=multi-user.target -------------------------------------------------------------------------------- /driver-nats/deploy/templates/nats.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | 16 | name: NATS 17 | driverClass: io.openmessaging.benchmark.driver.nats.NatsBenchmarkDriver 18 | 19 | # NATS client specific configurations 20 | 21 | workers: 22 | {% for worker in groups['nats'] %} 23 | - nats://{{ hostvars[worker].private_ip }}:4222 24 | {% endfor %} 25 | 26 | natsHostUrl: nats://localhost:4222 27 | -------------------------------------------------------------------------------- /driver-nats/deploy/templates/prometheus.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Prometheus server 3 | After=network.target 4 | 5 | [Service] 6 | WorkingDirectory=/opt/prometheus 7 | ExecStart=/opt/prometheus/prometheus \ 8 | --web.enable-admin-api \ 9 | --storage.tsdb.path=/opt/prometheus/data \ 10 | --config.file=/opt/prometheus/prometheus.yml 11 | 12 | [Install] 13 | WantedBy=multi-user.target 14 | -------------------------------------------------------------------------------- /driver-nats/deploy/templates/workers.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | 16 | workers: 17 | {% for worker in groups['client'] %} 18 | - http://{{ hostvars[worker].private_ip }}:8080 19 | {% endfor %} 20 | -------------------------------------------------------------------------------- /driver-nats/deploy/terraform.tfvars: -------------------------------------------------------------------------------- 1 | public_key_path = "~/.ssh/nats_aws.pub" 2 | region = "us-west-2" 3 | az = "us-west-2a" 4 | ami = "ami-08970fb2e5767e3b8" // RHEL-8 5 | 6 | instance_types = { 7 | "nats" = "i3en.6xlarge" 8 | "client" = "m5n.8xlarge" 9 | "prometheus" = "t2.large" 10 | } 11 | 12 | num_instances = { 13 | "nats" = 3 14 | "client" = 4 15 | "prometheus" = 1 16 | } 17 | -------------------------------------------------------------------------------- /driver-nats/nats.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: NATS 16 | driverClass: io.openmessaging.benchmark.driver.nats.NatsBenchmarkDriver 17 | 18 | natsHostUrl: localhost:4222 19 | 20 | replicationFactor: 3 21 | -------------------------------------------------------------------------------- /driver-nats/src/main/java/io/openmessaging/benchmark/driver/nats/NatsBenchmarkConsumer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark.driver.nats; 15 | 16 | 17 | import io.openmessaging.benchmark.driver.BenchmarkConsumer; 18 | 19 | public class NatsBenchmarkConsumer implements BenchmarkConsumer { 20 | @Override 21 | public void close() {} 22 | } 23 | -------------------------------------------------------------------------------- /driver-nats/src/main/java/io/openmessaging/benchmark/driver/nats/NatsConfig.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark.driver.nats; 15 | 16 | 17 | import io.nats.client.api.StorageType; 18 | 19 | public class NatsConfig { 20 | public String natsHostUrl; 21 | 22 | public int replicationFactor; 23 | 24 | public StorageType storageType = StorageType.File; 25 | 26 | // -1 is unlimited 27 | public int maxBytes = -1; 28 | } 29 | -------------------------------------------------------------------------------- /driver-nsq/deploy/ansible.cfg: -------------------------------------------------------------------------------- 1 | [defaults] 2 | host_key_checking=false 3 | private_key_file=alicloud.pem 4 | -------------------------------------------------------------------------------- /driver-nsq/deploy/templates/benchmark-worker.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Pulsar Broker 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/benchmark/bin/benchmark-worker 7 | WorkingDirectory=/opt/benchmark 8 | RestartSec=1s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | 13 | [Install] 14 | WantedBy=multi-user.target 15 | -------------------------------------------------------------------------------- /driver-nsq/deploy/templates/nsqd-server.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Nsq broker 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/nsq/bin/nsqd -lookupd-tcp-address=localhost:4160 -broadcast-address=localhost 7 | WorkingDirectory=/opt/nsq/ 8 | RestartSec=1s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | 13 | [Install] 14 | WantedBy=multi-user.target -------------------------------------------------------------------------------- /driver-nsq/deploy/templates/nsqlookupd-server.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Nsqlookupd broker 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/nsq/bin/nsqlookupd -broadcast-address=localhost 7 | WorkingDirectory=/opt/nsq/ 8 | RestartSec=1s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | 13 | [Install] 14 | WantedBy=multi-user.target -------------------------------------------------------------------------------- /driver-nsq/deploy/templates/workers.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | 16 | workers: 17 | {% for worker in groups['client'] %} 18 | - http://{{ hostvars[worker].private_ip }}:8080 19 | {% endfor %} 20 | -------------------------------------------------------------------------------- /driver-nsq/deploy/terraform.tfvars: -------------------------------------------------------------------------------- 1 | region = "cn-shenzhen" 2 | availability_zone = "cn-shenzhen-c" 3 | private_key_file = "alicloud.pem" 4 | key_name = "key-pair-from-terraform-nsq" 5 | image_id = "centos_7_04_64_20G_alibase_201701015.vhd" 6 | 7 | instance_types = { 8 | "nsq" = "ecs.se1.4xlarge" #4c16g 9 | "client" = "ecs.n4.4xlarge" 10 | } 11 | 12 | num_instances = { 13 | "nsq" = 1 14 | "client" = 2 15 | } -------------------------------------------------------------------------------- /driver-nsq/nsq.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | 16 | name: Nsq 17 | driverClass: io.openmessaging.benchmark.driver.nsq.NsqBenchmarkDriver 18 | 19 | # RabbitMq client specific configurations 20 | 21 | nsqdHost: localhost 22 | lookupHost: localhost -------------------------------------------------------------------------------- /driver-nsq/src/main/java/io/openmessaging/benchmark/driver/nsq/NsqConfig.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark.driver.nsq; 15 | 16 | public class NsqConfig { 17 | public String nsqdHost; 18 | public String lookupHost; 19 | } 20 | -------------------------------------------------------------------------------- /driver-pravega/deploy/ansible.cfg: -------------------------------------------------------------------------------- 1 | [defaults] 2 | host_key_checking=false 3 | private_key_file=~/.ssh/pravega_aws 4 | any_errors_fatal=true 5 | 6 | [privilege_escalation] 7 | become=true 8 | become_method='sudo' 9 | become_user='root' 10 | -------------------------------------------------------------------------------- /driver-pravega/deploy/templates/benchmark-worker.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=OpenMessaging Benchmark Worker 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/benchmark/bin/benchmark-worker 7 | WorkingDirectory=/opt/benchmark 8 | RestartSec=15s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | 13 | [Install] 14 | WantedBy=multi-user.target 15 | -------------------------------------------------------------------------------- /driver-pravega/deploy/templates/bookkeeper.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Bookkeeper 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/bookkeeper/bin/bookkeeper bookie 7 | WorkingDirectory=/opt/bookkeeper 8 | RestartSec=15s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | 13 | [Install] 14 | WantedBy=multi-user.target 15 | -------------------------------------------------------------------------------- /driver-pravega/deploy/templates/influxdb.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=InfluxDB 3 | After=docker.service 4 | Requires=docker.service 5 | 6 | [Service] 7 | ExecStartPre=/usr/bin/docker pull influxdb:1.8.10 8 | 9 | ExecStart=\ 10 | /bin/bash -c "\ 11 | docker start --attach systemd_influxdb || \ 12 | docker run --restart=always --name=systemd_influxdb -p 8086:8086 \ 13 | influxdb:1.8.10 \ 14 | " 15 | 16 | ExecStop=/usr/bin/docker stop systemd_influxdb 17 | ExecReload=/usr/bin/docker restart systemd_influxdb 18 | 19 | [Install] 20 | WantedBy=multi-user.target 21 | -------------------------------------------------------------------------------- /driver-pravega/deploy/templates/myid: -------------------------------------------------------------------------------- 1 | {{ zid }} 2 | -------------------------------------------------------------------------------- /driver-pravega/deploy/templates/pravega-controller.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Pravega Controller 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/pravega/bin/pravega-controller 7 | WorkingDirectory=/opt/pravega 8 | RestartSec=15s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | TimeoutStopSec=5 13 | 14 | [Install] 15 | WantedBy=multi-user.target 16 | -------------------------------------------------------------------------------- /driver-pravega/deploy/templates/pravega-dashboard.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Pravega Dashboard 3 | After=docker.service 4 | Requires=docker.service 5 | 6 | [Service] 7 | ExecStartPre=/usr/bin/docker pull grafana/grafana:latest 8 | 9 | # TODO: set random password 10 | 11 | ExecStart=\ 12 | /bin/bash -c "\ 13 | docker start --attach systemd_pravega_dashboard || \ 14 | docker run --restart=always --name=systemd_pravega_dashboard -p 3000:3000 \ 15 | --network=host \ 16 | -e GF_SECURITY_ADMIN_PASSWORD=pravega \ 17 | grafana/grafana:latest \ 18 | " 19 | 20 | ExecStop=/usr/bin/docker stop systemd_pravega_dashboard 21 | ExecReload=/usr/bin/docker restart systemd_pravega_dashboard 22 | 23 | [Install] 24 | WantedBy=multi-user.target 25 | -------------------------------------------------------------------------------- /driver-pravega/deploy/templates/pravega-segmentstore.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Pravega Segment Store 3 | After=network.target 4 | 5 | [Service] 6 | Environment="PRAVEGA_SEGMENTSTORE_OPTS=-Xms6G -Xmx6G -XX:MaxDirectMemorySize=24g" 7 | ExecStart=/opt/pravega/bin/pravega-segmentstore 8 | WorkingDirectory=/opt/pravega 9 | RestartSec=15s 10 | Restart=on-failure 11 | Type=simple 12 | LimitNOFILE=300000 13 | 14 | [Install] 15 | WantedBy=multi-user.target 16 | -------------------------------------------------------------------------------- /driver-pravega/deploy/templates/prometheus.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Prometheus server 3 | After=network.target 4 | 5 | [Service] 6 | WorkingDirectory=/opt/prometheus 7 | ExecStart=/opt/prometheus/prometheus \ 8 | --storage.tsdb.path=/opt/prometheus/data \ 9 | --config.file=/opt/prometheus/prometheus.yml 10 | 11 | [Install] 12 | WantedBy=multi-user.target 13 | -------------------------------------------------------------------------------- /driver-pravega/deploy/templates/workers.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | 16 | workers: 17 | {% for worker in groups['client'] %} 18 | - http://{{ hostvars[worker].private_ip }}:8080 19 | {% endfor %} 20 | -------------------------------------------------------------------------------- /driver-pravega/deploy/templates/zookeeper.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=ZooKeeper Local 3 | After=network.target 4 | 5 | [Service] 6 | Environment=OPTS=-Dstats_server_port=2182 7 | ExecStart=/opt/zookeeper/bin/zkServer.sh start-foreground 8 | WorkingDirectory=/opt/zookeeper 9 | RestartSec=15s 10 | Restart=on-failure 11 | Type=simple 12 | LimitNOFILE=300000 13 | 14 | [Install] 15 | WantedBy=multi-user.target 16 | -------------------------------------------------------------------------------- /driver-pravega/deploy/terraform.tfvars: -------------------------------------------------------------------------------- 1 | public_key_path = "~/.ssh/pravega_aws.pub" 2 | region = "us-east-2" 3 | ami = "ami-0bb2449c2217cb9b0" // RHEL-7.9 us-east-2 4 | 5 | instance_types = { 6 | "controller" = "m5.large" 7 | "bookkeeper" = "i3en.2xlarge" 8 | "zookeeper" = "t2.small" 9 | "client" = "m5n.xlarge" 10 | "metrics" = "t2.large" 11 | } 12 | 13 | num_instances = { 14 | "controller" = 1 15 | "bookkeeper" = 3 16 | "zookeeper" = 3 17 | "client" = 1 18 | "metrics" = 1 19 | } 20 | -------------------------------------------------------------------------------- /driver-pravega/deploy/vars.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | --- 16 | pravegaVersion: "0.12.0" 17 | zookeeperVersion: "3.5.5" 18 | bookkeeperVersion: "4.14.2" 19 | prometheusVersion: "2.2.1" 20 | pravegaContainersPerSegmentStore: 4 21 | benchmark_version: "0.0.1-SNAPSHOT" 22 | -------------------------------------------------------------------------------- /driver-pravega/doc/run_in_k8s.md: -------------------------------------------------------------------------------- 1 | # RUN IN KUBERNETES 2 | 3 | ## Build Docker container 4 | 5 | ``` 6 | ./docker-build.sh 7 | ``` 8 | 9 | ## Run local driver on Kubernetes: 10 | 11 | ``` 12 | kubectl run -n examples --rm -it --image pravega/openmessaging-benchmark:latest --serviceaccount examples-pravega openmessaging-benchmark 13 | ``` 14 | 15 | ## Run in Kubernetes 16 | 17 | ``` 18 | ./deploy-k8s-components.sh 19 | ``` 20 | 21 | -------------------------------------------------------------------------------- /driver-pravega/doc/troubleshooting.md: -------------------------------------------------------------------------------- 1 | # TROUBLESHOOTING 2 | 3 | ## Log into one of the hosts and check the logs 4 | 5 | ``` 6 | terraform-inventory -inventory 7 | ssh -i ~/.ssh/pravega_aws ec2-user@`terraform output segmentstore_0_ssh_host` 8 | journalctl -u pravega-segmentstore 9 | ``` 10 | 11 | ## Ansible failed to parse /usr/local/bin/terraform-inventory with script plugin 12 | 13 | ``` 14 | export TF_STATE=./ 15 | ``` 16 | 17 | -------------------------------------------------------------------------------- /driver-pravega/driver-bash.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | # 3 | # Licensed under the Apache License, Version 2.0 (the "License"); 4 | # you may not use this file except in compliance with the License. 5 | # You may obtain a copy of the License at 6 | # 7 | # http://www.apache.org/licenses/LICENSE-2.0 8 | # 9 | # Unless required by applicable law or agreed to in writing, software 10 | # distributed under the License is distributed on an "AS IS" BASIS, 11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | # See the License for the specific language governing permissions and 13 | # limitations under the License. 14 | # 15 | 16 | set -ex 17 | kubectl exec -n examples -it examples-openmessaging-benchmarking-driver bash 18 | -------------------------------------------------------------------------------- /driver-pravega/pravega-exactly-once.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: Pravega 16 | driverClass: io.openmessaging.benchmark.driver.pravega.PravegaBenchmarkDriver 17 | 18 | # Pravega client-specific configuration 19 | client: 20 | controllerURI: tcp://localhost:9090 21 | scopeName: examples 22 | 23 | writer: 24 | enableConnectionPooling: True 25 | 26 | includeTimestampInEvent: True 27 | enableTransaction: True 28 | eventPerTransaction: 1 29 | -------------------------------------------------------------------------------- /driver-pravega/pravega.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: Pravega 16 | driverClass: io.openmessaging.benchmark.driver.pravega.PravegaBenchmarkDriver 17 | 18 | # Pravega client-specific configuration 19 | client: 20 | controllerURI: tcp://localhost:9090 21 | scopeName: examples 22 | 23 | writer: 24 | enableConnectionPooling: True 25 | 26 | includeTimestampInEvent: True 27 | enableStreamAutoScaling: False 28 | eventsPerSecond: 10000 29 | -------------------------------------------------------------------------------- /driver-pravega/src/main/java/io/openmessaging/benchmark/driver/pravega/config/PravegaClientConfig.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark.driver.pravega.config; 15 | 16 | public class PravegaClientConfig { 17 | public String controllerURI; 18 | public String scopeName; 19 | } 20 | -------------------------------------------------------------------------------- /driver-pravega/src/main/java/io/openmessaging/benchmark/driver/pravega/config/PravegaWriterConfig.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark.driver.pravega.config; 15 | 16 | public class PravegaWriterConfig { 17 | public boolean enableConnectionPooling = false; 18 | } 19 | -------------------------------------------------------------------------------- /driver-pravega/uninstall.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | # 3 | # Licensed under the Apache License, Version 2.0 (the "License"); 4 | # you may not use this file except in compliance with the License. 5 | # You may obtain a copy of the License at 6 | # 7 | # http://www.apache.org/licenses/LICENSE-2.0 8 | # 9 | # Unless required by applicable law or agreed to in writing, software 10 | # distributed under the License is distributed on an "AS IS" BASIS, 11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | # See the License for the specific language governing permissions and 13 | # limitations under the License. 14 | # 15 | 16 | set -x 17 | 18 | ROOT_DIR=$(dirname $0)/.. 19 | NAMESPACE=${NAMESPACE:-examples} 20 | 21 | helm del --purge \ 22 | ${NAMESPACE}-openmessaging-benchmarking 23 | 24 | kubectl wait --for=delete --timeout=300s statefulset/${NAMESPACE}-openmessaging-benchmarking-worker -n ${NAMESPACE} 25 | kubectl wait --for=delete --timeout=300s pod/${NAMESPACE}-openmessaging-benchmarking-driver -n ${NAMESPACE} 26 | 27 | true 28 | -------------------------------------------------------------------------------- /driver-pulsar/deploy/ssd/alicloud/ansible.cfg: -------------------------------------------------------------------------------- 1 | [defaults] 2 | host_key_checking=false 3 | private_key_file=benchmark_message_alicloud.pem 4 | 5 | [privilege_escalation] 6 | become=true 7 | become_method='sudo' 8 | become_user='root' 9 | -------------------------------------------------------------------------------- /driver-pulsar/deploy/ssd/alicloud/terraform.tfvars: -------------------------------------------------------------------------------- 1 | region = "cn-shenzhen" 2 | availability_zone = "cn-shenzhen-b" 3 | private_key_file = "benchmark_message_alicloud.pem" 4 | key_name = "key-pair-from-terraform-benchmark-pulsar" 5 | image_id = "centos_7_04_64_20G_alibase_201701015.vhd" 6 | 7 | 8 | 9 | instance_types = { 10 | "pulsar" = "ecs.i1.xlarge" 11 | "zookeeper" = "ecs.se1.xlarge" 12 | "client" = "ecs.se1.xlarge" 13 | "prometheus" = "ecs.se1.xlarge" 14 | } 15 | 16 | num_instances = { 17 | "client" = 4 18 | "pulsar" = 3 19 | "zookeeper" = 3 20 | "prometheus" = 1 21 | } 22 | -------------------------------------------------------------------------------- /driver-pulsar/deploy/ssd/ansible.cfg: -------------------------------------------------------------------------------- 1 | [defaults] 2 | host_key_checking=false 3 | private_key_file=~/.ssh/pulsar_aws 4 | 5 | [privilege_escalation] 6 | become=true 7 | become_method='sudo' 8 | become_user='root' -------------------------------------------------------------------------------- /driver-pulsar/deploy/ssd/restart-workers.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | - name: Benchmarking worker restart 16 | hosts: client 17 | connection: ssh 18 | become: true 19 | tasks: 20 | - name: Benchmark - Start service 21 | systemd: 22 | state: restarted 23 | daemon_reload: yes 24 | name: "benchmark-worker" 25 | -------------------------------------------------------------------------------- /driver-pulsar/deploy/ssd/templates/benchmark-worker.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Benchmark worker 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/benchmark/bin/benchmark-worker 7 | WorkingDirectory=/opt/benchmark 8 | RestartSec=1s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | 13 | [Install] 14 | WantedBy=multi-user.target 15 | -------------------------------------------------------------------------------- /driver-pulsar/deploy/ssd/templates/bookkeeper.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Bookkeeper 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/pulsar/bin/pulsar bookie 7 | WorkingDirectory=/opt/pulsar 8 | RestartSec=1s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | 13 | [Install] 14 | WantedBy=multi-user.target 15 | -------------------------------------------------------------------------------- /driver-pulsar/deploy/ssd/templates/client.conf: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | 16 | # Pulsar Client configuration 17 | webServiceUrl=http://{{ hostvars[groups['pulsar'][0]].private_ip }}:8080/ 18 | 19 | brokerServiceUrl=pulsar://{{ hostvars[groups['pulsar'][0]].private_ip }}:6650/ 20 | -------------------------------------------------------------------------------- /driver-pulsar/deploy/ssd/templates/kop.conf: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | kafkaListeners=PLAINTEXT://{{ hostvars[inventory_hostname].private_ip }}:9092 16 | kafkaAdvertisedListeners=PLAINTEXT://{{ hostvars[inventory_hostname].private_ip }}:9092 17 | allowAutoTopicCreationType=partitioned 18 | brokerEntryMetadataInterceptors=org.apache.pulsar.common.intercept.AppendIndexMetadataInterceptor 19 | brokerDeleteInactiveTopicsEnabled=false 20 | -------------------------------------------------------------------------------- /driver-pulsar/deploy/ssd/templates/mop.conf: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | mqttListeners=mqtt://{{ hostvars[inventory_hostname].private_ip }}:1883 16 | -------------------------------------------------------------------------------- /driver-pulsar/deploy/ssd/templates/myid: -------------------------------------------------------------------------------- 1 | {{ zid }} 2 | -------------------------------------------------------------------------------- /driver-pulsar/deploy/ssd/templates/prometheus.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Prometheus server 3 | After=network.target 4 | 5 | [Service] 6 | WorkingDirectory=/opt/prometheus 7 | ExecStart=/opt/prometheus/prometheus \ 8 | --web.enable-admin-api \ 9 | --storage.tsdb.path=/opt/prometheus/data \ 10 | --config.file=/opt/prometheus/prometheus.yml 11 | 12 | [Install] 13 | WantedBy=multi-user.target 14 | -------------------------------------------------------------------------------- /driver-pulsar/deploy/ssd/templates/pulsar-dashboard.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Pulsar Dashboard 3 | After=prometheus.service 4 | Requires=prometheus.service 5 | 6 | [Service] 7 | WorkingDirectory=/opt/pulsar 8 | ExecStartPre=/usr/bin/docker pull streamnative/apache-pulsar-grafana-dashboard:latest 9 | ExecStart=/usr/bin/docker run --restart=always --name=systemd_pulsar_dashboard -p3000:3000 -e PULSAR_PROMETHEUS_URL=http://{{ hostvars[groups['prometheus'][0]].private_ip }}:9090/ -e PULSAR_CLUSTER=local streamnative/apache-pulsar-grafana-dashboard:latest 10 | ExecStop=/usr/bin/docker stop systemd_pulsar_dashboard 11 | ExecStopPost=/usr/bin/docker rm -f systemd_pulsar_dashboard 12 | ExecReload=/usr/bin/docker restart systemd_pulsar_dashboard 13 | 14 | [Install] 15 | WantedBy=multi-user.target 16 | -------------------------------------------------------------------------------- /driver-pulsar/deploy/ssd/templates/pulsar.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Pulsar Broker 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/pulsar/bin/pulsar broker 7 | WorkingDirectory=/opt/pulsar 8 | RestartSec=1s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | 13 | [Install] 14 | WantedBy=multi-user.target 15 | -------------------------------------------------------------------------------- /driver-pulsar/deploy/ssd/templates/workers.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | 16 | workers: 17 | {% for worker in groups['client'] %} 18 | - http://{{ hostvars[worker].private_ip }}:8080 19 | {% endfor %} 20 | -------------------------------------------------------------------------------- /driver-pulsar/deploy/ssd/templates/zookeeper.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=ZooKeeper Local 3 | After=network.target 4 | 5 | [Service] 6 | Environment=OPTS=-Dstats_server_port=2182 7 | ExecStart=/opt/pulsar/bin/pulsar zookeeper 8 | WorkingDirectory=/opt/pulsar 9 | RestartSec=1s 10 | Restart=on-failure 11 | Type=simple 12 | LimitNOFILE=300000 13 | 14 | [Install] 15 | WantedBy=multi-user.target 16 | -------------------------------------------------------------------------------- /driver-pulsar/deploy/ssd/terraform.tfvars: -------------------------------------------------------------------------------- 1 | public_key_path = "~/.ssh/pulsar_aws.pub" 2 | region = "us-west-2" 3 | az = "us-west-2a" 4 | ami = "ami-08970fb2e5767e3b8" // RHEL-8 5 | 6 | instance_types = { 7 | "pulsar" = "i3en.6xlarge" 8 | "zookeeper" = "i3en.2xlarge" 9 | "client" = "m5n.8xlarge" 10 | "prometheus" = "t2.large" 11 | } 12 | 13 | num_instances = { 14 | "client" = 4 15 | "pulsar" = 5 16 | "zookeeper" = 3 17 | "prometheus" = 1 18 | } 19 | -------------------------------------------------------------------------------- /driver-pulsar/src/main/java/io/openmessaging/benchmark/driver/pulsar/config/PulsarConfig.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package io.openmessaging.benchmark.driver.pulsar.config; 16 | 17 | public class PulsarConfig { 18 | public PulsarClientConfig client = new PulsarClientConfig(); 19 | public PulsarProducerConfig producer = new PulsarProducerConfig(); 20 | public PulsarConsumerConfig consumer = new PulsarConsumerConfig(); 21 | } 22 | -------------------------------------------------------------------------------- /driver-pulsar/src/main/java/io/openmessaging/benchmark/driver/pulsar/config/PulsarConsumerConfig.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark.driver.pulsar.config; 15 | 16 | 17 | import org.apache.pulsar.client.api.SubscriptionType; 18 | 19 | public class PulsarConsumerConfig { 20 | 21 | public int receiverQueueSize = 10000; 22 | public SubscriptionType subscriptionType = SubscriptionType.Failover; 23 | } 24 | -------------------------------------------------------------------------------- /driver-pulsar/src/main/java/io/openmessaging/benchmark/driver/pulsar/config/PulsarProducerConfig.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark.driver.pulsar.config; 15 | 16 | public class PulsarProducerConfig { 17 | public boolean batchingEnabled = true; 18 | public int batchingMaxPublishDelayMs = 1; 19 | public int batchingMaxBytes = 1024 * 1024; 20 | public boolean blockIfQueueFull = true; 21 | public int pendingQueueSize = 0; 22 | } 23 | -------------------------------------------------------------------------------- /driver-rabbitmq/deploy/alicloud/ansible.cfg: -------------------------------------------------------------------------------- 1 | [defaults] 2 | host_key_checking=false 3 | private_key_file=./benchmark_message_alicloud.pem 4 | 5 | [privilege_escalation] 6 | become=true 7 | become_method='sudo' 8 | become_user='root' 9 | -------------------------------------------------------------------------------- /driver-rabbitmq/deploy/alicloud/terraform.tfvars: -------------------------------------------------------------------------------- 1 | region = "cn-shenzhen" 2 | availability_zone = "cn-shenzhen-b" 3 | private_key_file = "benchmark_message_alicloud.pem" 4 | key_name = "key-pair-from-terraform-benchmark-rabbitmq" 5 | image_id = "centos_7_04_64_20G_alibase_201701015.vhd" 6 | 7 | instance_types = { 8 | "rabbitmq" = "ecs.se1.xlarge" 9 | "client" = "ecs.se1.xlarge" 10 | } 11 | 12 | num_instances = { 13 | "rabbitmq" = 3 14 | "client" = 4 15 | } 16 | -------------------------------------------------------------------------------- /driver-rabbitmq/deploy/ansible.cfg: -------------------------------------------------------------------------------- 1 | [defaults] 2 | host_key_checking=false 3 | private_key_file=~/.ssh/rabbitmq_aws 4 | 5 | [privilege_escalation] 6 | become=true 7 | become_method='sudo' 8 | become_user='root' -------------------------------------------------------------------------------- /driver-rabbitmq/deploy/erlang.cookie: -------------------------------------------------------------------------------- 1 | LSKNKBELKPSTDBBCHETL -------------------------------------------------------------------------------- /driver-rabbitmq/deploy/restart-brokers.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | - name: Restart brokers 16 | hosts: rabbitmq 17 | connection: ssh 18 | become: true 19 | tasks: 20 | - name: RabbitMQ - Start service 21 | systemd: 22 | state: restarted 23 | name: "rabbitmq-server" 24 | -------------------------------------------------------------------------------- /driver-rabbitmq/deploy/restart-workers.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | - name: Benchmarking worker restart 16 | hosts: client 17 | connection: ssh 18 | become: true 19 | tasks: 20 | - name: Benchmark - Start service 21 | systemd: 22 | state: restarted 23 | daemon_reload: yes 24 | name: "benchmark-worker" 25 | -------------------------------------------------------------------------------- /driver-rabbitmq/deploy/templates/benchmark-worker.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Benchmark worker 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/benchmark/bin/benchmark-worker 7 | WorkingDirectory=/opt/benchmark 8 | RestartSec=1s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | 13 | [Install] 14 | WantedBy=multi-user.target 15 | -------------------------------------------------------------------------------- /driver-rabbitmq/deploy/templates/grafana-datasource.yml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | apiVersion: 1 16 | 17 | datasources: 18 | {% for prom in groups['prometheus'] %} 19 | - name: {{ hostvars[prom].inventory_hostname }} 20 | url: http://{{ hostvars[prom].private_ip }}:9090 21 | type: prometheus 22 | access: proxy 23 | orgId: 1 24 | isDefault: true 25 | version: 1 26 | editable: false 27 | {% endfor %} 28 | -------------------------------------------------------------------------------- /driver-rabbitmq/deploy/templates/prometheus.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Prometheus server 3 | After=network.target 4 | 5 | [Service] 6 | WorkingDirectory=/opt/prometheus 7 | ExecStart=/opt/prometheus/prometheus \ 8 | --web.enable-admin-api \ 9 | --storage.tsdb.path=/opt/prometheus/data \ 10 | --config.file=/opt/prometheus/prometheus.yml 11 | 12 | [Install] 13 | WantedBy=multi-user.target 14 | -------------------------------------------------------------------------------- /driver-rabbitmq/deploy/templates/rabbitmq-classic.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | 16 | name: RabbitMQ 17 | driverClass: io.openmessaging.benchmark.driver.rabbitmq.RabbitMqBenchmarkDriver 18 | 19 | # RabbitMq client specific configurations 20 | 21 | amqpUris: 22 | {% for pulsar in groups['rabbitmq'] %} 23 | - amqp://admin:admin@{{ hostvars[pulsar].private_ip }}:5672 24 | {% endfor %} 25 | messagePersistence: true 26 | queueType: CLASSIC 27 | -------------------------------------------------------------------------------- /driver-rabbitmq/deploy/templates/rabbitmq-dashboard.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Pulsar Dashboard 3 | After=prometheus.service 4 | Requires=prometheus.service 5 | 6 | [Service] 7 | WorkingDirectory=/opt/grafana 8 | ExecStartPre=/usr/bin/docker pull grafana/grafana:8.3.4 9 | ExecStart=/usr/bin/docker run \ 10 | --restart=always \ 11 | --name=systemd_rabbitmq_dashboard \ 12 | -p3000:3000 \ 13 | -e GF_INSTALL_PLUGINS="flant-statusmap-panel,grafana-piechart-panel" \ 14 | -v /opt/rabbitmq/grafana-datasource.yml:/etc/grafana/provisioning/datasources/prometheus.yaml \ 15 | -v /repos/rabbitmq-server/deps/rabbitmq_prometheus/docker/grafana/dashboards.yml:/etc/grafana/provisioning/dashboards/rabbitmq.yaml \ 16 | -v /repos/rabbitmq-server/deps/rabbitmq_prometheus/docker/grafana/dashboards:/dashboards \ 17 | grafana/grafana:8.3.4 18 | ExecStop=/usr/bin/docker stop systemd_rabbitmq_dashboard 19 | ExecStopPost=/usr/bin/docker rm -f systemd_rabbitmq_dashboard 20 | ExecReload=/usr/bin/docker restart systemd_rabbitmq_dashboard 21 | 22 | [Install] 23 | WantedBy=multi-user.target 24 | -------------------------------------------------------------------------------- /driver-rabbitmq/deploy/templates/rabbitmq-env.conf: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | MNESIA_BASE=/mnt/data 16 | USE_LONGNAME=true 17 | NODENAME=rabbit@{{ ansible_eth0.ipv4.address }} -------------------------------------------------------------------------------- /driver-rabbitmq/deploy/templates/rabbitmq-server.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=RabbitMQ broker 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/usr/lib/rabbitmq/bin/rabbitmq-server 7 | WorkingDirectory=/var/lib/rabbitmq 8 | RestartSec=1s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | 13 | [Install] 14 | WantedBy=multi-user.target -------------------------------------------------------------------------------- /driver-rabbitmq/deploy/templates/rabbitmq.conf: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | ## Pauses all nodes on the minority side of a partition. The cluster 16 | ## MUST have an odd number of nodes (3, 5, etc) 17 | cluster_partition_handling = pause_minority 18 | -------------------------------------------------------------------------------- /driver-rabbitmq/deploy/templates/workers.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | 16 | workers: 17 | {% for worker in groups['client'] %} 18 | - http://{{ hostvars[worker].private_ip }}:8080 19 | {% endfor %} 20 | -------------------------------------------------------------------------------- /driver-rabbitmq/deploy/terraform.tfvars: -------------------------------------------------------------------------------- 1 | public_key_path = "~/.ssh/rabbitmq_aws.pub" 2 | region = "us-west-2" 3 | az = "us-west-2a" 4 | ami = "ami-08970fb2e5767e3b8" // RHEL-8 5 | 6 | instance_types = { 7 | "rabbitmq" = "i3en.6xlarge" 8 | "client" = "m5n.8xlarge" 9 | "prometheus" = "t2.large" 10 | } 11 | 12 | num_instances = { 13 | "rabbitmq" = 3 14 | "client" = 4 15 | "prometheus" = 1 16 | } 17 | -------------------------------------------------------------------------------- /driver-rabbitmq/rabbitmq.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | 16 | name: RabbitMQ 17 | driverClass: io.openmessaging.benchmark.driver.rabbitmq.RabbitMqBenchmarkDriver 18 | 19 | # RabbitMq client specific configurations 20 | 21 | producerCreationDelay: 100 22 | producerCreationBatchSize: 5 23 | consumerCreationDelay: 100 24 | consumerCreationBatchSize: 5 25 | 26 | amqpUris: 27 | - amqp://localhost 28 | messagePersistence: false 29 | queueType: QUORUM 30 | -------------------------------------------------------------------------------- /driver-redis/redis.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: Redis 16 | driverClass: io.openmessaging.benchmark.driver.redis.RedisBenchmarkDriver 17 | 18 | # Redis client-specific configuration 19 | redisPort: 6379 20 | redisHost: 127.0.0.1 21 | jedisPoolMaxTotal: 8 22 | jedisPoolMaxIdle: 8 -------------------------------------------------------------------------------- /driver-redis/src/main/java/io/openmessaging/benchmark/driver/redis/client/RedisClientConfig.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package io.openmessaging.benchmark.driver.redis.client; 15 | 16 | public class RedisClientConfig { 17 | public String redisHost; 18 | public String redisUser; 19 | public String redisPass; 20 | public Integer redisPort; 21 | public Integer jedisPoolMaxTotal; 22 | public Integer jedisPoolMaxIdle; 23 | } 24 | -------------------------------------------------------------------------------- /driver-rocketmq/deploy/ansible.cfg: -------------------------------------------------------------------------------- 1 | [defaults] 2 | host_key_checking=false 3 | private_key_file=rocketmq_alicloud.pem -------------------------------------------------------------------------------- /driver-rocketmq/deploy/templates/benchmark-worker.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=Benchmark worker 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/benchmark/bin/benchmark-worker 7 | WorkingDirectory=/opt/benchmark 8 | RestartSec=1s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | 13 | [Install] 14 | WantedBy=multi-user.target 15 | -------------------------------------------------------------------------------- /driver-rocketmq/deploy/templates/rmq-broker.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | brokerClusterName=DefaultCluster 16 | brokerName=rmq-{{ brokerName }} 17 | brokerId=0 18 | brokerIP1={{ privateIp }} 19 | deleteWhen=04 20 | fileReservedTime=48 21 | brokerRole=ASYNC_MASTER 22 | flushDiskType=ASYNC_FLUSH 23 | namesrvAddr={{ nameServers }} 24 | commitCommitLogLeastPages=1 25 | commitCommitLogThoroughInterval=10 26 | transientStorePoolEnable=true -------------------------------------------------------------------------------- /driver-rocketmq/deploy/templates/rmq-broker.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=RocketMQ-Broker 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/rocketmq/rocketmq-all-4.3.0-bin-release/bin/mqbroker -c conf/rmq-broker.properties 7 | WorkingDirectory=/opt/rocketmq/rocketmq-all-4.3.0-bin-release/ 8 | RestartSec=1s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | 13 | [Install] 14 | WantedBy=multi-user.target 15 | -------------------------------------------------------------------------------- /driver-rocketmq/deploy/templates/rmq-namesrv.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | -------------------------------------------------------------------------------- /driver-rocketmq/deploy/templates/rmq-namesrv.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=RocketMQ-NameServer 3 | After=network.target 4 | 5 | [Service] 6 | ExecStart=/opt/rocketmq/rocketmq-all-4.3.0-bin-release/bin/mqnamesrv -c conf/rmq-namesrv.properties 7 | WorkingDirectory=/opt/rocketmq/rocketmq-all-4.3.0-bin-release/ 8 | RestartSec=1s 9 | Restart=on-failure 10 | Type=simple 11 | LimitNOFILE=300000 12 | 13 | [Install] 14 | WantedBy=multi-user.target 15 | -------------------------------------------------------------------------------- /driver-rocketmq/deploy/templates/workers.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | 16 | workers: 17 | {% for worker in groups['rmq-client'] %} 18 | - http://{{ hostvars[worker].private_ip }}:8080 19 | {% endfor %} 20 | -------------------------------------------------------------------------------- /driver-rocketmq/deploy/terraform.tfvars: -------------------------------------------------------------------------------- 1 | region = "cn-hangzhou" 2 | availability_zone = "cn-hangzhou-b" 3 | private_key_file = "rocketmq_alicloud.pem" 4 | key_name = "key-pair-from-terraform" 5 | image_id = "centos_7_04_64_20G_alibase_201701015.vhd" 6 | 7 | instance_types = { 8 | "broker" = "ecs.mn4.xlarge" #4c16g 9 | "client" = "ecs.mn4.xlarge" 10 | "namesrv" = "ecs.n4.xlarge" #4c8g 11 | } 12 | 13 | num_instances = { 14 | "broker" = 2 15 | "namesrv" = 1 16 | "client" = 4 17 | } 18 | -------------------------------------------------------------------------------- /driver-rocketmq/rocketmq.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: RocketMQ 16 | driverClass: io.openmessaging.benchmark.driver.rocketmq.RocketMQBenchmarkDriver 17 | 18 | # RocketMQ client-specific configuration 19 | clusterName: DefaultCluster 20 | namesrvAddr: 127.0.0.1:9876 21 | vipChannelEnabled: false 22 | 23 | batchCQ: true 24 | autoBatch: true 25 | # batchMaxBytes: 32768 26 | # batchMaxDelayMs: 10 27 | # totalBatchMaxBytes: 33554432 28 | 29 | enableBackpressure: true 30 | backpressureConcurrency: 1024 31 | 32 | accessKey: 33 | secretKey: -------------------------------------------------------------------------------- /etc/APACHE-2.txt: -------------------------------------------------------------------------------- 1 | Licensed under the Apache License, Version 2.0 (the "License"); 2 | you may not use this file except in compliance with the License. 3 | You may obtain a copy of the License at 4 | 5 | http://www.apache.org/licenses/LICENSE-2.0 6 | 7 | Unless required by applicable law or agreed to in writing, software 8 | distributed under the License is distributed on an "AS IS" BASIS, 9 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 10 | See the License for the specific language governing permissions and 11 | limitations under the License. -------------------------------------------------------------------------------- /payload/payload-100b.data: -------------------------------------------------------------------------------- 1 | fbb0cb87a24408039b4f07c89929b5f619a682ecf61a1d97d495e38f876729ddc5a74c04d2fe9382b5eb85379d0f3027fece -------------------------------------------------------------------------------- /payload/payload-1Kb.data: -------------------------------------------------------------------------------- 1 | 6b8d0ca6d616a2e39d674b73c9b11f9a3262a4f03444451f01f27003e84dc3469e61ddcfad9c2bc7c591988add0132b934e5ec0e7b396c2ef3ae8b229090650a422072555dee7685c030eef99cda045c368ce4b119b864b3665056109984ebcf3061e10f56940618f59f7b4b0099d1ea6aae11390a42c46151939ad5f866ae0d2aaedb3cdc4bac917d4419ce1440a6b4cd68b5609baf787fe28eb349bddef61bd9a6d304c57cd20f7ec9eab1cbc8cd6d07fe16e3bf134c87cfd71879ed047f0fd1951223e9b3d006ca0da4c10dc388b3a32a7db3a4516e9d6bcc178b8a0a7fdb21e454ce30f745cc97c7bd921d90db592e76fb349990146fa63bc579e897f6c4902203be20857a0794970d58db4365736f7ab14762c24095b270ecd8b1e42501b2282774e3cebb98d6d46fbcc5817ab65cf0cda02d1679d6ccd23bdec3de734211e657f4104cd34c962fd4b6e45fb34b4141fccfcdb70312b7a3d6f0e4b79077756e7d8d9ded9d586f9039f4f9e55801b2273297dc618a10f72ca3540d4345dec9996fb7d36dabaf9733629d3f47ed3748a6236daacf9c9271f8701dfc0730413bba3b73df235377963d21e685027aafd123fea1b1a7f4e91cfaf16672658e2f7b6ad86d97fd8dc240a94f87bf0ba025b08573e403e4ab0bc1372357c2461a8e5e190c96eb5d1df2f95a414e7cfeeb7f31667b4fe0bea6385d15faadc6cc83a3 -------------------------------------------------------------------------------- /payload/payload-200b.data: -------------------------------------------------------------------------------- 1 | 6b8d0ca6d616a2e39d674b73c9b11f9a3262a4f03444451f01f27003e84dc3469e61ddcfad9c2bc7c591988add0132b934e5ec0e7b396c2ef3ae8b229090650a422072555dee7685c030eef99cda045c368ce4b119b864b3665056109984ebcf3061e10f -------------------------------------------------------------------------------- /payload/payload-400b.data: -------------------------------------------------------------------------------- 1 | 6b8d0ca6d616a2e39d674b73c9b11f9a3262a4f03444451f01f27003e84dc3469e61ddcfad9c2bc7c591988add0132b934e5ec0e7b396c2ef3ae8b229090650a422072555dee7685c030eef99cda045c368ce4b119b864b3665056109984ebcf3061e10f56940618f59f7b4b0099d1ea6aae11390a42c46151939ad5f866ae0d2aaedb3cdc4bac917d4419ce1440a6b4cd68b5609baf787fe28eb349bddef61bd9a6d304c57cd20f7ec9eab1cbc8cd6d07fe16e3bf134c87cfd71879ed047f0fd1951223e9b3d006 -------------------------------------------------------------------------------- /tool/src/main/resources/log4j2.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | Configuration: 16 | status: INFO 17 | name: driver-kafka-test 18 | 19 | Appenders: 20 | Console: 21 | name: Console 22 | target: SYSTEM_OUT 23 | PatternLayout: 24 | Pattern: "%d{HH:mm:ss.SSS} [%t] %-4level %c{1} - %msg%n" 25 | Loggers: 26 | Root: 27 | level: info 28 | additivity: false 29 | AppenderRef: 30 | - ref: Console 31 | -------------------------------------------------------------------------------- /tool/src/test/resources/template.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | nameFormat: "${topics}-topics-${partitionsPerTopic}-partitions-${messageSize}b-${producersPerTopic}p-${consumerPerSubscription}c-${producerRate}" 16 | topics: [1] 17 | partitionsPerTopic: [1] 18 | messageSize: [10000] 19 | payloadFile: "payload/payload-100b.data" 20 | subscriptionsPerTopic: [1] 21 | consumerPerSubscription: [1, 2, 4, 8, 16, 32, 64] 22 | producersPerTopic: [1, 2, 4, 8, 16, 32, 64] 23 | producerRate: [50000] 24 | consumerBacklogSizeGB: 0 25 | testDurationMinutes: 15 -------------------------------------------------------------------------------- /workloads/1-topic-1-partition-100b.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 1 topic / 1 partition / 100b 16 | 17 | topics: 1 18 | partitionsPerTopic: 1 19 | messageSize: 100 20 | payloadFile: "payload/payload-100b.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | producerRate: 50000 25 | consumerBacklogSizeGB: 0 26 | testDurationMinutes: 15 27 | -------------------------------------------------------------------------------- /workloads/1-topic-1-partition-1kb.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 1 topic / 1 partition / 1Kb 16 | 17 | topics: 1 18 | partitionsPerTopic: 1 19 | keyDistributor: "NO_KEY" 20 | messageSize: 1024 21 | payloadFile: "payload/payload-1Kb.data" 22 | subscriptionsPerTopic: 1 23 | consumerPerSubscription: 1 24 | producersPerTopic: 1 25 | producerRate: 50000 26 | consumerBacklogSizeGB: 0 27 | testDurationMinutes: 15 28 | -------------------------------------------------------------------------------- /workloads/1-topic-100-partitions-1kb-4p-4c-1000k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 1000k rate 4 producers and 4 consumers on 1 topic / 100 partition 16 | 17 | topics: 1 18 | partitionsPerTopic: 100 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 1000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/1-topic-100-partitions-1kb-4p-4c-2000k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 2000k rate 4 producers and 4 consumers on 1 topic / 100 partition 16 | 17 | topics: 1 18 | partitionsPerTopic: 100 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 2000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/1-topic-100-partitions-1kb-4p-4c-200k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 200k rate 4 producers and 4 consumers on 1 topic / 100 partition 16 | 17 | topics: 1 18 | partitionsPerTopic: 100 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 200000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/1-topic-100-partitions-1kb-4p-4c-500k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 500k rate 4 producers and 4 consumers on 1 topic / 100 partition 16 | 17 | topics: 1 18 | partitionsPerTopic: 100 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 500000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/1-topic-100-partitions-1kb.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 1 producer / 1 consumers on 1 topic 16 | 17 | topics: 1 18 | partitionsPerTopic: 100 19 | messageSize: 1024 20 | payloadFile: "payload/payload-1Kb.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | producerRate: 50000 25 | consumerBacklogSizeGB: 0 26 | testDurationMinutes: 240 27 | -------------------------------------------------------------------------------- /workloads/1-topic-10000-partitions-1kb-4p-4c-200k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 200k rate 4 producers and 4 consumers on 1 topic / 10000 partition 16 | 17 | topics: 1 18 | partitionsPerTopic: 10000 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 200000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/1-topic-10000-partitions-1kb-4p-4c-500k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 500k rate 4 producers and 4 consumers on 1 topic / 10000 partition 16 | 17 | topics: 1 18 | partitionsPerTopic: 10000 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 500000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/1-topic-16-partition-100b.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 1 topic / 16 partition / 100b 16 | 17 | topics: 1 18 | partitionsPerTopic: 16 19 | messageSize: 100 20 | payloadFile: "payload/payload-100b.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | producerRate: 50000 25 | consumerBacklogSizeGB: 0 26 | testDurationMinutes: 15 27 | -------------------------------------------------------------------------------- /workloads/1-topic-16-partitions-1kb.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 1 producer / 1 consumers on 1 topic 16 | 17 | topics: 1 18 | partitionsPerTopic: 16 19 | messageSize: 1024 20 | payloadFile: "payload/payload-1Kb.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | producerRate: 50000 25 | consumerBacklogSizeGB: 0 26 | testDurationMinutes: 15 27 | -------------------------------------------------------------------------------- /workloads/1-topic-3-partition-100b-3producers.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 1 topic / 3 partition / 100b / 3 producers 16 | 17 | topics: 1 18 | partitionsPerTopic: 3 19 | messageSize: 100 20 | payloadFile: "payload/payload-100b.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 3 24 | producerRate: 0 25 | consumerBacklogSizeGB: 0 26 | testDurationMinutes: 15 27 | -------------------------------------------------------------------------------- /workloads/1-topic-6-partition-100b.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 1 topic / 6 partition / 100b 16 | 17 | topics: 1 18 | partitionsPerTopic: 6 19 | messageSize: 100 20 | payloadFile: "payload/payload-100b.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | producerRate: 50000 25 | consumerBacklogSizeGB: 0 26 | testDurationMinutes: 15 27 | -------------------------------------------------------------------------------- /workloads/100-topic-1kb-4p-4c-2000k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 2000k rate 4 producers and 4 consumers on 100 topic 16 | 17 | topics: 100 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 2000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/100-topic-1kb-4p-4c-500k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 500k rate 4 producers and 4 consumers on 100 topic 16 | 17 | topics: 100 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 500000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/100-topics-1-partitions-1kb.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 1 producer / 3 consumers on 100 topics 16 | 17 | topics: 100 18 | partitionsPerTopic: 1 19 | messageSize: 1024 20 | payloadFile: "payload/payload-1Kb.data" 21 | subscriptionsPerTopic: 3 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | producerRate: 150000 25 | consumerBacklogSizeGB: 0 26 | testDurationMinutes: 240 27 | -------------------------------------------------------------------------------- /workloads/100k-topic-1kb-4p-4c-100k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 100k rate 4 producers and 4 consumers on 100k topic 16 | 17 | topics: 100000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 100000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/100k-topic-1kb-4p-4c-2000k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 2000k rate 4 producers and 4 consumers on 100k topic 16 | 17 | topics: 100000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 2000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/100k-topic-1kb-4p-4c-500k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 500k rate 4 producers and 4 consumers on 100k topic 16 | 17 | topics: 100000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 500000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/10k-topic-1kb-4p-4c-100k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 100k rate 4 producers and 4 consumers on 10k topic 16 | 17 | topics: 10000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 100000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/10k-topic-1kb-4p-4c-2000k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 2000k rate 4 producers and 4 consumers on 10k topic 16 | 17 | topics: 10000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 2000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/10k-topic-1kb-4p-4c-500k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 500k rate 4 producers and 4 consumers on 10k topic 16 | 17 | topics: 10000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 500000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/1k-topic-1kb-4p-4c-100k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 100k rate 4 producers and 4 consumers on 1k topic 16 | 17 | topics: 1000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 100000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/1k-topic-1kb-4p-4c-2000k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 2000k rate 4 producers and 4 consumers on 1k topic 16 | 17 | topics: 1000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 2000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/1k-topic-1kb-4p-4c-500k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 500k rate 4 producers and 4 consumers on 1k topic 16 | 17 | topics: 1000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 500000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/1m-10-topics-1-partition-100b.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 1m-10-topics-1-partition-100b 16 | 17 | topics: 10 18 | partitionsPerTopic: 1 19 | messageSize: 100 20 | #payloadFile: "payload/payload-100b.data" 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 1 27 | producersPerTopic: 1 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 1000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 15 34 | -------------------------------------------------------------------------------- /workloads/1m-10-topics-2-partitions-100b.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 1m-10-topics-2-partitions-100b 16 | 17 | topics: 10 18 | partitionsPerTopic: 2 19 | messageSize: 100 20 | #payloadFile: "payload/payload-100b.data" 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 1 27 | producersPerTopic: 1 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 1000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 15 34 | -------------------------------------------------------------------------------- /workloads/1m-10-topics-3-partitions-100b.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 1m-10-topics-3-partitions-100b 16 | 17 | topics: 10 18 | partitionsPerTopic: 3 19 | messageSize: 100 20 | #payloadFile: "payload/payload-100b.data" 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 1 27 | producersPerTopic: 1 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 1000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 15 34 | -------------------------------------------------------------------------------- /workloads/1m-10-topics-4-partitions-100b.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 1m-10-topics-4-partitions-100b 16 | 17 | topics: 10 18 | partitionsPerTopic: 4 19 | messageSize: 100 20 | #payloadFile: "payload/payload-100b.data" 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 1 27 | producersPerTopic: 1 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 1000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 15 34 | -------------------------------------------------------------------------------- /workloads/1m-topic-1kb-4p-4c-2000k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 2000k rate 4 producers and 4 consumers on 1m topic 16 | 17 | topics: 1000000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 2000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/1m-topic-1kb-4p-4c-500k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 500k rate 4 producers and 4 consumers on 1m topic 16 | 17 | topics: 1000000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 500000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/200k-topic-1kb-4p-4c-100k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 100k rate 4 producers and 4 consumers on 200k topic 16 | 17 | topics: 200000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 100000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/200k-topic-1kb-4p-4c-2000k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 2000k rate 4 producers and 4 consumers on 200k topic 16 | 17 | topics: 200000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 2000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/200k-topic-1kb-4p-4c-500k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 500k rate 4 producers and 4 consumers on 200k topic 16 | 17 | topics: 200000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 500000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/300k-topic-1kb-4p-4c-2000k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 2000k rate 4 producers and 4 consumers on 300k topic 16 | 17 | topics: 300000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 2000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/300k-topic-1kb-4p-4c-500k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 500k rate 4 producers and 4 consumers on 300k topic 16 | 17 | topics: 300000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 500000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/400k-topic-1kb-4p-4c-2000k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 2000k rate 4 producers and 4 consumers on 400k topic 16 | 17 | topics: 400000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 2000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/400k-topic-1kb-4p-4c-500k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 500k rate 4 producers and 4 consumers on 400k topic 16 | 17 | topics: 400000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 500000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/500-topic-1kb-4p-4c-2000k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 2000k rate 4 producers and 4 consumers on 500 topic 16 | 17 | topics: 500 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 2000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/500-topic-1kb-4p-4c-500k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 500k rate 4 producers and 4 consumers on 500 topic 16 | 17 | topics: 500 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 500000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/500k-topic-1kb-4p-4c-2000k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 2000k rate 4 producers and 4 consumers on 500k topic 16 | 17 | topics: 500000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 2000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/500k-topic-1kb-4p-4c-500k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 500k rate 4 producers and 4 consumers on 500k topic 16 | 17 | topics: 500000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 500000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/50k-topic-1kb-4p-4c-100k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 100k rate 4 producers and 4 consumers on 50k topic 16 | 17 | topics: 50000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 100000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/50k-topic-1kb-4p-4c-2000k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 2000k rate 4 producers and 4 consumers on 50k topic 16 | 17 | topics: 50000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 2000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/50k-topic-1kb-4p-4c-500k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 500k rate 4 producers and 4 consumers on 50k topic 16 | 17 | topics: 50000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 500000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/5k-topic-1kb-4p-4c-100k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 100k rate 4 producers and 4 consumers on 5k topic 16 | 17 | topics: 5000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 100000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/5k-topic-1kb-4p-4c-2000k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 2000k rate 4 producers and 4 consumers on 5k topic 16 | 17 | topics: 5000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 2000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/5k-topic-1kb-4p-4c-500k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 500k rate 4 producers and 4 consumers on 5k topic 16 | 17 | topics: 5000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 500000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/600k-topic-1kb-4p-4c-2000k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 2000k rate 4 producers and 4 consumers on 600k topic 16 | 17 | topics: 600000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 2000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/600k-topic-1kb-4p-4c-600k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 500k rate 4 producers and 4 consumers on 600k topic 16 | 17 | topics: 600000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 500000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/700k-topic-1kb-4p-4c-2000k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 2000k rate 4 producers and 4 consumers on 700k topic 16 | 17 | topics: 700000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 2000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/700k-topic-1kb-4p-4c-500k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 500k rate 4 producers and 4 consumers on 700k topic 16 | 17 | topics: 700000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 500000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/800k-topic-1kb-4p-4c-2000k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 2000k rate 4 producers and 4 consumers on 800k topic 16 | 17 | topics: 800000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 2000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/800k-topic-1kb-4p-4c-500k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 500k rate 4 producers and 4 consumers on 800k topic 16 | 17 | topics: 800000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 500000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/900k-topic-1kb-4p-4c-2000k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 2000k rate 4 producers and 4 consumers on 900k topic 16 | 17 | topics: 900000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 2000000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/900k-topic-1kb-4p-4c-500k.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 500k rate 4 producers and 4 consumers on 900k topic 16 | 17 | topics: 900000 18 | partitionsPerTopic: 1 19 | 20 | messageSize: 1024 21 | useRandomizedPayloads: true 22 | randomBytesRatio: 0.5 23 | randomizedPayloadPoolSize: 1000 24 | 25 | subscriptionsPerTopic: 1 26 | consumerPerSubscription: 4 27 | producersPerTopic: 4 28 | 29 | # Discover max-sustainable rate 30 | producerRate: 500000 31 | 32 | consumerBacklogSizeGB: 0 33 | testDurationMinutes: 5 -------------------------------------------------------------------------------- /workloads/backlog-1-topic-1-partition-1kb.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 1 producer / 1 consumers on 1 topic 16 | 17 | topics: 1 18 | partitionsPerTopic: 1 19 | messageSize: 1024 20 | payloadFile: "payload/payload-1Kb.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | producerRate: 100000 25 | consumerBacklogSizeGB: 100 26 | testDurationMinutes: 5 27 | -------------------------------------------------------------------------------- /workloads/backlog-1-topic-16-partitions-1kb.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: 1 producer / 1 consumers on 1 topic 16 | 17 | topics: 1 18 | partitionsPerTopic: 16 19 | messageSize: 1024 20 | payloadFile: "payload/payload-1Kb.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | producerRate: 100000 25 | consumerBacklogSizeGB: 100 26 | testDurationMinutes: 5 27 | -------------------------------------------------------------------------------- /workloads/max-rate-1-topic-100-partitions-100b.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: Max rate 1 producer on 1 topic / 100 partition / 100 bytes 16 | 17 | topics: 1 18 | partitionsPerTopic: 100 19 | messageSize: 100 20 | payloadFile: "payload/payload-100b.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 0 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-1-topic-100-partitions-1kb.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: Max rate 1 producer on 1 topic / 100 partition 16 | 17 | topics: 1 18 | partitionsPerTopic: 100 19 | messageSize: 1024 20 | payloadFile: "payload/payload-1Kb.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 0 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-1-topic-100-partitions-1p-1c-100b.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: max-rate-1-topic-100-partitions-1p-1c-100b 16 | 17 | topics: 1 18 | partitionsPerTopic: 100 19 | messageSize: 100 20 | payloadFile: "payload/payload-100b.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 10000000 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-1-topic-100-partitions-1p-1c-1kb.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: max-rate-1-topic-100-partitions-1p-1c-1kb 16 | 17 | topics: 1 18 | partitionsPerTopic: 100 19 | messageSize: 1024 20 | payloadFile: "payload/payload-1Kb.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 10000000 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-1-topic-16-partitions-100b.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: Max rate 1 producer on 1 topic / 16 partitions 16 | 17 | topics: 1 18 | partitionsPerTopic: 16 19 | messageSize: 100 20 | payloadFile: "payload/payload-100b.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 0 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-1-topic-16-partitions-1kb.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: Max rate 1 producer on 1 topic / 1 partition 16 | 17 | topics: 1 18 | partitionsPerTopic: 16 19 | messageSize: 1024 20 | payloadFile: "payload/payload-1Kb.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 0 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-1-topic-30-partitions-30p-30c-100b.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: max-rate-1-topic-30-partitions-30p-30c-100b 16 | 17 | topics: 1 18 | partitionsPerTopic: 30 19 | messageSize: 100 20 | payloadFile: "payload/payload-100b.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 30 23 | producersPerTopic: 30 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 10000000 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-1-topic-30-partitions-30p-30c-1kb.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: max-rate-1-topic-30-partitions-30p-30c-1kb 16 | 17 | topics: 1 18 | partitionsPerTopic: 30 19 | messageSize: 1024 20 | payloadFile: "payload/payload-1Kb.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 30 23 | producersPerTopic: 30 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 10000000 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-1-topic-50-partitions-50p-50c-100b.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: max-rate-1-topic-50-partitions-50p-50c-100b 16 | 17 | topics: 1 18 | partitionsPerTopic: 50 19 | messageSize: 100 20 | payloadFile: "payload/payload-100b.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 50 23 | producersPerTopic: 50 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 10000000 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-1-topic-50-partitions-50p-50c-1kb.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: max-rate-1-topic-50-partitions-50p-50c-1kb 16 | 17 | topics: 1 18 | partitionsPerTopic: 50 19 | messageSize: 1024 20 | payloadFile: "payload/payload-1Kb.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 50 23 | producersPerTopic: 50 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 10000000 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-1-topic-70-partitions-70p-70c-100b.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: max-rate-1-topic-70-partitions-70p-70c-100b 16 | 17 | topics: 1 18 | partitionsPerTopic: 70 19 | messageSize: 100 20 | payloadFile: "payload/payload-100b.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 70 23 | producersPerTopic: 70 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 10000000 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-1-topic-70-partitions-70p-70c-1kb.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: max-rate-1-topic-70-partitions-70p-70c-1kb 16 | 17 | topics: 1 18 | partitionsPerTopic: 70 19 | messageSize: 1024 20 | payloadFile: "payload/payload-1Kb.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 70 23 | producersPerTopic: 70 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 10000000 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-1-topic-90-partitions-90p-90c-100b.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: max-rate-1-topic-90-partitions-90p-90c-100b 16 | 17 | topics: 1 18 | partitionsPerTopic: 90 19 | messageSize: 100 20 | payloadFile: "payload/payload-100b.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 90 23 | producersPerTopic: 90 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 10000000 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-1-topic-90-partitions-90p-90c-1kb.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: max-rate-1-topic-90-partitions-90p-90c-1kb 16 | 17 | topics: 1 18 | partitionsPerTopic: 90 19 | messageSize: 1024 20 | payloadFile: "payload/payload-1Kb.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 90 23 | producersPerTopic: 90 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 10000000 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-30-topics-1-partition-100b.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: max-rate-30-topics-1-partition-100b 16 | 17 | topics: 30 18 | partitionsPerTopic: 1 19 | messageSize: 100 20 | payloadFile: "payload/payload-100b.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 10000000 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-30-topics-1-partition-1kb.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: max-rate-30-topics-1-partition-1kb 16 | 17 | topics: 30 18 | partitionsPerTopic: 1 19 | messageSize: 1024 20 | payloadFile: "payload/payload-1Kb.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 10000000 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-50-topics-1-partition-100b.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: max-rate-50-topics-1-partition-100b 16 | 17 | topics: 50 18 | partitionsPerTopic: 1 19 | messageSize: 100 20 | payloadFile: "payload/payload-100b.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 10000000 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-50-topics-1-partition-1kb.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: max-rate-50-topics-1-partition-1kb 16 | 17 | topics: 50 18 | partitionsPerTopic: 1 19 | messageSize: 1024 20 | payloadFile: "payload/payload-1Kb.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 10000000 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-70-topics-1-partition-100b.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: max-rate-70-topics-1-partition-100b 16 | 17 | topics: 70 18 | partitionsPerTopic: 1 19 | messageSize: 100 20 | payloadFile: "payload/payload-100b.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 10000000 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-70-topics-1-partition-1kb.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: max-rate-70-topics-1-partition-1kb 16 | 17 | topics: 70 18 | partitionsPerTopic: 1 19 | messageSize: 1024 20 | payloadFile: "payload/payload-1Kb.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 10000000 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-90-topics-1-partition-100b.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: max-rate-90-topics-1-partition-100b 16 | 17 | topics: 90 18 | partitionsPerTopic: 1 19 | messageSize: 100 20 | payloadFile: "payload/payload-100b.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 10000000 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/max-rate-90-topics-1-partition-1kb.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | name: max-rate-90-topics-1-partition-1kb 16 | 17 | topics: 90 18 | partitionsPerTopic: 1 19 | messageSize: 1024 20 | payloadFile: "payload/payload-1Kb.data" 21 | subscriptionsPerTopic: 1 22 | consumerPerSubscription: 1 23 | producersPerTopic: 1 24 | 25 | # Discover max-sustainable rate 26 | producerRate: 10000000 27 | 28 | consumerBacklogSizeGB: 0 29 | testDurationMinutes: 5 30 | -------------------------------------------------------------------------------- /workloads/simple-workload.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | # 14 | 15 | 16 | name: Simple Workload 1 producer on 1 topic 17 | 18 | topics: 1 19 | partitionsPerTopic: 10 20 | messageSize: 1024 21 | payloadFile: "payload/payload-1Kb.data" 22 | subscriptionsPerTopic: 1 23 | producersPerTopic: 1 24 | consumerPerSubscription: 1 25 | producerRate: 10000 26 | consumerBacklogSizeGB: 0 27 | testDurationMinutes: 5 28 | 29 | 30 | --------------------------------------------------------------------------------