Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-17645: KIP-1052: Enable warmup in producer performance test #17340

Open
wants to merge 17 commits into
base: trunk
Choose a base branch
from

Conversation

matt-welch
Copy link

In order to better analyze steady-state performance of Kafka, this PR enables a warmup in the Producer Performance test. The warmup duration is specified as a number of records that are a subset of the total numRecords. If warmup records is greater than 0, the warmup is represented by a second Stats object which holds warmup results. Once warmup records have been exhausted, the test switches to using the existing Stats object. At end of test, if warmup was enabled, the summary of the whole test (warump + steady state) is printed followed by the summary of the steady-state portion of the test. If no warmup is used, summary prints don't change from existing behavior.
This contribution is an original work and is licensed to the Kafka project under the Apache license

Testing strategy comprises new Java unit tests added to ProducerPerformanceTests.java.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

This commit should enable a test PR against the trunk
Now using an overallStats object with new constructor to contin the
combined stats at end of test and a single printTotal method.
Refactor to remove declaration of overallStats and limit its scope to
only the location where printTotal is called.
Set Stats combine-constructor as package-private.
Remove commented code.
@github-actions github-actions bot added the tools label Oct 1, 2024
Copy link
Contributor

@fvaleri fvaleri left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @matt-welch, thanks for working on this.

I didn't have time to look further into this, but new results don't make much sense to me:

$ bin/kafka-producer-perf-test.sh --topic my-topic --record-size 1024 --num-records 1000000 --throughput 10000 --warmup-records 60000 --producer-props bootstrap.servers=:9092 linger.ms=0 batch.size=16384
...
1000000 records sent, 9998.000400 records/sec (0.00 MB/sec), 44181.96 ms avg latency, 94001.00 ms max latency, 266 ms 50th, 266 ms 95th, 266 ms 99th, 266 ms 99.9th.
940000 steady state records sent, 9997.553790 records/sec (9.76 MB/sec), 47001.95 ms avg latency, 94001.00 ms max latency, 47002 ms 50th, 89301 ms 95th, 93062 ms 99th, 93908 ms 99.9th.

As you can see, we have 0.00 MB/sec in total results, and very high latencies in steady-state result. This is the output without your changes:

1000000 records sent, 9998.0 records/sec (9.76 MB/sec), 0.76 ms avg latency, 255.00 ms max latency, 1 ms 50th, 1 ms 95th, 2 ms 99th, 38 ms 99.9th.

Another problem I found is the following:

$ bin/kafka-producer-perf-test.sh --topic my-topic --payload-file LICENSE --num-records -10 --throughput 50000 --producer-props bootstrap.servers=:9092 linger.ms=0 batch.size=16384
usage: producer-performance [-h] --topic TOPIC --num-records NUM-RECORDS [--payload-delimiter PAYLOAD-DELIMITER] --throughput THROUGHPUT [--producer-props PROP-NAME=PROP-VALUE [PROP-NAME=PROP-VALUE ...]] [--producer.config CONFIG-FILE] [--print-metrics] [--transactional-id TRANSACTIONAL-ID]
                            [--transaction-duration-ms TRANSACTION-DURATION] [--warmup-records WARMUP-RECORDS] (--record-size RECORD-SIZE | --payload-file PAYLOAD-FILE | --payload-monotonic)
producer-performance: error: The value for --warmup-records must be strictly fewer than the number of records in the test, --num-records.

The check is fine, but it should be skipped if the warmup flag is not used. I also think we should add a non-negative check for numRecords in ConfigPostProcessor. I opened another PR with similar improvements.

Comment on lines 79 to 80
// TODO: Keep this message? Maybe unnecessary
System.out.println("Warmup first " + config.warmupRecords + " records. Steady-state results will print after the complete-test summary.");
Copy link
Contributor

@fvaleri fvaleri Oct 2, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Using the tool I changed my mind about this. It's useful to know that the tool is running the warmup, but we would also need a similar log for the steady state phase, otherwise you wouldn't be able to distinguish if the periodic log is about warmup or steady state.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Are you suggesting we have an additional print message that happens during the switchover to steady state?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Exactly. I think it could be useful.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I like this to explicitly inform the user that they're in steady state.
The wording is up for debate, but I've added a basic message "Beginning steady state" to the record() method to indicate to the user that following prints are steady state. In testing, I've observed that you might not see window prints for different number of records, depending on how you set your warmup records, relative the size of the print window. This is similar to behavior in the existing test where the final window gets included in the final summary print.

