@@ -76,7 +76,12 @@ void start(String[] args) throws IOException {
76
76
// not thread-safe, do not share with other threads
77
77
SplittableRandom random = new SplittableRandom (0 );
78
78
ProducerRecord <byte [], byte []> record ;
79
- stats = new Stats (config .numRecords , 5000 );
79
+
80
+ if (config .warmupRecords > 0 ) {
81
+ System .out .println ("Warmup first " + config .warmupRecords + " records. Steady state results will print after the complete test summary." );
82
+ }
83
+ boolean isSteadyState = false ;
84
+ stats = new Stats (config .numRecords , isSteadyState );
80
85
long startMs = System .currentTimeMillis ();
81
86
82
87
ThroughputThrottler throttler = new ThroughputThrottler (config .throughput , startMs );
@@ -95,7 +100,11 @@ void start(String[] args) throws IOException {
95
100
record = new ProducerRecord <>(config .topicName , payload );
96
101
97
102
long sendStartMs = System .currentTimeMillis ();
98
- cb = new PerfCallback (sendStartMs , payload .length , stats );
103
+ if ((isSteadyState = config .warmupRecords > 0 ) && i == config .warmupRecords ) {
104
+ steadyStateStats = new Stats (config .numRecords - config .warmupRecords , isSteadyState );
105
+ stats .suppressPrinting ();
106
+ }
107
+ cb = new PerfCallback (sendStartMs , payload .length , stats , steadyStateStats );
99
108
producer .send (record , cb );
100
109
101
110
currentTransactionSize ++;
@@ -117,6 +126,10 @@ record = new ProducerRecord<>(config.topicName, payload);
117
126
118
127
/* print final results */
119
128
stats .printTotal ();
129
+ /* print steady-state stats if relevant */
130
+ if (steadyStateStats != null ) {
131
+ steadyStateStats .printTotal ();
132
+ }
120
133
} else {
121
134
// Make sure all messages are sent before printing out the stats and the metrics
122
135
// We need to do this in a different branch for now since tests/kafkatest/sanity_checks/test_performance_services.py
@@ -125,6 +138,10 @@ record = new ProducerRecord<>(config.topicName, payload);
125
138
126
139
/* print final results */
127
140
stats .printTotal ();
141
+ /* print steady-state stats if relevant */
142
+ if (steadyStateStats != null ) {
143
+ steadyStateStats .printTotal ();
144
+ }
128
145
129
146
/* print out metrics */
130
147
ToolsUtils .printMetrics (producer .metrics ());
@@ -147,8 +164,8 @@ KafkaProducer<byte[], byte[]> createKafkaProducer(Properties props) {
147
164
}
148
165
149
166
Callback cb ;
150
-
151
167
Stats stats ;
168
+ Stats steadyStateStats ;
152
169
153
170
static byte [] generateRandomPayload (Integer recordSize , List <byte []> payloadByteList , byte [] payload ,
154
171
SplittableRandom random , boolean payloadMonotonic , long recordValue ) {
@@ -164,7 +181,7 @@ static byte[] generateRandomPayload(Integer recordSize, List<byte[]> payloadByte
164
181
}
165
182
return payload ;
166
183
}
167
-
184
+
168
185
static Properties readProps (List <String > producerProps , String producerConfig ) throws IOException {
169
186
Properties props = new Properties ();
170
187
if (producerConfig != null ) {
@@ -331,6 +348,16 @@ static ArgumentParser argParser() {
331
348
"--producer.config, or --transactional-id but --transaction-duration-ms is not specified, " +
332
349
"the default value will be 3000." );
333
350
351
+ parser .addArgument ("--warmup-records" )
352
+ .action (store ())
353
+ .required (false )
354
+ .type (Long .class )
355
+ .metavar ("WARMUP-RECORDS" )
356
+ .dest ("warmupRecords" )
357
+ .setDefault (0L )
358
+ .help ("The number of records to treat as warmup; these initial records will not be included in steady-state statistics. " +
359
+ "An additional summary line will be printed describing the steady-state statistics. (default: 0)." );
360
+
334
361
return parser ;
335
362
}
336
363
@@ -351,8 +378,10 @@ static class Stats {
351
378
private long windowTotalLatency ;
352
379
private long windowBytes ;
353
380
private long windowStart ;
381
+ private final boolean isSteadyState ;
382
+ private boolean suppressPrint ;
354
383
355
- public Stats (long numRecords , int reportingInterval ) {
384
+ public Stats (long numRecords , boolean isSteadyState ) {
356
385
this .start = System .currentTimeMillis ();
357
386
this .windowStart = System .currentTimeMillis ();
358
387
this .iteration = 0 ;
@@ -365,7 +394,9 @@ public Stats(long numRecords, int reportingInterval) {
365
394
this .windowTotalLatency = 0 ;
366
395
this .windowBytes = 0 ;
367
396
this .totalLatency = 0 ;
368
- this .reportingInterval = reportingInterval ;
397
+ this .reportingInterval = 5000 ;
398
+ this .isSteadyState = isSteadyState ;
399
+ this .suppressPrint = false ;
369
400
}
370
401
371
402
public void record (int latency , int bytes , long time ) {
@@ -383,9 +414,15 @@ public void record(int latency, int bytes, long time) {
383
414
}
384
415
/* maybe report the recent perf */
385
416
if (time - windowStart >= reportingInterval ) {
386
- printWindow ();
417
+ if (this .isSteadyState && count == windowCount ) {
418
+ System .out .println ("In steady state." );
419
+ }
420
+ if (!this .suppressPrint ) {
421
+ printWindow ();
422
+ }
387
423
newWindow ();
388
424
}
425
+ this .iteration ++;
389
426
}
390
427
391
428
public long totalCount () {
@@ -433,8 +470,9 @@ public void printTotal() {
433
470
double recsPerSec = 1000.0 * count / (double ) elapsed ;
434
471
double mbPerSec = 1000.0 * this .bytes / (double ) elapsed / (1024.0 * 1024.0 );
435
472
int [] percs = percentiles (this .latencies , index , 0.5 , 0.95 , 0.99 , 0.999 );
436
- 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" ,
473
+ 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" ,
437
474
count ,
475
+ this .isSteadyState ? " steady state" : "" ,
438
476
recsPerSec ,
439
477
mbPerSec ,
440
478
totalLatency / (double ) count ,
@@ -455,16 +493,22 @@ private static int[] percentiles(int[] latencies, int count, double... percentil
455
493
}
456
494
return values ;
457
495
}
496
+
497
+ public void suppressPrinting () {
498
+ this .suppressPrint = true ;
499
+ }
458
500
}
459
501
460
502
static final class PerfCallback implements Callback {
461
503
private final long start ;
462
504
private final int bytes ;
463
505
private final Stats stats ;
506
+ private final Stats steadyStateStats ;
464
507
465
- public PerfCallback (long start , int bytes , Stats stats ) {
508
+ public PerfCallback (long start , int bytes , Stats stats , Stats steadyStateStats ) {
466
509
this .start = start ;
467
510
this .stats = stats ;
511
+ this .steadyStateStats = steadyStateStats ;
468
512
this .bytes = bytes ;
469
513
}
470
514
@@ -475,7 +519,9 @@ public void onCompletion(RecordMetadata metadata, Exception exception) {
475
519
// magically printed when the sending fails.
476
520
if (exception == null ) {
477
521
this .stats .record (latency , bytes , now );
478
- this .stats .iteration ++;
522
+ if (steadyStateStats != null ) {
523
+ this .steadyStateStats .record (latency , bytes , now );
524
+ }
479
525
}
480
526
if (exception != null )
481
527
exception .printStackTrace ();
@@ -484,7 +530,8 @@ public void onCompletion(RecordMetadata metadata, Exception exception) {
484
530
485
531
static final class ConfigPostProcessor {
486
532
final String topicName ;
487
- final Long numRecords ;
533
+ final long numRecords ;
534
+ final long warmupRecords ;
488
535
final Integer recordSize ;
489
536
final double throughput ;
490
537
final boolean payloadMonotonic ;
@@ -498,6 +545,7 @@ public ConfigPostProcessor(ArgumentParser parser, String[] args) throws IOExcept
498
545
Namespace namespace = parser .parseArgs (args );
499
546
this .topicName = namespace .getString ("topic" );
500
547
this .numRecords = namespace .getLong ("numRecords" );
548
+ this .warmupRecords = Math .max (namespace .getLong ("warmupRecords" ), 0 );
501
549
this .recordSize = namespace .getInt ("recordSize" );
502
550
this .throughput = namespace .getDouble ("throughput" );
503
551
this .payloadMonotonic = namespace .getBoolean ("payloadMonotonic" );
@@ -508,9 +556,12 @@ public ConfigPostProcessor(ArgumentParser parser, String[] args) throws IOExcept
508
556
String payloadFilePath = namespace .getString ("payloadFile" );
509
557
Long transactionDurationMsArg = namespace .getLong ("transactionDurationMs" );
510
558
String transactionIdArg = namespace .getString ("transactionalId" );
511
- if (numRecords != null && numRecords <= 0 ) {
559
+ if (numRecords <= 0 ) {
512
560
throw new ArgumentParserException ("--num-records should be greater than zero" , parser );
513
561
}
562
+ if (warmupRecords >= numRecords ) {
563
+ throw new ArgumentParserException ("The value for --warmup-records must be strictly fewer than the number of records in the test, --num-records." , parser );
564
+ }
514
565
if (recordSize != null && recordSize <= 0 ) {
515
566
throw new ArgumentParserException ("--record-size should be greater than zero" , parser );
516
567
}
0 commit comments