-
Notifications
You must be signed in to change notification settings - Fork 0
KAFKA-17645: KIP-1052: Enable warmup in producer performance test #19
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
base: trunk
Are you sure you want to change the base?
Changes from 21 commits
4cfb897
125a246
6dcaed8
46fcad9
df6a119
6dfbaa7
74f2f1c
baef288
50a7565
4dd891f
ce9ce6b
9be5385
fb8a4ca
440329b
a57554b
818d1c6
b0a6c20
b127087
a55a429
4ff4daa
7def5f4
2026e14
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change | ||||||||||||||||||||||||||||||||||||
---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
@@ -75,7 +75,11 @@ 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) { | ||||||||||||||||||||||||||||||||||||||
System.out.println("Warmup first " + config.warmupRecords + " records. Steady state results will print after the complete test summary."); | ||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
stats = new Stats(config.numRecords); | ||||||||||||||||||||||||||||||||||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Stats Constructor MismatchThe main Stats object is initialized with just numRecords, using the new constructor that defaults isSteadyState to false. However, the steadyStateActive field is later set to true on this object, creating an inconsistent state where the object is not initialized as steady state but is marked as active. Standards
|
||||||||||||||||||||||||||||||||||||||
long startMs = System.currentTimeMillis(); | ||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||
ThroughputThrottler throttler = new ThroughputThrottler(config.throughput, startMs); | ||||||||||||||||||||||||||||||||||||||
|
@@ -94,7 +98,11 @@ void start(String[] args) throws IOException { | |||||||||||||||||||||||||||||||||||||
record = new ProducerRecord<>(config.topicName, payload); | ||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||
long sendStartMs = System.currentTimeMillis(); | ||||||||||||||||||||||||||||||||||||||
cb = new PerfCallback(sendStartMs, payload.length, stats); | ||||||||||||||||||||||||||||||||||||||
if ( config.warmupRecords > 0 && i == config.warmupRecords ) { | ||||||||||||||||||||||||||||||||||||||
steadyStateStats = new Stats(config.numRecords - config.warmupRecords, config.warmupRecords > 0); | ||||||||||||||||||||||||||||||||||||||
stats.steadyStateActive = true; | ||||||||||||||||||||||||||||||||||||||
Comment on lines
+102
to
+103
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Redundant Stats InitializationThe code initializes steadyStateStats with a boolean parameter derived from config.warmupRecords > 0, which is always true at this point. This redundant check creates confusion about the initialization state and could lead to inconsistent behavior. Standards
|
||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
cb = new PerfCallback(sendStartMs, payload.length, stats, steadyStateStats); | ||||||||||||||||||||||||||||||||||||||
Comment on lines
+101
to
+105
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Potential NullPointerException RiskThe steadyStateStats variable is passed to PerfCallback constructor even when it's null (when warmupRecords is 0 or i hasn't reached warmupRecords yet). This will cause NullPointerException in PerfCallback.onCompletion() when it tries to access steadyStateStats methods.
Commitable Suggestion
Suggested change
Standards
Comment on lines
+101
to
+105
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Inconsistent null handling for steadyStateStatsThe steadyStateStats variable is passed to PerfCallback constructor before it's initialized when warmupRecords is 0 or before reaching the warmup threshold. This creates a risk of NullPointerException when the callback tries to access steadyStateStats methods. Standards
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Callback NPE RiskThe steadyStateStats variable is passed to PerfCallback constructor even when null (when warmupRecords is 0 or i hasn't reached warmupRecords). This creates a null pointer exception risk in PerfCallback.onCompletion() when accessing steadyStateStats methods. Standards
|
||||||||||||||||||||||||||||||||||||||
producer.send(record, cb); | ||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||
currentTransactionSize++; | ||||||||||||||||||||||||||||||||||||||
|
@@ -116,6 +124,10 @@ record = new ProducerRecord<>(config.topicName, payload); | |||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||
/* print final results */ | ||||||||||||||||||||||||||||||||||||||
stats.printTotal(); | ||||||||||||||||||||||||||||||||||||||
/* print steady-state stats if relevant */ | ||||||||||||||||||||||||||||||||||||||
if (steadyStateStats != null) { | ||||||||||||||||||||||||||||||||||||||
steadyStateStats.printTotal(); | ||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
visz11 marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||||||||||||||||||||||||||||||||||||||
} else { | ||||||||||||||||||||||||||||||||||||||
// Make sure all messages are sent before printing out the stats and the metrics | ||||||||||||||||||||||||||||||||||||||
// We need to do this in a different branch for now since tests/kafkatest/sanity_checks/test_performance_services.py | ||||||||||||||||||||||||||||||||||||||
Comment on lines
125
to
134
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Duplicate CodeIdentical code blocks for printing stats appear in both branches of the conditional. This violates the DRY principle and creates maintenance burden when changes to the stats printing logic are needed, requiring updates in multiple places. Standards
|
||||||||||||||||||||||||||||||||||||||
|
@@ -124,6 +136,10 @@ record = new ProducerRecord<>(config.topicName, payload); | |||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||
/* print final results */ | ||||||||||||||||||||||||||||||||||||||
stats.printTotal(); | ||||||||||||||||||||||||||||||||||||||
/* print steady-state stats if relevant */ | ||||||||||||||||||||||||||||||||||||||
if (steadyStateStats != null) { | ||||||||||||||||||||||||||||||||||||||
steadyStateStats.printTotal(); | ||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||
/* print out metrics */ | ||||||||||||||||||||||||||||||||||||||
ToolsUtils.printMetrics(producer.metrics()); | ||||||||||||||||||||||||||||||||||||||
|
@@ -146,8 +162,8 @@ KafkaProducer<byte[], byte[]> createKafkaProducer(Properties props) { | |||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||
Callback cb; | ||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||
Stats stats; | ||||||||||||||||||||||||||||||||||||||
Stats steadyStateStats; | ||||||||||||||||||||||||||||||||||||||
Comment on lines
166
to
+167
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Inconsistent Variable DeclarationThe steadyStateStats variable is declared but not initialized, while stats is similarly declared. This inconsistency in variable initialization pattern could lead to confusion and potential null pointer exceptions if not properly managed throughout the code. Standards
|
||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||
static byte[] generateRandomPayload(Integer recordSize, List<byte[]> payloadByteList, byte[] payload, | ||||||||||||||||||||||||||||||||||||||
SplittableRandom random, boolean payloadMonotonic, long recordValue) { | ||||||||||||||||||||||||||||||||||||||
|
@@ -163,7 +179,7 @@ static byte[] generateRandomPayload(Integer recordSize, List<byte[]> payloadByte | |||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
return payload; | ||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||
static Properties readProps(List<String> producerProps, String producerConfig) throws IOException { | ||||||||||||||||||||||||||||||||||||||
Properties props = new Properties(); | ||||||||||||||||||||||||||||||||||||||
if (producerConfig != null) { | ||||||||||||||||||||||||||||||||||||||
|
@@ -326,6 +342,16 @@ static ArgumentParser argParser() { | |||||||||||||||||||||||||||||||||||||
"--producer.config, or --transactional-id but --transaction-duration-ms is not specified, " + | ||||||||||||||||||||||||||||||||||||||
"the default value will be 3000."); | ||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||
parser.addArgument("--warmup-records") | ||||||||||||||||||||||||||||||||||||||
.action(store()) | ||||||||||||||||||||||||||||||||||||||
.required(false) | ||||||||||||||||||||||||||||||||||||||
.type(Long.class) | ||||||||||||||||||||||||||||||||||||||
.metavar("WARMUP-RECORDS") | ||||||||||||||||||||||||||||||||||||||
.dest("warmupRecords") | ||||||||||||||||||||||||||||||||||||||
.setDefault(0L) | ||||||||||||||||||||||||||||||||||||||
.help("The number of records to treat as warmup; these initial records will not be included in steady-state statistics. " + | ||||||||||||||||||||||||||||||||||||||
"An additional summary line will be printed describing the steady-state statistics. (default: 0)."); | ||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||
return parser; | ||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||
|
@@ -346,8 +372,14 @@ static class Stats { | |||||||||||||||||||||||||||||||||||||
private long windowTotalLatency; | ||||||||||||||||||||||||||||||||||||||
private long windowBytes; | ||||||||||||||||||||||||||||||||||||||
private long windowStart; | ||||||||||||||||||||||||||||||||||||||
private final boolean isSteadyState; | ||||||||||||||||||||||||||||||||||||||
private boolean steadyStateActive; | ||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||
public Stats(long numRecords) { | ||||||||||||||||||||||||||||||||||||||
this(numRecords, false); | ||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
Comment on lines
+379
to
+381
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Redundant Parameter CheckThe Stats constructor with one parameter calls the two-parameter constructor with false for isSteadyState, but later in the code, steadyStateActive is set based on config.warmupRecords > 0. This inconsistency in initialization could lead to incorrect behavior in steady state reporting. Standards
|
||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||
public Stats(long numRecords, int reportingInterval) { | ||||||||||||||||||||||||||||||||||||||
public Stats(long numRecords, boolean isSteadyState) { | ||||||||||||||||||||||||||||||||||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Parameter Name ChangeThe constructor parameter was changed from 'reportingInterval' to 'isSteadyState' with different types (int to boolean), but the parameter is still used as reportingInterval in the constructor body. This creates a significant maintainability issue as the parameter name no longer reflects its usage.
Commitable Suggestion
Suggested change
Standards
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Inconsistent parameter naming in Stats constructorThe constructor parameter was changed from 'reportingInterval' to 'isSteadyState' with different types, but the parameter name no longer reflects its usage. This creates confusion as the parameter name suggests one purpose while the implementation uses it differently. Standards
|
||||||||||||||||||||||||||||||||||||||
this.start = System.currentTimeMillis(); | ||||||||||||||||||||||||||||||||||||||
this.windowStart = System.currentTimeMillis(); | ||||||||||||||||||||||||||||||||||||||
this.iteration = 0; | ||||||||||||||||||||||||||||||||||||||
|
@@ -360,7 +392,9 @@ public Stats(long numRecords, int reportingInterval) { | |||||||||||||||||||||||||||||||||||||
this.windowTotalLatency = 0; | ||||||||||||||||||||||||||||||||||||||
this.windowBytes = 0; | ||||||||||||||||||||||||||||||||||||||
this.totalLatency = 0; | ||||||||||||||||||||||||||||||||||||||
this.reportingInterval = reportingInterval; | ||||||||||||||||||||||||||||||||||||||
this.reportingInterval = 5000; | ||||||||||||||||||||||||||||||||||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hardcoded ValueThe reporting interval is now hardcoded to 5000 instead of being passed as a parameter. This reduces flexibility and makes the code less configurable. The previous parameterized approach was more maintainable as it allowed for different reporting intervals without code changes. Standards
|
||||||||||||||||||||||||||||||||||||||
this.isSteadyState = isSteadyState; | ||||||||||||||||||||||||||||||||||||||
this.steadyStateActive = isSteadyState; | ||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||
public void record(int latency, int bytes, long time) { | ||||||||||||||||||||||||||||||||||||||
|
@@ -378,7 +412,12 @@ public void record(int latency, int bytes, long time) { | |||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
/* maybe report the recent perf */ | ||||||||||||||||||||||||||||||||||||||
if (time - windowStart >= reportingInterval) { | ||||||||||||||||||||||||||||||||||||||
printWindow(); | ||||||||||||||||||||||||||||||||||||||
if (this.isSteadyState && count == windowCount) { | ||||||||||||||||||||||||||||||||||||||
System.out.println("Beginning steady state."); | ||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
if (this.isSteadyState || !this.steadyStateActive) { | ||||||||||||||||||||||||||||||||||||||
printWindow(); | ||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
newWindow(); | ||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
|
@@ -428,8 +467,9 @@ public void printTotal() { | |||||||||||||||||||||||||||||||||||||
double recsPerSec = 1000.0 * count / (double) elapsed; | ||||||||||||||||||||||||||||||||||||||
double mbPerSec = 1000.0 * this.bytes / (double) elapsed / (1024.0 * 1024.0); | ||||||||||||||||||||||||||||||||||||||
int[] percs = percentiles(this.latencies, index, 0.5, 0.95, 0.99, 0.999); | ||||||||||||||||||||||||||||||||||||||
System.out.printf("%d records sent, %.1f records/sec (%.2f MB/sec), %.2f ms avg latency, %.2f ms max latency, %d ms 50th, %d ms 95th, %d ms 99th, %d ms 99.9th.%n", | ||||||||||||||||||||||||||||||||||||||
System.out.printf("%d%s records sent, %f records/sec (%.2f MB/sec), %.2f ms avg latency, %.2f ms max latency, %d ms 50th, %d ms 95th, %d ms 99th, %d ms 99.9th.%n", | ||||||||||||||||||||||||||||||||||||||
count, | ||||||||||||||||||||||||||||||||||||||
this.isSteadyState ? " steady state" : "", | ||||||||||||||||||||||||||||||||||||||
recsPerSec, | ||||||||||||||||||||||||||||||||||||||
mbPerSec, | ||||||||||||||||||||||||||||||||||||||
totalLatency / (double) count, | ||||||||||||||||||||||||||||||||||||||
|
@@ -456,10 +496,12 @@ static final class PerfCallback implements Callback { | |||||||||||||||||||||||||||||||||||||
private final long start; | ||||||||||||||||||||||||||||||||||||||
private final int bytes; | ||||||||||||||||||||||||||||||||||||||
private final Stats stats; | ||||||||||||||||||||||||||||||||||||||
private final Stats steadyStateStats; | ||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||
public PerfCallback(long start, int bytes, Stats stats) { | ||||||||||||||||||||||||||||||||||||||
public PerfCallback(long start, int bytes, Stats stats, Stats steadyStateStats) { | ||||||||||||||||||||||||||||||||||||||
this.start = start; | ||||||||||||||||||||||||||||||||||||||
this.stats = stats; | ||||||||||||||||||||||||||||||||||||||
this.steadyStateStats = steadyStateStats; | ||||||||||||||||||||||||||||||||||||||
this.bytes = bytes; | ||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||
|
@@ -471,6 +513,10 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { | |||||||||||||||||||||||||||||||||||||
if (exception == null) { | ||||||||||||||||||||||||||||||||||||||
this.stats.record(latency, bytes, now); | ||||||||||||||||||||||||||||||||||||||
this.stats.iteration++; | ||||||||||||||||||||||||||||||||||||||
if (steadyStateStats != null) { | ||||||||||||||||||||||||||||||||||||||
this.steadyStateStats.record(latency, bytes, now); | ||||||||||||||||||||||||||||||||||||||
this.steadyStateStats.iteration++; | ||||||||||||||||||||||||||||||||||||||
Comment on lines
+517
to
+519
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The
Suggested change
|
||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
Comment on lines
+517
to
+520
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Missing Initialization CheckThe code checks if steadyStateStats is null before accessing it, but then uses this.steadyStateStats which might still be null if the field wasn't initialized. This inconsistent null check pattern could lead to NullPointerException.
Commitable Suggestion
Suggested change
Standards
Comment on lines
+517
to
+520
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. NullPointerException in PerfCallback.onCompletionThe code checks if steadyStateStats is null before accessing it, but then uses this.steadyStateStats which could still be null. This inconsistent null check pattern could lead to NullPointerException when steadyStateStats is null. Standards
Comment on lines
+517
to
+520
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Steady State Recording LogicThe code records all messages in steadyStateStats regardless of whether they're warmup or steady state records. This defeats the purpose of separating warmup from steady state measurements and will skew the steady state statistics. Standards
|
||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
if (exception != null) | ||||||||||||||||||||||||||||||||||||||
exception.printStackTrace(); | ||||||||||||||||||||||||||||||||||||||
|
@@ -479,7 +525,8 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { | |||||||||||||||||||||||||||||||||||||
|
||||||||||||||||||||||||||||||||||||||
static final class ConfigPostProcessor { | ||||||||||||||||||||||||||||||||||||||
final String topicName; | ||||||||||||||||||||||||||||||||||||||
final Long numRecords; | ||||||||||||||||||||||||||||||||||||||
final long numRecords; | ||||||||||||||||||||||||||||||||||||||
final long warmupRecords; | ||||||||||||||||||||||||||||||||||||||
final Integer recordSize; | ||||||||||||||||||||||||||||||||||||||
final double throughput; | ||||||||||||||||||||||||||||||||||||||
final boolean payloadMonotonic; | ||||||||||||||||||||||||||||||||||||||
|
@@ -493,6 +540,7 @@ public ConfigPostProcessor(ArgumentParser parser, String[] args) throws IOExcept | |||||||||||||||||||||||||||||||||||||
Namespace namespace = parser.parseArgs(args); | ||||||||||||||||||||||||||||||||||||||
this.topicName = namespace.getString("topic"); | ||||||||||||||||||||||||||||||||||||||
this.numRecords = namespace.getLong("numRecords"); | ||||||||||||||||||||||||||||||||||||||
this.warmupRecords = Math.max(namespace.getLong("warmupRecords"), 0); | ||||||||||||||||||||||||||||||||||||||
this.recordSize = namespace.getInt("recordSize"); | ||||||||||||||||||||||||||||||||||||||
this.throughput = namespace.getDouble("throughput"); | ||||||||||||||||||||||||||||||||||||||
this.payloadMonotonic = namespace.getBoolean("payloadMonotonic"); | ||||||||||||||||||||||||||||||||||||||
|
@@ -503,9 +551,12 @@ public ConfigPostProcessor(ArgumentParser parser, String[] args) throws IOExcept | |||||||||||||||||||||||||||||||||||||
String payloadFilePath = namespace.getString("payloadFile"); | ||||||||||||||||||||||||||||||||||||||
Long transactionDurationMsArg = namespace.getLong("transactionDurationMs"); | ||||||||||||||||||||||||||||||||||||||
String transactionIdArg = namespace.getString("transactionalId"); | ||||||||||||||||||||||||||||||||||||||
if (numRecords != null && numRecords <= 0) { | ||||||||||||||||||||||||||||||||||||||
if (numRecords <= 0) { | ||||||||||||||||||||||||||||||||||||||
throw new ArgumentParserException("--num-records should be greater than zero", parser); | ||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
if (warmupRecords >= numRecords) { | ||||||||||||||||||||||||||||||||||||||
throw new ArgumentParserException("The value for --warmup-records must be strictly fewer than the number of records in the test, --num-records.", parser); | ||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
Comment on lines
+558
to
+560
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This check prevents
Suggested change
Comment on lines
+558
to
+560
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Warmup Validation FlawThe validation allows negative warmup records to pass through since they're less than numRecords. While line 543 sets a default of 0 and the code later uses Math.max(warmupRecords, 0), this validation should explicitly check for negative values to prevent confusing behavior.
Commitable Suggestion
Suggested change
Standards
Comment on lines
+558
to
+560
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Negative Warmup Records Validation MissingThe validation allows negative warmup records to pass through since they're less than numRecords. While line 544 sets a default of 0 and Math.max is used later, explicit validation would prevent confusing behavior. Standards
|
||||||||||||||||||||||||||||||||||||||
if (recordSize != null && recordSize <= 0) { | ||||||||||||||||||||||||||||||||||||||
throw new ArgumentParserException("--record-size should be greater than zero", parser); | ||||||||||||||||||||||||||||||||||||||
} | ||||||||||||||||||||||||||||||||||||||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The
stats
object is initialized withconfig.numRecords
, which includes the warmup records. Consider initializing it withconfig.numRecords - config.warmupRecords
to represent only the steady-state records. This might provide a clearer representation of the steady-state statistics.