forked from confluentinc/librdkafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
rdkafka_performance.c
1781 lines (1510 loc) · 67.6 KB
/
rdkafka_performance.c
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2022, Magnus Edenhill
* 2023, Confluent Inc.
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met:
*
* 1. Redistributions of source code must retain the above copyright notice,
* this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright notice,
* this list of conditions and the following disclaimer in the documentation
* and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
/**
* Apache Kafka consumer & producer performance tester
* using the Kafka driver from librdkafka
* (https://github.com/confluentinc/librdkafka)
*/
#ifdef _MSC_VER
#define _CRT_SECURE_NO_WARNINGS /* Silence nonsense on MSVC */
#endif
#include "../src/rd.h"
#define _GNU_SOURCE /* for strndup() */
#include <ctype.h>
#include <signal.h>
#include <string.h>
#include <errno.h>
/* Typical include path would be <librdkafka/rdkafka.h>, but this program
* is built from within the librdkafka source tree and thus differs. */
#include "rdkafka.h" /* for Kafka driver */
/* Do not include these defines from your program, they will not be
* provided by librdkafka. */
#include "rd.h"
#include "rdtime.h"
#ifdef _WIN32
#include "../win32/wingetopt.h"
#include "../win32/wintime.h"
#endif
static volatile sig_atomic_t run = 1;
static int forever = 1;
static rd_ts_t dispintvl = 1000;
static int do_seq = 0;
static int exit_after = 0;
static int exit_eof = 0;
static FILE *stats_fp;
static int dr_disp_div;
static int verbosity = 1;
static int latency_mode = 0;
static FILE *latency_fp = NULL;
static int msgcnt = -1;
static int incremental_mode = 0;
static int partition_cnt = 0;
static int eof_cnt = 0;
static int with_dr = 1;
static int read_hdrs = 0;
static void stop(int sig) {
if (!run)
exit(0);
run = 0;
}
static long int msgs_wait_cnt = 0;
static long int msgs_wait_produce_cnt = 0;
static rd_ts_t t_end;
static rd_kafka_t *global_rk;
struct avg {
int64_t val;
int cnt;
uint64_t ts_start;
};
static struct {
rd_ts_t t_start;
rd_ts_t t_end;
rd_ts_t t_end_send;
uint64_t msgs;
uint64_t msgs_last;
uint64_t msgs_dr_ok;
uint64_t msgs_dr_err;
uint64_t bytes_dr_ok;
uint64_t bytes;
uint64_t bytes_last;
uint64_t tx;
uint64_t tx_err;
uint64_t avg_rtt;
uint64_t offset;
rd_ts_t t_fetch_latency;
rd_ts_t t_last;
rd_ts_t t_enobufs_last;
rd_ts_t t_total;
rd_ts_t latency_last;
rd_ts_t latency_lo;
rd_ts_t latency_hi;
rd_ts_t latency_sum;
int latency_cnt;
int64_t last_offset;
} cnt;
uint64_t wall_clock(void) {
struct timeval tv;
gettimeofday(&tv, NULL);
return ((uint64_t)tv.tv_sec * 1000000LLU) + ((uint64_t)tv.tv_usec);
}
static void err_cb(rd_kafka_t *rk, int err, const char *reason, void *opaque) {
if (err == RD_KAFKA_RESP_ERR__FATAL) {
char errstr[512];
err = rd_kafka_fatal_error(rk, errstr, sizeof(errstr));
printf("%% FATAL ERROR CALLBACK: %s: %s: %s\n",
rd_kafka_name(rk), rd_kafka_err2str(err), errstr);
} else {
printf("%% ERROR CALLBACK: %s: %s: %s\n", rd_kafka_name(rk),
rd_kafka_err2str(err), reason);
}
}
static void throttle_cb(rd_kafka_t *rk,
const char *broker_name,
int32_t broker_id,
int throttle_time_ms,
void *opaque) {
printf("%% THROTTLED %dms by %s (%" PRId32 ")\n", throttle_time_ms,
broker_name, broker_id);
}
static void offset_commit_cb(rd_kafka_t *rk,
rd_kafka_resp_err_t err,
rd_kafka_topic_partition_list_t *offsets,
void *opaque) {
int i;
if (err || verbosity >= 2)
printf("%% Offset commit of %d partition(s): %s\n",
offsets->cnt, rd_kafka_err2str(err));
for (i = 0; i < offsets->cnt; i++) {
rd_kafka_topic_partition_t *rktpar = &offsets->elems[i];
if (rktpar->err || verbosity >= 2)
printf("%% %s [%" PRId32 "] @ %" PRId64 ": %s\n",
rktpar->topic, rktpar->partition, rktpar->offset,
rd_kafka_err2str(err));
}
}
/**
* @brief Add latency measurement
*/
static void latency_add(int64_t ts, const char *who) {
if (ts > cnt.latency_hi)
cnt.latency_hi = ts;
if (!cnt.latency_lo || ts < cnt.latency_lo)
cnt.latency_lo = ts;
cnt.latency_last = ts;
cnt.latency_cnt++;
cnt.latency_sum += ts;
if (latency_fp)
fprintf(latency_fp, "%" PRIu64 "\n", ts);
}
static void msg_delivered(rd_kafka_t *rk,
const rd_kafka_message_t *rkmessage,
void *opaque) {
static rd_ts_t last;
rd_ts_t now = rd_clock();
static int msgs;
msgs++;
msgs_wait_cnt--;
if (rkmessage->err)
cnt.msgs_dr_err++;
else {
cnt.msgs_dr_ok++;
cnt.bytes_dr_ok += rkmessage->len;
}
if (latency_mode) {
/* Extract latency */
int64_t source_ts;
if (sscanf(rkmessage->payload, "LATENCY:%" SCNd64,
&source_ts) == 1)
latency_add(wall_clock() - source_ts, "producer");
}
if ((rkmessage->err && (cnt.msgs_dr_err < 50 ||
!(cnt.msgs_dr_err % (dispintvl / 1000)))) ||
!last || msgs_wait_cnt < 5 || !(msgs_wait_cnt % dr_disp_div) ||
(now - last) >= dispintvl * 1000 || verbosity >= 3) {
if (rkmessage->err && verbosity >= 2)
printf("%% Message delivery failed (broker %" PRId32
"): "
"%s [%" PRId32
"]: "
"%s (%li remain)\n",
rd_kafka_message_broker_id(rkmessage),
rd_kafka_topic_name(rkmessage->rkt),
rkmessage->partition,
rd_kafka_err2str(rkmessage->err), msgs_wait_cnt);
else if (verbosity > 2)
printf("%% Message delivered (offset %" PRId64
", broker %" PRId32
"): "
"%li remain\n",
rkmessage->offset,
rd_kafka_message_broker_id(rkmessage),
msgs_wait_cnt);
if (verbosity >= 3 && do_seq)
printf(" --> \"%.*s\"\n", (int)rkmessage->len,
(const char *)rkmessage->payload);
last = now;
}
cnt.last_offset = rkmessage->offset;
if (msgs_wait_produce_cnt == 0 && msgs_wait_cnt == 0 && !forever) {
if (verbosity >= 2 && cnt.msgs > 0) {
double error_percent =
(double)(cnt.msgs - cnt.msgs_dr_ok) / cnt.msgs *
100;
printf(
"%% Messages delivered with failure "
"percentage of %.5f%%\n",
error_percent);
}
t_end = rd_clock();
run = 0;
}
if (exit_after && exit_after <= msgs) {
printf("%% Hard exit after %i messages, as requested\n",
exit_after);
exit(0);
}
}
static void msg_consume(rd_kafka_message_t *rkmessage, void *opaque) {
if (rkmessage->err) {
if (rkmessage->err == RD_KAFKA_RESP_ERR__PARTITION_EOF) {
cnt.offset = rkmessage->offset;
if (verbosity >= 1)
printf(
"%% Consumer reached end of "
"%s [%" PRId32
"] "
"message queue at offset %" PRId64 "\n",
rd_kafka_topic_name(rkmessage->rkt),
rkmessage->partition, rkmessage->offset);
if (exit_eof && ++eof_cnt == partition_cnt)
run = 0;
return;
}
printf("%% Consume error for topic \"%s\" [%" PRId32
"] "
"offset %" PRId64 ": %s\n",
rkmessage->rkt ? rd_kafka_topic_name(rkmessage->rkt)
: "",
rkmessage->partition, rkmessage->offset,
rd_kafka_message_errstr(rkmessage));
if (rkmessage->err == RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION ||
rkmessage->err == RD_KAFKA_RESP_ERR__UNKNOWN_TOPIC)
run = 0;
cnt.msgs_dr_err++;
return;
}
/* Start measuring from first message received */
if (!cnt.t_start)
cnt.t_start = cnt.t_last = rd_clock();
cnt.offset = rkmessage->offset;
cnt.msgs++;
cnt.bytes += rkmessage->len;
if (verbosity >= 3 || (verbosity >= 2 && !(cnt.msgs % 1000000)))
printf("@%" PRId64 ": %.*s: %.*s\n", rkmessage->offset,
(int)rkmessage->key_len, (char *)rkmessage->key,
(int)rkmessage->len, (char *)rkmessage->payload);
if (latency_mode) {
int64_t remote_ts, ts;
if (rkmessage->len > 8 &&
!memcmp(rkmessage->payload, "LATENCY:", 8) &&
sscanf(rkmessage->payload, "LATENCY:%" SCNd64,
&remote_ts) == 1) {
ts = wall_clock() - remote_ts;
if (ts > 0 && ts < (1000000 * 60 * 5)) {
latency_add(ts, "consumer");
} else {
if (verbosity >= 1)
printf(
"Received latency timestamp is too "
"far off: %" PRId64
"us (message offset %" PRId64
"): ignored\n",
ts, rkmessage->offset);
}
} else if (verbosity > 1)
printf("not a LATENCY payload: %.*s\n",
(int)rkmessage->len, (char *)rkmessage->payload);
}
if (read_hdrs) {
rd_kafka_headers_t *hdrs;
/* Force parsing of headers but don't do anything with them. */
rd_kafka_message_headers(rkmessage, &hdrs);
}
if (msgcnt != -1 && (int)cnt.msgs >= msgcnt)
run = 0;
}
static void rebalance_cb(rd_kafka_t *rk,
rd_kafka_resp_err_t err,
rd_kafka_topic_partition_list_t *partitions,
void *opaque) {
rd_kafka_error_t *error = NULL;
rd_kafka_resp_err_t ret_err = RD_KAFKA_RESP_ERR_NO_ERROR;
if (exit_eof && !strcmp(rd_kafka_rebalance_protocol(rk), "COOPERATIVE"))
fprintf(stderr,
"%% This example has not been modified to "
"support -e (exit on EOF) when "
"partition.assignment.strategy "
"is set to an incremental/cooperative strategy: "
"-e will not behave as expected\n");
switch (err) {
case RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS:
fprintf(stderr,
"%% Group rebalanced (%s): "
"%d new partition(s) assigned\n",
rd_kafka_rebalance_protocol(rk), partitions->cnt);
if (!strcmp(rd_kafka_rebalance_protocol(rk), "COOPERATIVE")) {
error = rd_kafka_incremental_assign(rk, partitions);
} else {
ret_err = rd_kafka_assign(rk, partitions);
eof_cnt = 0;
}
partition_cnt += partitions->cnt;
break;
case RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS:
fprintf(stderr,
"%% Group rebalanced (%s): %d partition(s) revoked\n",
rd_kafka_rebalance_protocol(rk), partitions->cnt);
if (!strcmp(rd_kafka_rebalance_protocol(rk), "COOPERATIVE")) {
error = rd_kafka_incremental_unassign(rk, partitions);
partition_cnt -= partitions->cnt;
} else {
ret_err = rd_kafka_assign(rk, NULL);
partition_cnt = 0;
}
eof_cnt = 0; /* FIXME: Not correct for incremental case */
break;
default:
break;
}
if (error) {
fprintf(stderr, "%% incremental assign failure: %s\n",
rd_kafka_error_string(error));
rd_kafka_error_destroy(error);
} else if (ret_err) {
fprintf(stderr, "%% assign failure: %s\n",
rd_kafka_err2str(ret_err));
}
}
/**
* Find and extract single value from a two-level search.
* First find 'field1', then find 'field2' and extract its value.
* Returns 0 on miss else the value.
*/
static uint64_t json_parse_fields(const char *json,
const char **end,
const char *field1,
const char *field2) {
const char *t = json;
const char *t2;
int len1 = (int)strlen(field1);
int len2 = (int)strlen(field2);
while ((t2 = strstr(t, field1))) {
uint64_t v;
t = t2;
t += len1;
/* Find field */
if (!(t2 = strstr(t, field2)))
continue;
t2 += len2;
while (isspace((int)*t2))
t2++;
v = strtoull(t2, (char **)&t, 10);
if (t2 == t)
continue;
*end = t;
return v;
}
*end = t + strlen(t);
return 0;
}
/**
* Parse various values from rdkafka stats
*/
static void json_parse_stats(const char *json) {
const char *t;
#define MAX_AVGS 100 /* max number of brokers to scan for rtt */
uint64_t avg_rtt[MAX_AVGS + 1];
int avg_rtt_i = 0;
/* Store totals at end of array */
avg_rtt[MAX_AVGS] = 0;
/* Extract all broker RTTs */
t = json;
while (avg_rtt_i < MAX_AVGS && *t) {
avg_rtt[avg_rtt_i] =
json_parse_fields(t, &t, "\"rtt\":", "\"avg\":");
/* Skip low RTT values, means no messages are passing */
if (avg_rtt[avg_rtt_i] < 100 /*0.1ms*/)
continue;
avg_rtt[MAX_AVGS] += avg_rtt[avg_rtt_i];
avg_rtt_i++;
}
if (avg_rtt_i > 0)
avg_rtt[MAX_AVGS] /= avg_rtt_i;
cnt.avg_rtt = avg_rtt[MAX_AVGS];
}
static int stats_cb(rd_kafka_t *rk, char *json, size_t json_len, void *opaque) {
/* Extract values for our own stats */
json_parse_stats(json);
if (stats_fp)
fprintf(stats_fp, "%s\n", json);
return 0;
}
#define _OTYPE_TAB 0x1 /* tabular format */
#define _OTYPE_SUMMARY 0x2 /* summary format */
#define _OTYPE_FORCE 0x4 /* force output regardless of interval timing */
static void
print_stats(rd_kafka_t *rk, int mode, int otype, const char *compression) {
rd_ts_t now = rd_clock();
rd_ts_t t_total;
static int rows_written = 0;
int print_header;
double latency_avg = 0.0f;
char extra[512];
int extra_of = 0;
*extra = '\0';
if (!(otype & _OTYPE_FORCE) &&
(((otype & _OTYPE_SUMMARY) && verbosity == 0) ||
cnt.t_last + dispintvl > now))
return;
print_header = !rows_written || (verbosity > 0 && !(rows_written % 20));
if (cnt.t_end_send)
t_total = cnt.t_end_send - cnt.t_start;
else if (cnt.t_end)
t_total = cnt.t_end - cnt.t_start;
else if (cnt.t_start)
t_total = now - cnt.t_start;
else
t_total = 1;
if (latency_mode && cnt.latency_cnt)
latency_avg = (double)cnt.latency_sum / (double)cnt.latency_cnt;
if (mode == 'P') {
if (otype & _OTYPE_TAB) {
#define ROW_START() \
do { \
} while (0)
#define COL_HDR(NAME) printf("| %10.10s ", (NAME))
#define COL_PR64(NAME, VAL) printf("| %10" PRIu64 " ", (VAL))
#define COL_PRF(NAME, VAL) printf("| %10.2f ", (VAL))
#define ROW_END() \
do { \
printf("\n"); \
rows_written++; \
} while (0)
if (print_header) {
/* First time, print header */
ROW_START();
COL_HDR("elapsed");
COL_HDR("msgs");
COL_HDR("bytes");
COL_HDR("rtt");
COL_HDR("dr");
COL_HDR("dr_m/s");
COL_HDR("dr_MB/s");
COL_HDR("dr_err");
COL_HDR("tx_err");
COL_HDR("outq");
COL_HDR("offset");
if (latency_mode) {
COL_HDR("lat_curr");
COL_HDR("lat_avg");
COL_HDR("lat_lo");
COL_HDR("lat_hi");
}
ROW_END();
}
ROW_START();
COL_PR64("elapsed", t_total / 1000);
COL_PR64("msgs", cnt.msgs);
COL_PR64("bytes", cnt.bytes);
COL_PR64("rtt", cnt.avg_rtt / 1000);
COL_PR64("dr", cnt.msgs_dr_ok);
COL_PR64("dr_m/s",
((cnt.msgs_dr_ok * 1000000) / t_total));
COL_PRF("dr_MB/s",
(float)((cnt.bytes_dr_ok) / (float)t_total));
COL_PR64("dr_err", cnt.msgs_dr_err);
COL_PR64("tx_err", cnt.tx_err);
COL_PR64("outq",
rk ? (uint64_t)rd_kafka_outq_len(rk) : 0);
COL_PR64("offset", (uint64_t)cnt.last_offset);
if (latency_mode) {
COL_PRF("lat_curr", cnt.latency_last / 1000.0f);
COL_PRF("lat_avg", latency_avg / 1000.0f);
COL_PRF("lat_lo", cnt.latency_lo / 1000.0f);
COL_PRF("lat_hi", cnt.latency_hi / 1000.0f);
}
ROW_END();
}
if (otype & _OTYPE_SUMMARY) {
printf("%% %" PRIu64
" messages produced "
"(%" PRIu64
" bytes), "
"%" PRIu64
" delivered "
"(offset %" PRId64 ", %" PRIu64
" failed) "
"in %" PRIu64 "ms: %" PRIu64
" msgs/s and "
"%.02f MB/s, "
"%" PRIu64
" produce failures, %i in queue, "
"%s compression\n",
cnt.msgs, cnt.bytes, cnt.msgs_dr_ok,
cnt.last_offset, cnt.msgs_dr_err, t_total / 1000,
((cnt.msgs_dr_ok * 1000000) / t_total),
(float)((cnt.bytes_dr_ok) / (float)t_total),
cnt.tx_err, rk ? rd_kafka_outq_len(rk) : 0,
compression);
}
} else {
if (otype & _OTYPE_TAB) {
if (print_header) {
/* First time, print header */
ROW_START();
COL_HDR("elapsed");
COL_HDR("msgs");
COL_HDR("bytes");
COL_HDR("rtt");
COL_HDR("m/s");
COL_HDR("MB/s");
COL_HDR("rx_err");
COL_HDR("offset");
if (latency_mode) {
COL_HDR("lat_curr");
COL_HDR("lat_avg");
COL_HDR("lat_lo");
COL_HDR("lat_hi");
}
ROW_END();
}
ROW_START();
COL_PR64("elapsed", t_total / 1000);
COL_PR64("msgs", cnt.msgs);
COL_PR64("bytes", cnt.bytes);
COL_PR64("rtt", cnt.avg_rtt / 1000);
COL_PR64("m/s", ((cnt.msgs * 1000000) / t_total));
COL_PRF("MB/s", (float)((cnt.bytes) / (float)t_total));
COL_PR64("rx_err", cnt.msgs_dr_err);
COL_PR64("offset", cnt.offset);
if (latency_mode) {
COL_PRF("lat_curr", cnt.latency_last / 1000.0f);
COL_PRF("lat_avg", latency_avg / 1000.0f);
COL_PRF("lat_lo", cnt.latency_lo / 1000.0f);
COL_PRF("lat_hi", cnt.latency_hi / 1000.0f);
}
ROW_END();
}
if (otype & _OTYPE_SUMMARY) {
if (latency_avg >= 1.0f)
extra_of += rd_snprintf(
extra + extra_of, sizeof(extra) - extra_of,
", latency "
"curr/avg/lo/hi "
"%.2f/%.2f/%.2f/%.2fms",
cnt.latency_last / 1000.0f,
latency_avg / 1000.0f,
cnt.latency_lo / 1000.0f,
cnt.latency_hi / 1000.0f);
printf("%% %" PRIu64 " messages (%" PRIu64
" bytes) "
"consumed in %" PRIu64 "ms: %" PRIu64
" msgs/s "
"(%.02f MB/s)"
"%s\n",
cnt.msgs, cnt.bytes, t_total / 1000,
((cnt.msgs * 1000000) / t_total),
(float)((cnt.bytes) / (float)t_total), extra);
}
if (incremental_mode && now > cnt.t_last) {
uint64_t i_msgs = cnt.msgs - cnt.msgs_last;
uint64_t i_bytes = cnt.bytes - cnt.bytes_last;
uint64_t i_time = cnt.t_last ? now - cnt.t_last : 0;
printf("%% INTERVAL: %" PRIu64
" messages "
"(%" PRIu64
" bytes) "
"consumed in %" PRIu64 "ms: %" PRIu64
" msgs/s "
"(%.02f MB/s)"
"%s\n",
i_msgs, i_bytes, i_time / 1000,
((i_msgs * 1000000) / i_time),
(float)((i_bytes) / (float)i_time), extra);
}
}
cnt.t_last = now;
cnt.msgs_last = cnt.msgs;
cnt.bytes_last = cnt.bytes;
}
static void sig_usr1(int sig) {
rd_kafka_dump(stdout, global_rk);
}
/**
* @brief Read config from file
* @returns -1 on error, else 0.
*/
static int read_conf_file(rd_kafka_conf_t *conf, const char *path) {
FILE *fp;
char buf[512];
int line = 0;
char errstr[512];
if (!(fp = fopen(path, "r"))) {
fprintf(stderr, "%% Failed to open %s: %s\n", path,
strerror(errno));
return -1;
}
while (fgets(buf, sizeof(buf), fp)) {
char *s = buf;
char *t;
rd_kafka_conf_res_t r = RD_KAFKA_CONF_UNKNOWN;
line++;
while (isspace((int)*s))
s++;
if (!*s || *s == '#')
continue;
if ((t = strchr(buf, '\n')))
*t = '\0';
t = strchr(buf, '=');
if (!t || t == s || !*(t + 1)) {
fprintf(stderr, "%% %s:%d: expected key=value\n", path,
line);
fclose(fp);
return -1;
}
*(t++) = '\0';
/* Try global config */
r = rd_kafka_conf_set(conf, s, t, errstr, sizeof(errstr));
if (r == RD_KAFKA_CONF_OK)
continue;
fprintf(stderr, "%% %s:%d: %s=%s: %s\n", path, line, s, t,
errstr);
fclose(fp);
return -1;
}
fclose(fp);
return 0;
}
static rd_kafka_resp_err_t do_produce(rd_kafka_t *rk,
rd_kafka_topic_t *rkt,
int32_t partition,
int msgflags,
void *payload,
size_t size,
const void *key,
size_t key_size,
const rd_kafka_headers_t *hdrs) {
/* Send/Produce message. */
if (hdrs) {
rd_kafka_headers_t *hdrs_copy;
rd_kafka_resp_err_t err;
hdrs_copy = rd_kafka_headers_copy(hdrs);
err = rd_kafka_producev(
rk, RD_KAFKA_V_RKT(rkt), RD_KAFKA_V_PARTITION(partition),
RD_KAFKA_V_MSGFLAGS(msgflags),
RD_KAFKA_V_VALUE(payload, size),
RD_KAFKA_V_KEY(key, key_size),
RD_KAFKA_V_HEADERS(hdrs_copy), RD_KAFKA_V_END);
if (err)
rd_kafka_headers_destroy(hdrs_copy);
return err;
} else {
if (rd_kafka_produce(rkt, partition, msgflags, payload, size,
key, key_size, NULL) == -1)
return rd_kafka_last_error();
}
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
/**
* @brief Sleep for \p sleep_us microseconds.
*/
static void do_sleep(int sleep_us) {
if (sleep_us > 100) {
#ifdef _WIN32
Sleep(sleep_us / 1000);
#else
usleep(sleep_us);
#endif
} else {
rd_ts_t next = rd_clock() + (rd_ts_t)sleep_us;
while (next > rd_clock())
;
}
}
int main(int argc, char **argv) {
char *brokers = NULL;
char mode = 'C';
char *topic = NULL;
const char *key = NULL;
int *partitions = NULL;
int opt;
int sendflags = 0;
char *msgpattern = "librdkafka_performance testing!";
int msgsize = -1;
const char *debug = NULL;
int do_conf_dump = 0;
rd_ts_t now;
char errstr[512];
uint64_t seq = 0;
int seed = (int)time(NULL);
rd_kafka_t *rk;
rd_kafka_topic_t *rkt;
rd_kafka_conf_t *conf;
rd_kafka_queue_t *rkqu = NULL;
const char *compression = "no";
int64_t start_offset = 0;
int batch_size = 0;
int idle = 0;
const char *stats_cmd = NULL;
char *stats_intvlstr = NULL;
char tmp[128];
char *tmp2;
int otype = _OTYPE_SUMMARY;
double dtmp;
int rate_sleep = 0;
rd_kafka_topic_partition_list_t *topics;
int exitcode = 0;
rd_kafka_headers_t *hdrs = NULL;
rd_kafka_resp_err_t err;
/* Kafka configuration */
conf = rd_kafka_conf_new();
rd_kafka_conf_set_error_cb(conf, err_cb);
rd_kafka_conf_set_throttle_cb(conf, throttle_cb);
rd_kafka_conf_set_offset_commit_cb(conf, offset_commit_cb);
#ifdef SIGIO
/* Quick termination */
rd_snprintf(tmp, sizeof(tmp), "%i", SIGIO);
rd_kafka_conf_set(conf, "internal.termination.signal", tmp, NULL, 0);
#endif
/* Producer config */
rd_kafka_conf_set(conf, "linger.ms", "1000", NULL, 0);
rd_kafka_conf_set(conf, "message.send.max.retries", "3", NULL, 0);
rd_kafka_conf_set(conf, "retry.backoff.ms", "500", NULL, 0);
/* Consumer config */
/* Tell rdkafka to (try to) maintain 1M messages
* in its internal receive buffers. This is to avoid
* application -> rdkafka -> broker per-message ping-pong
* latency.
* The larger the local queue, the higher the performance.
* Try other values with: ... -X queued.min.messages=1000
*/
rd_kafka_conf_set(conf, "queued.min.messages", "1000000", NULL, 0);
rd_kafka_conf_set(conf, "session.timeout.ms", "6000", NULL, 0);
rd_kafka_conf_set(conf, "auto.offset.reset", "earliest", NULL, 0);
topics = rd_kafka_topic_partition_list_new(1);
while ((opt = getopt(argc, argv,
"PCG:t:p:b:s:k:c:fi:MDd:m:S:x:"
"R:a:z:o:X:B:eT:Y:qvIur:lA:OwNH:")) != -1) {
switch (opt) {
case 'G':
if (rd_kafka_conf_set(conf, "group.id", optarg, errstr,
sizeof(errstr)) !=
RD_KAFKA_CONF_OK) {
fprintf(stderr, "%% %s\n", errstr);
exit(1);
}
/* FALLTHRU */
case 'P':
case 'C':
mode = opt;
break;
case 't':
rd_kafka_topic_partition_list_add(
topics, optarg, RD_KAFKA_PARTITION_UA);
break;
case 'p':
partition_cnt++;
partitions = realloc(partitions, sizeof(*partitions) *
partition_cnt);
partitions[partition_cnt - 1] = atoi(optarg);
break;
case 'b':
brokers = optarg;
break;
case 's':
msgsize = atoi(optarg);
break;
case 'k':
key = optarg;
break;
case 'c':
msgcnt = atoi(optarg);
break;
case 'D':
sendflags |= RD_KAFKA_MSG_F_FREE;
break;
case 'i':
dispintvl = atoi(optarg);
break;
case 'm':
msgpattern = optarg;
break;
case 'S':
seq = strtoull(optarg, NULL, 10);
do_seq = 1;
break;
case 'x':
exit_after = atoi(optarg);
break;
case 'R':
seed = atoi(optarg);
break;
case 'a':
if (rd_kafka_conf_set(conf, "acks", optarg, errstr,
sizeof(errstr)) !=
RD_KAFKA_CONF_OK) {
fprintf(stderr, "%% %s\n", errstr);
exit(1);
}
break;
case 'B':
batch_size = atoi(optarg);
break;
case 'z':
if (rd_kafka_conf_set(conf, "compression.codec", optarg,
errstr, sizeof(errstr)) !=
RD_KAFKA_CONF_OK) {
fprintf(stderr, "%% %s\n", errstr);
exit(1);
}
compression = optarg;
break;
case 'o':
if (!strcmp(optarg, "end"))
start_offset = RD_KAFKA_OFFSET_END;
else if (!strcmp(optarg, "beginning"))
start_offset = RD_KAFKA_OFFSET_BEGINNING;
else if (!strcmp(optarg, "stored"))
start_offset = RD_KAFKA_OFFSET_STORED;
else {
start_offset = strtoll(optarg, NULL, 10);
if (start_offset < 0)
start_offset =
RD_KAFKA_OFFSET_TAIL(-start_offset);
}
break;
case 'e':
exit_eof = 1;
break;
case 'd':
debug = optarg;
break;
case 'H':
if (!strcmp(optarg, "parse"))
read_hdrs = 1;
else {
char *name, *val;
size_t name_sz = -1;