forked from edenhill/kcat
-
Notifications
You must be signed in to change notification settings - Fork 0
/
kafkacat.c
1135 lines (953 loc) · 41.1 KB
/
kafkacat.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
/*
* kafkacat - Apache Kafka consumer and producer
*
* Copyright (c) 2014, Magnus Edenhill
* 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.
*/
#include <unistd.h>
#include <stdio.h>
#include <stdlib.h>
#include <stdarg.h>
#include <signal.h>
#include <syslog.h>
#include <sys/types.h>
#include <sys/stat.h>
#include <fcntl.h>
#include <sys/mman.h>
#include "kafkacat.h"
struct conf conf = {
.run = 1,
.verbosity = 1,
.partition = RD_KAFKA_PARTITION_UA,
.msg_size = 1024*1024,
.null_str = "NULL",
};
static struct stats {
uint64_t tx;
uint64_t tx_err_q;
uint64_t tx_err_dr;
uint64_t tx_delivered;
uint64_t rx;
} stats;
/* Partition's at EOF state array */
int *part_eof = NULL;
/* Number of partitions that has reached EOF */
int part_eof_cnt = 0;
/* Threshold level (partitions at EOF) before exiting */
int part_eof_thres = 0;
/**
* Fatal error: print error and exit
*/
void __attribute__((noreturn)) fatal0 (const char *func, int line,
const char *fmt, ...) {
va_list ap;
char buf[1024];
va_start(ap, fmt);
vsnprintf(buf, sizeof(buf), fmt, ap);
va_end(ap);
INFO(2, "Fatal error at %s:%i:\n", func, line);
fprintf(stderr, "%% ERROR: %s\n", buf);
exit(1);
}
/**
* The delivery report callback is called once per message to
* report delivery success or failure.
*/
static void dr_msg_cb (rd_kafka_t *rk, const rd_kafka_message_t *rkmessage,
void *opaque) {
static int say_once = 1;
if (rkmessage->err) {
INFO(1, "Delivery failed for message: %s\n",
rd_kafka_err2str(rkmessage->err));
stats.tx_err_dr++;
return;
}
INFO(3, "Message delivered to partition %"PRId32" (offset %"PRId64")\n",
rkmessage->partition, rkmessage->offset);
if (rkmessage->offset == 0 && say_once) {
INFO(3, "Enable message offset reporting "
"with '-X topic.produce.offset.report=true'\n");
say_once = 0;
}
stats.tx_delivered++;
}
/**
* Produces a single message, retries on queue congestion, and
* exits hard on error.
*/
static void produce (void *buf, size_t len,
const void *key, size_t key_len, int msgflags) {
/* Produce message: keep trying until it succeeds. */
do {
rd_kafka_resp_err_t err;
if (!conf.run)
FATAL("Program terminated while "
"producing message of %zd bytes", len);
if (rd_kafka_produce(conf.rkt, conf.partition, msgflags,
buf, len, key, key_len, NULL) != -1) {
stats.tx++;
break;
}
err = rd_kafka_errno2err(errno);
if (err != RD_KAFKA_RESP_ERR__QUEUE_FULL)
FATAL("Failed to produce message (%zd bytes): %s",
len, rd_kafka_err2str(err));
stats.tx_err_q++;
/* Internal queue full, sleep to allow
* messages to be produced/time out
* before trying again. */
rd_kafka_poll(conf.rk, 5);
} while (1);
/* Poll for delivery reports, errors, etc. */
rd_kafka_poll(conf.rk, 0);
}
/**
* Produce contents of file as a single message.
* Returns the file length on success, else -1.
*/
static ssize_t produce_file (const char *path) {
int fd;
void *ptr;
struct stat st;
if ((fd = open(path, O_RDONLY)) == -1) {
INFO(1, "Failed to open %s: %s\n", path, strerror(errno));
return -1;
}
if (fstat(fd, &st) == -1) {
INFO(1, "Failed to stat %s: %s\n", path, strerror(errno));
close(fd);
return -1;
}
if (st.st_size == 0) {
INFO(3, "Skipping empty file %s\n", path);
close(fd);
return 0;
}
ptr = mmap(NULL, st.st_size, PROT_READ, MAP_PRIVATE, fd, 0);
if (ptr == MAP_FAILED) {
INFO(1, "Failed to mmap %s: %s\n", path, strerror(errno));
close(fd);
return -1;
}
INFO(4, "Producing file %s (%"PRIdMAX" bytes)\n",
path, (intmax_t)st.st_size);
produce(ptr, st.st_size, NULL, 0, RD_KAFKA_MSG_F_COPY);
munmap(ptr, st.st_size);
return st.st_size;
}
/**
* Run producer, reading messages from 'fp' and producing to kafka.
* Or if 'pathcnt' is > 0, read messages from files in 'paths' instead.
*/
static void producer_run (FILE *fp, char **paths, int pathcnt) {
char *sbuf = NULL;
size_t size = 0;
ssize_t len;
char errstr[512];
/* Assign per-message delivery report callback. */
rd_kafka_conf_set_dr_msg_cb(conf.rk_conf, dr_msg_cb);
/* Create producer */
if (!(conf.rk = rd_kafka_new(RD_KAFKA_PRODUCER, conf.rk_conf,
errstr, sizeof(errstr))))
FATAL("Failed to create producer: %s", errstr);
if (conf.debug)
rd_kafka_set_log_level(conf.rk, LOG_DEBUG);
else if (conf.verbosity == 0)
rd_kafka_set_log_level(conf.rk, 0);
/* Create topic */
if (!(conf.rkt = rd_kafka_topic_new(conf.rk, conf.topic,
conf.rkt_conf)))
FATAL("Failed to create topic %s: %s", conf.topic,
rd_kafka_err2str(rd_kafka_errno2err(errno)));
conf.rk_conf = NULL;
conf.rkt_conf = NULL;
if (pathcnt > 0 && !(conf.flags & CONF_F_LINE)) {
int i;
int good = 0;
/* Read messages from files, each file is its own message. */
for (i = 0 ; i < pathcnt ; i++)
if (produce_file(paths[i]) != -1)
good++;
if (!good)
conf.exitcode = 1;
else if (good < pathcnt)
INFO(1, "Failed to produce from %i/%i files\n",
pathcnt - good, pathcnt);
} else {
/* Read messages from input, delimited by conf.delim */
while (conf.run &&
(len = getdelim(&sbuf, &size, conf.delim, fp)) != -1) {
int msgflags = 0;
char *buf = sbuf;
char *key = NULL;
size_t key_len = 0;
size_t orig_len = len;
if (len == 0)
continue;
/* Shave off delimiter */
if ((int)buf[len-1] == conf.delim)
len--;
if (len == 0)
continue;
/* Extract key, if desired and found. */
if (conf.flags & CONF_F_KEY_DELIM) {
char *t;
if ((t = memchr(buf, conf.key_delim, len))) {
key_len = (size_t)(t-sbuf);
key = buf;
buf += key_len+1;
len -= key_len+1;
/* Since buf has been forwarded
* from its initial allocation point
* we must make sure we dont tell
* librdkafka to free it (since the
* address would be wrong). */
msgflags |= RD_KAFKA_MSG_F_COPY;
if (conf.flags & CONF_F_NULL) {
if (len == 0)
buf = NULL;
if (key_len == 0)
key = NULL;
}
}
}
if (!(msgflags & RD_KAFKA_MSG_F_COPY) &&
len > 1024 && !(conf.flags & CONF_F_TEE)) {
/* If message is larger than this arbitrary
* threshold it will be more effective to
* not copy the data but let rdkafka own it
* instead.
*
* Note that CONF_T_TEE must be checked,
* otherwise a possible race might occur.
* */
msgflags |= RD_KAFKA_MSG_F_FREE;
} else {
/* For smaller messages a copy is
* more efficient. */
msgflags |= RD_KAFKA_MSG_F_COPY;
}
/* Produce message */
produce(buf, len, key, key_len, msgflags);
if (conf.flags & CONF_F_TEE &&
fwrite(sbuf, orig_len, 1, stdout) != 1)
FATAL("Tee write error for message of %zd bytes: %s",
orig_len, strerror(errno));
if (msgflags & RD_KAFKA_MSG_F_FREE) {
/* rdkafka owns the allocated buffer
* memory now. */
sbuf = NULL;
size = 0;
}
/* Enforce -c <cnt> */
if (stats.tx == conf.msg_cnt)
conf.run = 0;
}
if (conf.run) {
if (!feof(fp))
FATAL("Unable to read message: %s",
strerror(errno));
}
}
/* Wait for all messages to be transmitted */
conf.run = 1;
while (conf.run && rd_kafka_outq_len(conf.rk))
rd_kafka_poll(conf.rk, 50);
rd_kafka_topic_destroy(conf.rkt);
rd_kafka_destroy(conf.rk);
if (sbuf)
free(sbuf);
if (stats.tx_err_q || stats.tx_err_dr)
conf.exitcode = 1;
}
/**
* Consume callback, called for each message consumed.
*/
static void consume_cb (rd_kafka_message_t *rkmessage, void *opaque) {
FILE *fp = opaque;
if (!conf.run)
return;
if (rkmessage->err) {
if (rkmessage->err == RD_KAFKA_RESP_ERR__PARTITION_EOF) {
/* Store EOF offset.
* If partition is empty and at offset 0,
* store future first message (0). */
rd_kafka_offset_store(rkmessage->rkt,
rkmessage->partition,
rkmessage->offset == 0 ?
0 : rkmessage->offset-1);
if (conf.exit_eof) {
if (!part_eof[rkmessage->partition]) {
/* Stop consuming this partition */
rd_kafka_consume_stop(rkmessage->rkt,
rkmessage->partition);
part_eof[rkmessage->partition] = 1;
part_eof_cnt++;
if (part_eof_cnt >= part_eof_thres)
conf.run = 0;
}
INFO(1, "Reached end of topic %s [%"PRId32"] "
"at offset %"PRId64"%s\n",
rd_kafka_topic_name(rkmessage->rkt),
rkmessage->partition,
rkmessage->offset,
!conf.run ? ": exiting" : "");
}
return;
}
FATAL("Topic %s [%"PRId32"] error: %s",
rd_kafka_topic_name(rkmessage->rkt),
rkmessage->partition,
rd_kafka_message_errstr(rkmessage));
}
/* Print message */
fmt_msg_output(fp, rkmessage);
rd_kafka_offset_store(rkmessage->rkt,
rkmessage->partition,
rkmessage->offset);
if (++stats.rx == conf.msg_cnt)
conf.run = 0;
}
/**
* Run consumer, consuming messages from Kafka and writing to 'fp'.
*/
static void consumer_run (FILE *fp) {
char errstr[512];
rd_kafka_resp_err_t err;
const rd_kafka_metadata_t *metadata;
int i;
rd_kafka_queue_t *rkqu;
/* Create consumer */
if (!(conf.rk = rd_kafka_new(RD_KAFKA_CONSUMER, conf.rk_conf,
errstr, sizeof(errstr))))
FATAL("Failed to create producer: %s", errstr);
if (conf.debug)
rd_kafka_set_log_level(conf.rk, LOG_DEBUG);
else if (conf.verbosity == 0)
rd_kafka_set_log_level(conf.rk, 0);
/* The callback-based consumer API's offset store granularity is
* not good enough for us, disable automatic offset store
* and do it explicitly per-message in the consume callback instead. */
if (rd_kafka_topic_conf_set(conf.rkt_conf,
"auto.commit.enable", "false",
errstr, sizeof(errstr)) != RD_KAFKA_CONF_OK)
FATAL("%s", errstr);
/* Create topic */
if (!(conf.rkt = rd_kafka_topic_new(conf.rk, conf.topic,
conf.rkt_conf)))
FATAL("Failed to create topic %s: %s", conf.topic,
rd_kafka_err2str(rd_kafka_errno2err(errno)));
conf.rk_conf = NULL;
conf.rkt_conf = NULL;
/* Query broker for topic + partition information. */
if ((err = rd_kafka_metadata(conf.rk, 0, conf.rkt, &metadata, 5000)))
FATAL("Failed to query metadata for topic %s: %s",
rd_kafka_topic_name(conf.rkt), rd_kafka_err2str(err));
/* Error handling */
if (metadata->topic_cnt == 0)
FATAL("No such topic in cluster: %s",
rd_kafka_topic_name(conf.rkt));
if ((err = metadata->topics[0].err))
FATAL("Topic %s error: %s",
rd_kafka_topic_name(conf.rkt), rd_kafka_err2str(err));
if (metadata->topics[0].partition_cnt == 0)
FATAL("Topic %s has no partitions",
rd_kafka_topic_name(conf.rkt));
/* If Exit-at-EOF is enabled, set up array to track EOF
* state for each partition. */
if (conf.exit_eof) {
part_eof = calloc(sizeof(*part_eof),
metadata->topics[0].partition_cnt);
if (conf.partition != RD_KAFKA_PARTITION_UA)
part_eof_thres = 1;
else
part_eof_thres = metadata->topics[0].partition_cnt;
}
/* Create a shared queue that combines messages from
* all wanted partitions. */
rkqu = rd_kafka_queue_new(conf.rk);
/* Start consuming from all wanted partitions. */
for (i = 0 ; i < metadata->topics[0].partition_cnt ; i++) {
int32_t partition = metadata->topics[0].partitions[i].id;
/* If -p <part> was specified: skip unwanted partitions */
if (conf.partition != RD_KAFKA_PARTITION_UA &&
conf.partition != partition)
continue;
/* Start consumer for this partition */
if (rd_kafka_consume_start_queue(conf.rkt, partition,
conf.offset, rkqu) == -1)
FATAL("Failed to start consuming "
"topic %s [%"PRId32"]: %s",
conf.topic, partition,
rd_kafka_err2str(rd_kafka_errno2err(errno)));
if (conf.partition != RD_KAFKA_PARTITION_UA)
break;
}
if (conf.partition != RD_KAFKA_PARTITION_UA &&
i == metadata->topics[0].partition_cnt)
FATAL("Topic %s (with partitions 0..%i): "
"partition %i does not exist",
rd_kafka_topic_name(conf.rkt),
metadata->topics[0].partition_cnt-1,
conf.partition);
/* Read messages from Kafka, write to 'fp'. */
while (conf.run) {
rd_kafka_consume_callback_queue(rkqu, 100,
consume_cb, fp);
/* Poll for errors, etc */
rd_kafka_poll(conf.rk, 0);
}
/* Stop consuming */
for (i = 0 ; i < metadata->topics[0].partition_cnt ; i++) {
int32_t partition = metadata->topics[0].partitions[i].id;
/* If -p <part> was specified: skip unwanted partitions */
if (conf.partition != RD_KAFKA_PARTITION_UA &&
conf.partition != partition)
continue;
/* Dont stop already stopped partitions */
if (!part_eof || !part_eof[partition])
rd_kafka_consume_stop(conf.rkt, partition);
rd_kafka_consume_stop(conf.rkt, partition);
}
/* Destroy shared queue */
rd_kafka_queue_destroy(rkqu);
/* Wait for outstanding requests to finish. */
conf.run = 1;
while (conf.run && rd_kafka_outq_len(conf.rk) > 0)
rd_kafka_poll(conf.rk, 50);
rd_kafka_topic_destroy(conf.rkt);
rd_kafka_destroy(conf.rk);
}
/**
* Print metadata information
*/
static void metadata_print (const rd_kafka_metadata_t *metadata) {
int i, j, k;
printf("Metadata for %s (from broker %"PRId32": %s):\n",
conf.topic ? : "all topics",
metadata->orig_broker_id, metadata->orig_broker_name);
/* Iterate brokers */
printf(" %i brokers:\n", metadata->broker_cnt);
for (i = 0 ; i < metadata->broker_cnt ; i++)
printf(" broker %"PRId32" at %s:%i\n",
metadata->brokers[i].id,
metadata->brokers[i].host,
metadata->brokers[i].port);
/* Iterate topics */
printf(" %i topics:\n", metadata->topic_cnt);
for (i = 0 ; i < metadata->topic_cnt ; i++) {
const rd_kafka_metadata_topic_t *t = &metadata->topics[i];
printf(" topic \"%s\" with %i partitions:",
t->topic,
t->partition_cnt);
if (t->err) {
printf(" %s", rd_kafka_err2str(t->err));
if (t->err == RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE)
printf(" (try again)");
}
printf("\n");
/* Iterate topic's partitions */
for (j = 0 ; j < t->partition_cnt ; j++) {
const rd_kafka_metadata_partition_t *p;
p = &t->partitions[j];
printf(" partition %"PRId32", "
"leader %"PRId32", replicas: ",
p->id, p->leader);
/* Iterate partition's replicas */
for (k = 0 ; k < p->replica_cnt ; k++)
printf("%s%"PRId32,
k > 0 ? ",":"", p->replicas[k]);
/* Iterate partition's ISRs */
printf(", isrs: ");
for (k = 0 ; k < p->isr_cnt ; k++)
printf("%s%"PRId32,
k > 0 ? ",":"", p->isrs[k]);
if (p->err)
printf(", %s\n", rd_kafka_err2str(p->err));
else
printf("\n");
}
}
}
/**
* Lists metadata
*/
static void metadata_list (void) {
char errstr[512];
rd_kafka_resp_err_t err;
const rd_kafka_metadata_t *metadata;
/* Create handle */
if (!(conf.rk = rd_kafka_new(RD_KAFKA_PRODUCER, conf.rk_conf,
errstr, sizeof(errstr))))
FATAL("Failed to create producer: %s", errstr);
rd_kafka_set_logger(conf.rk, rd_kafka_log_print);
if (conf.debug)
rd_kafka_set_log_level(conf.rk, LOG_DEBUG);
else if (conf.verbosity == 0)
rd_kafka_set_log_level(conf.rk, 0);
/* Create topic, if specified */
if (conf.topic &&
!(conf.rkt = rd_kafka_topic_new(conf.rk, conf.topic,
conf.rkt_conf)))
FATAL("Failed to create topic %s: %s", conf.topic,
rd_kafka_err2str(rd_kafka_errno2err(errno)));
conf.rk_conf = NULL;
conf.rkt_conf = NULL;
/* Fetch metadata */
err = rd_kafka_metadata(conf.rk, conf.rkt ? 0 : 1, conf.rkt,
&metadata, 5000);
if (err != RD_KAFKA_RESP_ERR_NO_ERROR)
FATAL("Failed to acquire metadata: %s", rd_kafka_err2str(err));
/* Print metadata */
#if ENABLE_JSON
if (conf.flags & CONF_F_FMT_JSON)
metadata_print_json(metadata);
else
#endif
metadata_print(metadata);
rd_kafka_metadata_destroy(metadata);
if (conf.rkt)
rd_kafka_topic_destroy(conf.rkt);
rd_kafka_destroy(conf.rk);
}
/**
* Print usage and exit.
*/
static void __attribute__((noreturn)) usage (const char *argv0, int exitcode,
const char *reason) {
if (reason)
printf("Error: %s\n\n", reason);
printf("Usage: %s <options> [file1 file2 ..]\n"
"kafkacat - Apache Kafka producer and consumer tool\n"
"https://github.com/edenhill/kafkacat\n"
"Copyright (c) 2014-2015, Magnus Edenhill\n"
"Version %s%s (librdkafka %s)\n"
"\n"
"\n"
"General options:\n"
" -C | -P | -L Mode: Consume, Produce or metadata List\n"
" -t <topic> Topic to consume from, produce to, "
"or list\n"
" -p <partition> Partition\n"
" -b <brokers,..> Bootstrap broker(s) (host[:port])\n"
" -D <delim> Message delimiter character:\n"
" a-z.. | \\r | \\n | \\t | \\xNN\n"
" Default: \\n\n"
" -K <delim> Key delimiter (same format as -D)\n"
" -c <cnt> Limit message count\n"
" -X list List available librdkafka configuration "
"properties\n"
" -X prop=val Set librdkafka configuration property.\n"
" Properties prefixed with \"topic.\" are\n"
" applied as topic properties.\n"
" -X dump Dump configuration and exit.\n"
" -d <dbg1,...> Enable librdkafka debugging:\n"
" " RD_KAFKA_DEBUG_CONTEXTS "\n"
" -q Be quiet (verbosity set to 0)\n"
" -v Increase verbosity\n"
"\n"
"Producer options:\n"
" -z snappy|gzip Message compression. Default: none\n"
" -p -1 Use random partitioner\n"
" -D <delim> Delimiter to split input into messages\n"
" -K <delim> Delimiter to split input key and message\n"
" -l Send messages from a file separated by\n"
" delimiter, as with stdin.\n"
" (only one file allowed)\n"
" -T Output sent messages to stdout, acting like tee.\n"
" -c <cnt> Exit after producing this number "
"of messages\n"
" -Z Send empty messages as NULL messages\n"
" file1 file2.. Read messages from files.\n"
" With -l, only one file permitted.\n"
" Otherwise, the entire file contents will\n"
" be sent as one single message.\n"
"\n"
"Consumer options:\n"
" -o <offset> Offset to start consuming from:\n"
" beginning | end | stored |\n"
" <value> (absolute offset) |\n"
" -<value> (relative offset from end)\n"
" -e Exit successfully when last message "
"received\n"
" -f <fmt..> Output formatting string, see below.\n"
" Takes precedence over -D and -K.\n"
#if ENABLE_JSON
" -J Output with JSON envelope\n"
#endif
" -D <delim> Delimiter to separate messages on output\n"
" -K <delim> Print message keys prefixing the message\n"
" with specified delimiter.\n"
" -O Print message offset using -K delimiter\n"
" -c <cnt> Exit after consuming this number "
"of messages\n"
" -Z Print NULL messages and keys as \"%s\""
"(instead of empty)\n"
" -u Unbuffered output\n"
"\n"
"Metadata options:\n"
" -t <topic> Topic to query (optional)\n"
"\n"
"\n"
"Format string tokens:\n"
" %%s Message payload\n"
" %%S Message payload length (or -1 for NULL)\n"
" %%k Message key\n"
" %%K Message key length (or -1 for NULL)\n"
" %%t Topic\n"
" %%p Partition\n"
" %%o Message offset\n"
" \\n \\r \\t Newlines, tab\n"
" \\xXX \\xNNN Any ASCII character\n"
" Example:\n"
" -f 'Topic %%t [%%p] at offset %%o: key %%k: %%s\\n'\n"
"\n"
"\n"
"Consumer mode (writes messages to stdout):\n"
" kafkacat -b <broker> -t <topic> -p <partition>\n"
" or:\n"
" kafkacat -C -b ...\n"
"\n"
"Producer mode (reads messages from stdin):\n"
" ... | kafkacat -b <broker> -t <topic> -p <partition>\n"
" or:\n"
" kafkacat -P -b ...\n"
"\n"
"Metadata listing:\n"
" kafkacat -L -b <broker> [-t <topic>]\n"
"\n",
argv0, KAFKACAT_VERSION,
#if ENABLE_JSON
" (JSON)",
#else
"",
#endif
rd_kafka_version_str(),
conf.null_str
);
exit(exitcode);
}
/**
* Terminate by putting out the run flag.
*/
static void term (int sig) {
conf.run = 0;
}
/**
* librdkafka error callback
*/
static void error_cb (rd_kafka_t *rk, int err,
const char *reason, void *opaque) {
if (err == RD_KAFKA_RESP_ERR__ALL_BROKERS_DOWN)
FATAL("%s: %s: terminating", rd_kafka_err2str(err),
reason ? reason : "");
INFO(1, "ERROR: %s: %s\n", rd_kafka_err2str(err),
reason ? reason : "");
}
/**
* Parse delimiter string from command line arguments.
*/
static int parse_delim (const char *str) {
int delim;
if (!strncmp(str, "\\x", strlen("\\x")))
delim = strtoul(str+strlen("\\x"), NULL, 16) & 0xff;
else if (!strcmp(str, "\\n"))
delim = (int)'\n';
else if (!strcmp(str, "\\t"))
delim = (int)'\t';
else
delim = (int)*str & 0xff;
return delim;
}
/**
* Parse command line arguments
*/
static void argparse (int argc, char **argv) {
char errstr[512];
int opt;
const char *fmt = NULL;
const char *delim = "\n";
const char *key_delim = NULL;
char tmp_fmt[64];
while ((opt = getopt(argc, argv,
"PCLt:p:b:z:o:eD:K:Od:qvX:c:Tuf:Zl"
#if ENABLE_JSON
"J"
#endif
)) != -1) {
switch (opt) {
case 'P':
case 'C':
case 'L':
conf.mode = opt;
break;
case 't':
conf.topic = optarg;
break;
case 'p':
conf.partition = atoi(optarg);
break;
case 'b':
conf.brokers = optarg;
break;
case 'z':
if (rd_kafka_conf_set(conf.rk_conf,
"compression.codec", optarg,
errstr, sizeof(errstr)) !=
RD_KAFKA_CONF_OK)
FATAL("%s", errstr);
break;
case 'o':
if (!strcmp(optarg, "end"))
conf.offset = RD_KAFKA_OFFSET_END;
else if (!strcmp(optarg, "beginning"))
conf.offset = RD_KAFKA_OFFSET_BEGINNING;
else if (!strcmp(optarg, "stored"))
conf.offset = RD_KAFKA_OFFSET_STORED;
else {
conf.offset = strtoll(optarg, NULL, 10);
if (conf.offset < 0)
conf.offset = RD_KAFKA_OFFSET_TAIL(-conf.offset);
}
break;
case 'e':
conf.exit_eof = 1;
break;
case 'f':
fmt = optarg;
break;
#if ENABLE_JSON
case 'J':
conf.flags |= CONF_F_FMT_JSON;
break;
#endif
case 'D':
delim = optarg;
break;
case 'K':
key_delim = optarg;
conf.flags |= CONF_F_KEY_DELIM;
break;
case 'l':
conf.flags |= CONF_F_LINE;
break;
case 'O':
conf.flags |= CONF_F_OFFSET;
break;
case 'c':
conf.msg_cnt = strtoll(optarg, NULL, 10);
break;
case 'Z':
conf.flags |= CONF_F_NULL;
conf.null_str_len = strlen(conf.null_str);
break;
case 'd':
conf.debug = optarg;
if (rd_kafka_conf_set(conf.rk_conf, "debug", conf.debug,
errstr, sizeof(errstr)) !=
RD_KAFKA_CONF_OK)
FATAL("%s", errstr);
break;
case 'q':
conf.verbosity = 0;
break;
case 'v':
conf.verbosity++;
break;
case 'T':
conf.flags |= CONF_F_TEE;
break;
case 'u':
setbuf(stdout, NULL);
break;
case 'X':
{
char *name, *val;
rd_kafka_conf_res_t res;
if (!strcmp(optarg, "list") ||
!strcmp(optarg, "help")) {
rd_kafka_conf_properties_show(stdout);
exit(0);
}
if (!strcmp(optarg, "dump")) {
conf.conf_dump = 1;
continue;
}
name = optarg;
if (!(val = strchr(name, '='))) {
fprintf(stderr, "%% Expected "
"-X property=value, not %s, "
"use -X list to display available "
"properties\n", name);
exit(1);
}
*val = '\0';
val++;
res = RD_KAFKA_CONF_UNKNOWN;
/* Try "topic." prefixed properties on topic
* conf first, and then fall through to global if
* it didnt match a topic configuration property. */
if (!strncmp(name, "topic.", strlen("topic.")))
res = rd_kafka_topic_conf_set(conf.rkt_conf,
name+
strlen("topic."),
val,
errstr,
sizeof(errstr));
if (res == RD_KAFKA_CONF_UNKNOWN)
res = rd_kafka_conf_set(conf.rk_conf, name, val,
errstr, sizeof(errstr));
if (res != RD_KAFKA_CONF_OK)
FATAL("%s", errstr);
}
break;
default:
usage(argv[0], 1, "unknown argument");
break;
}
}
if (!conf.brokers)
usage(argv[0], 1, "-b <broker,..> missing");
/* Decide mode if not specified */
if (!conf.mode) {
if (isatty(STDIN_FILENO))
conf.mode = 'C';
else
conf.mode = 'P';
INFO(1, "Auto-selecting %s mode (use -P or -C to override)\n",
conf.mode == 'C' ? "Consumer":"Producer");
}
if (conf.mode != 'L' && !conf.topic)
usage(argv[0], 1, "-t <topic> missing");
if (rd_kafka_conf_set(conf.rk_conf, "metadata.broker.list",
conf.brokers, errstr, sizeof(errstr)) !=
RD_KAFKA_CONF_OK)
usage(argv[0], 1, errstr);