- Fix callbacks after switch to steady state which were not being
  properly created.
- Add constant for DEFAULT_REPORTING_INTERVAL_MS
- Remove redundant producer.send calls
- Add ternerary operator in state assignment
- Add check for non-positive numRecords
@matt-welch
Copy link
Author

I was able to reproduce and fix the performance issue. I had failed to create the record callback for messages in steady state :( It should be fixed in the latest push, but I still think the logic in that conditional can be improved.

re: checking warmup records before num records, I've added an exception for when numRecords <= 0.

FYI, I will be out for the next week, but thanks so much for looking at this patch!!

@fvaleri
Copy link
Contributor

fvaleri commented Oct 4, 2024

Thanks for the latest changes. Results look much better now:

2000000 records sent, 9998.350272 records/sec (0.00 MB/sec), 0.58 ms avg latency, 417.00 ms max latency, 417 ms 50th, 417 ms 95th, 417 ms 99th, 417 ms 99.9th.
1500000 steady state records sent, 9997.733847 records/sec (0.95 MB/sec), 0.52 ms avg latency, 9.00 ms max latency, 1 ms 50th, 1 ms 95th, 1 ms 99th, 2 ms 99.9th.

There is still an issue with the overall throughput (0.00 MB/sec) though. I also see some checkstyle errors after rebasing.

FYI, I will be out for the next week, but thanks so much for looking at this patch!

No rush, take your time.

- bytes are now properly accoutned for.
- Refactor to use boolean isSteadyState over warmupRecords to track
  if a Stats object is of steady state
@fvaleri
Copy link
Contributor

fvaleri commented Oct 21, 2024

Hi @matt-welch, there is one checkstyle error and you also probably need to rebase.

Copy link
Member

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@matt-welch thanks for this implementation!

@@ -75,7 +76,13 @@ void start(String[] args) throws IOException {
// not thread-safe, do not share with other threads
SplittableRandom random = new SplittableRandom(0);
ProducerRecord<byte[], byte[]> record;
stats = new Stats(config.numRecords, 5000);
if (config.warmupRecords > 0) {
// TODO: Keep this message? Maybe unnecessary
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, please the unnecessary log

@@ -114,6 +133,10 @@ record = new ProducerRecord<>(config.topicName, payload);
if (!config.shouldPrintMetrics) {
producer.close();

/* print warmup stats if relevant */
if (warmupStats != null) {
new Stats(warmupStats, stats).printTotal();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe we should have a state for 'all' and another state for 'steady-only'. With this approach, we can avoid this kind of magic merging.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the review @chia7712!
Just to be clear, are you suggesting that, instead of using a warmup stats followed by a steady state stats object, we should use a whole-test stats object in addition to a steady-state stats object?
I don't see any serious issues in the implementation, but I the producer callback will need refactoring so it writes into two stats objects once steady state has triggered. In the initial design, I was trying to avoid this concurrent need for both stats objects and do the array-merging at the end, but I will give it a try to see how it looks.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@matt-welch did you tried the suggested approach?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry for the delay. I've refactored the stats objects as suggested in my latest commit.

@fvaleri
Copy link
Contributor

fvaleri commented Nov 4, 2024

@matt-welch sorry for the delay. Latest changes LGTM. Please, address the comments from @chia7712 and then I think we are good. Thanks.

PS: Feel free to ping people for review when you do changes, as we don't get notifications for new commits.

Stats objects now represent the whole test with
'stats' and the steady state portion of the test
with 'steadyStateStats'. Remove merging of
Stats latency arrays at end of test.
@matt-welch
Copy link
Author

Hi @chia7712 @fvaleri . My latest commit refactors the use of a warmupStats object and a main stats object in favor of having one stats that covers the whole test and a second object, steadyStateStats, that will be used during steady state operation.
In the case of a steady state record, I've added a new callback that will write into both data objects during steady state.
This new approach seems like a simpler, cleaner design and is much easier to understand what is happening. I've done some testing on both designs and they appear to perform similarly. My only reservation is that the producer will now have a larger memory requirement due to the redundant recording of latency data into the two objects. Let me know what you think.

Copy link
Contributor

@fvaleri fvaleri left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM. Nice feature. Thanks @matt-welch.

@chia7712 call for review.

@chia7712
Copy link
Member

@matt-welch thanks for your update. I will take a look later!

Copy link
Member

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@matt-welch thanks for this patch. some minor comments are left. Additionally, could you please run this patch and share the result to me? thanks!

this.isSteadyState = isSteadyState;
}

Stats(Stats first, Stats second) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you please remove this unused constructor?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

cb = new PerfCallback(sendStartMs, payload.length, stats);
} else {
if (i == config.warmupRecords) {
steadyStateStats = new Stats(config.numRecords - config.warmupRecords, DEFAULT_REPORTING_INTERVAL_MS, config.warmupRecords > 0);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

config.warmupRecords > 0 can be replaced by true

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you please clarify what you mean? That conditional is required when warmup is enabled.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Apologies for the unclear comment. Please see the following code.

if (config.warmupRecords > 0) {
                    if (i < config.warmupRecords) {
                        cb = new PerfCallback(sendStartMs, payload.length, stats);
                    } else {
                        if (i == config.warmupRecords) {
                            steadyStateStats = new Stats(config.numRecords - config.warmupRecords, DEFAULT_REPORTING_INTERVAL_MS, config.warmupRecords > 0);
                        }
                        cb = new PerfCallback(sendStartMs, payload.length, stats, steadyStateStats);
                    }
                }

In this case, config.warmupRecords > 0 is always true.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Apologies if I'm still misunderstanding your query, but the conditional "config.warmupRecords > 0" will only evaluate to true when the argument '--warmup-records N' is invoked on the command line and N is greater than zero. The value for config.warmupRecords defaults to '0' otherwise. The if-clause on that line is the path used when a warmup is invoked. The else-clause will be used when no warmup has been requested. This else clause represents the same code path used in previous versions of the test with no warmup.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Apologies for the unclear description. I have attached a screenshot to illustrate the warning from the IDE.

Screenshot From 2024-12-18 19-08-16

we can make IDE happy by a little change :)

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @chia7712, to address your concern, I have temporarily added a DEBUG message that prints the value of config.warmupRecords and the value of the conditional (config.warmupRecords > 0) before the test. Some sample output of running the test with various values of warmupRecords follows. Note that, when negative values of warmupRecords are supplied on the CLI, we set config.warmupRecords to 0.

# warmupRecords = -1
./kafka-producer-perf-test.sh --producer.config ../config/producer.properties --topic warmup--1-records-test --num-records 500000 --record-size 1024 --producer-props bootstrap.servers=127.0.0.1:9092 acks=all compression.type=none batch.size=16384 linger.ms=0 --throughput 10000 --warmup-records -1
DEBUG: config.warmupRecords=0, (config.warmupRecords > 0)=false
49987 records sent, 9997.4 records/sec (9.76 MB/sec), 18.8 ms avg latency, 290.0 ms max latency.
50005 records sent, 10001.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 9.0 ms max latency.
50019 records sent, 10001.8 records/sec (9.77 MB/sec), 0.7 ms avg latency, 9.0 ms max latency.
50005 records sent, 10001.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 2.0 ms max latency.
50006 records sent, 9999.2 records/sec (9.76 MB/sec), 0.7 ms avg latency, 3.0 ms max latency.
49930 records sent, 9960.1 records/sec (9.73 MB/sec), 0.7 ms avg latency, 22.0 ms max latency.
50229 records sent, 10045.8 records/sec (9.81 MB/sec), 0.7 ms avg latency, 23.0 ms max latency.
50015 records sent, 10003.0 records/sec (9.77 MB/sec), 0.6 ms avg latency, 2.0 ms max latency.
49995 records sent, 9997.0 records/sec (9.76 MB/sec), 0.7 ms avg latency, 2.0 ms max latency.
500000 records sent, 9996.801024 records/sec (9.76 MB/sec), 2.49 ms avg latency, 290.00 ms max latency, 1 ms 50th, 1 ms 95th, 82 ms 99th, 219 ms 99.9th.

# warmupRecords = 0
./kafka-producer-perf-test.sh --producer.config ../config/producer.properties --topic warmup-0-records-test --num-records 500000 --record-size 1024 --producer-props bootstrap.servers=127.0.0.1:9092 acks=all compression.type=none batch.size=16384 linger.ms=0 --throughput 10000 --warmup-records 0
DEBUG: config.warmupRecords=0, (config.warmupRecords > 0)=false
49982 records sent, 9996.4 records/sec (9.76 MB/sec), 2.3 ms avg latency, 177.0 ms max latency.
50010 records sent, 10002.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 4.0 ms max latency.
50010 records sent, 10000.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 2.0 ms max latency.
50010 records sent, 10000.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 2.0 ms max latency.
50000 records sent, 10000.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 2.0 ms max latency.
50000 records sent, 10000.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 12.0 ms max latency.
50000 records sent, 10000.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 3.0 ms max latency.
50020 records sent, 10002.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 2.0 ms max latency.
50008 records sent, 10001.6 records/sec (9.77 MB/sec), 0.7 ms avg latency, 2.0 ms max latency.
500000 records sent, 9997.200784 records/sec (9.76 MB/sec), 0.83 ms avg latency, 177.00 ms max latency, 1 ms 50th, 1 ms 95th, 2 ms 99th, 44 ms 99.9th.

# warmupRecords = 1
./kafka-producer-perf-test.sh --producer.config ../config/producer.properties --topic warmup-1-records-test --num-records 500000 --record-size 1024 --producer-props bootstrap.servers=127.0.0.1:9092 acks=all compression.type=none batch.size=16384 linger.ms=0 --throughput 10000 --warmup-records 1
DEBUG: config.warmupRecords=1, (config.warmupRecords > 0)=true
Warmup first 1 records. Steady state results will print after the complete test summary.
Beginning steady state.
51651 records sent, 10328.1 records/sec (10.09 MB/sec), 2.7 ms avg latency, 63.0 ms max latency.
50015 records sent, 10003.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 3.0 ms max latency.
49995 records sent, 9999.0 records/sec (9.76 MB/sec), 0.7 ms avg latency, 2.0 ms max latency.
50013 records sent, 10002.6 records/sec (9.77 MB/sec), 0.7 ms avg latency, 3.0 ms max latency.
50016 records sent, 10001.2 records/sec (9.77 MB/sec), 0.7 ms avg latency, 2.0 ms max latency.
50011 records sent, 10000.2 records/sec (9.77 MB/sec), 0.7 ms avg latency, 19.0 ms max latency.
50000 records sent, 10000.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 2.0 ms max latency.
50010 records sent, 10002.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 2.0 ms max latency.
50010 records sent, 10000.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 2.0 ms max latency.
500000 records sent, 9997.800484 records/sec (9.76 MB/sec), 0.87 ms avg latency, 177.00 ms max latency, 1 ms 50th, 1 ms 95th, 2 ms 99th, 54 ms 99.9th.
499999 steady state records sent, 10030.070211 records/sec (9.79 MB/sec), 0.87 ms avg latency, 63.00 ms max latency, 1 ms 50th, 1 ms 95th, 2 ms 99th, 54 ms 99.9th.

# warmupRecords = 10000
./kafka-producer-perf-test.sh --producer.config ../config/producer.properties --topic warmup-10000-records-test --num-records 500000 --record-size 1024 --producer-props bootstrap.servers=127.0.0.1:9092 acks=all compression.type=none batch.size=16384 linger.ms=0 --throughput 10000 --warmup-records 10000
DEBUG: config.warmupRecords=10000, (config.warmupRecords > 0)=true
Warmup first 10000 records. Steady state results will print after the complete test summary.
Beginning steady state.
49982 records sent, 9996.4 records/sec (9.76 MB/sec), 0.7 ms avg latency, 4.0 ms max latency.
50010 records sent, 10002.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 3.0 ms max latency.
50010 records sent, 10000.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 2.0 ms max latency.
50000 records sent, 10000.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 2.0 ms max latency.
50020 records sent, 10002.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 7.0 ms max latency.
50000 records sent, 10000.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 4.0 ms max latency.
50009 records sent, 10001.8 records/sec (9.77 MB/sec), 0.7 ms avg latency, 2.0 ms max latency.
50001 records sent, 9998.2 records/sec (9.76 MB/sec), 0.6 ms avg latency, 2.0 ms max latency.
50010 records sent, 10000.0 records/sec (9.77 MB/sec), 0.6 ms avg latency, 2.0 ms max latency.
500000 records sent, 9996.201443 records/sec (9.76 MB/sec), 0.83 ms avg latency, 178.00 ms max latency, 1 ms 50th, 1 ms 95th, 2 ms 99th, 46 ms 99.9th.
490000 steady state records sent, 9995.308325 records/sec (9.76 MB/sec), 0.66 ms avg latency, 7.00 ms max latency, 1 ms 50th, 1 ms 95th, 1 ms 99th, 2 ms 99.9th.

# warmupRecords = 50000
./kafka-producer-perf-test.sh --producer.config ../config/producer.properties --topic warmup-50000-records-test --num-records 500000 --record-size 1024 --producer-props bootstrap.servers=127.0.0.1:9092 acks=all compression.type=none batch.size=16384 linger.ms=0 --throughput 10000 --warmup-records 50000
DEBUG: config.warmupRecords=50000, (config.warmupRecords > 0)=true
Warmup first 50000 records. Steady state results will print after the complete test summary.
Beginning steady state.
49991 records sent, 9998.2 records/sec (9.76 MB/sec), 0.6 ms avg latency, 3.0 ms max latency.
50011 records sent, 10002.2 records/sec (9.77 MB/sec), 0.7 ms avg latency, 2.0 ms max latency.
50010 records sent, 10000.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 25.0 ms max latency.
50010 records sent, 10002.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 12.0 ms max latency.
50000 records sent, 10000.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 4.0 ms max latency.
50020 records sent, 10002.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 2.0 ms max latency.
50000 records sent, 10000.0 records/sec (9.77 MB/sec), 0.6 ms avg latency, 2.0 ms max latency.
50020 records sent, 10002.0 records/sec (9.77 MB/sec), 0.6 ms avg latency, 2.0 ms max latency.
500000 records sent, 9997.600576 records/sec (9.76 MB/sec), 0.81 ms avg latency, 172.00 ms max latency, 1 ms 50th, 1 ms 95th, 2 ms 99th, 40 ms 99.9th.
450000 steady state records sent, 9996.001599 records/sec (9.76 MB/sec), 0.66 ms avg latency, 25.00 ms max latency, 1 ms 50th, 1 ms 95th, 1 ms 99th, 4 ms 99.9th.

# warmupRecords = 100000
./kafka-producer-perf-test.sh --producer.config ../config/producer.properties --topic warmup-100000-records-test --num-records 500000 --record-size 1024 --producer-props bootstrap.servers=127.0.0.1:9092 acks=all compression.type=none batch.size=16384 linger.ms=0 --throughput 10000 --warmup-records 100000
DEBUG: config.warmupRecords=100000, (config.warmupRecords > 0)=true
Warmup first 100000 records. Steady state results will print after the complete test summary.
49994 records sent, 9996.8 records/sec (9.76 MB/sec), 2.2 ms avg latency, 184.0 ms max latency.
Beginning steady state.
49991 records sent, 9994.2 records/sec (9.76 MB/sec), 0.6 ms avg latency, 2.0 ms max latency.
50006 records sent, 10001.2 records/sec (9.77 MB/sec), 0.9 ms avg latency, 31.0 ms max latency.
49995 records sent, 9999.0 records/sec (9.76 MB/sec), 0.6 ms avg latency, 8.0 ms max latency.
50020 records sent, 10002.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 4.0 ms max latency.
50000 records sent, 10000.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 2.0 ms max latency.
50010 records sent, 10002.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 2.0 ms max latency.
50010 records sent, 10000.0 records/sec (9.77 MB/sec), 0.7 ms avg latency, 7.0 ms max latency.
500000 records sent, 9997.200784 records/sec (9.76 MB/sec), 0.84 ms avg latency, 184.00 ms max latency, 1 ms 50th, 1 ms 95th, 2 ms 99th, 42 ms 99.9th.
400000 steady state records sent, 9995.002499 records/sec (9.76 MB/sec), 0.69 ms avg latency, 31.00 ms max latency, 1 ms 50th, 1 ms 95th, 2 ms 99th, 17 ms 99.9th.

@@ -506,6 +584,9 @@ public ConfigPostProcessor(ArgumentParser parser, String[] args) throws IOExcept
if (numRecords != null && numRecords <= 0) {
throw new ArgumentParserException("--num-records should be greater than zero", parser);
}
if (warmupRecords != null && warmupRecords >= numRecords) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

warmupRecords != null is unnecessary. Also, could you please change the type of numRecords and warmupRecords from Long to long?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

sdp and others added 2 commits January 16, 2025 14:28
- Add boolean to stats objects to control if they should print or not
- Add DEBUG print to show value of config.warmupRecords
- Use "long" for numRecords and warmupRecords instead of "Long"
  - Remove "* != null" for above
- Remove unused "combiner" constructor
@matt-welch
Copy link
Author

Hi @chia7712 and @fvaleri. I've made some updates to this PR in my latest commits including some sample output for review. Please let me know if there's anything else I can or should do to complete the PR or answer any concerns about it.
Thanks!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants