forked from confluentinc/librdkafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
rdkafka.c
1334 lines (1055 loc) · 30.5 KB
/
rdkafka.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, 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.
*/
/**
* Apache Kafka consumer & producer implementation.
* This implementation used to
* reside in librd (https://github.com/edenhill/librd)
* but was moved here to provide a purely stand-alone library.
* Thus the rd-prefix.
*/
#include <sys/socket.h>
#include <sys/types.h>
#include <limits.h>
#include <arpa/inet.h>
#include <netdb.h>
#include <errno.h>
#include <string.h>
#include <stdarg.h>
#include <syslog.h>
#include <pthread.h>
#include <poll.h>
#define __need_IOV_MAX
#include <stdio.h>
#include <sys/socket.h>
#ifndef _BSD_SOURCE
#define _BSD_SOURCE
#endif
#include <endian.h>
#include <zlib.h>
#define NEED_RD_KAFKAPROTO_DEF
#include "rdkafka.h"
#ifndef WITH_LIBRD
#include "rdcrc32.h"
#include "rdgz.h"
#include "rdfile.h"
#include "rdtime.h"
#else
#include <librd/rdcrc32.h>
#include <librd/rdgz.h>
#include <librd/rdfile.h>
#include <librd/rdtime.h>
#endif
/**
* librd usage:
* trivial:
* -- RD_SIZE_*
* * rd_atomic
* -- rd_mutex
* -- rd_cond
* * rd_clock
* * RD_POLL
* * rd_ts
* -- rd_cond_timedwait
* -- rd_memdup
* * rd_file_mode
* - -rd_thread_create
*
* moderate:
* * rd_sockaddr_list*
* -- rd_tsprintf
* -- rd_io_poll_single
* * rd_gz_decompress
*
*
* critical:
*
*/
/**
* Default configuration.
* Use a copy of this struct as the base of your own configuration.
* See rdkafka.h for more information.
*/
const rd_kafka_conf_t rd_kafka_defaultconf = {
consumer: {
poll_interval: 1000 /* 1s */,
replyq_low_thres: 1,
max_size: 500000,
},
max_msg_size: 4000000,
};
static void (*rd_kafka_log_cb) (const rd_kafka_t *rk, int level,
const char *fac,
const char *buf) = rd_kafka_log_print;
static int rd_kafka_recv (rd_kafka_t *rk);
static void rd_kafka_op_reply (rd_kafka_t *rk,
rd_kafka_op_type_t type,
rd_kafka_resp_err_t err, uint8_t compression,
void *payload, int len,
uint64_t offset_len);
#ifndef WITH_LIBRD
/**
* Minimalistic replacement for rd_tsprintf() in case librd is not available.
*/
static char *rd_tsprintf (const char *fmt, ...) {
static __thread char ret[2048];
va_list ap;
va_start(ap, fmt);
vsnprintf(ret, sizeof(ret), fmt, ap);
va_end(ap);
return ret;
}
/**
* Wrapper for pthread_cond_timedwait() that makes it simpler to use
* for delta timeouts.
* `timeout_ms' is the delta timeout in milliseconds.
*/
static int pthread_cond_timedwait_ms (pthread_cond_t *cond,
pthread_mutex_t *mutex,
int timeout_ms) {
struct timeval tv;
struct timespec ts;
gettimeofday(&tv, NULL);
TIMEVAL_TO_TIMESPEC(&tv, &ts);
ts.tv_sec += timeout_ms / 1000;
ts.tv_nsec += (timeout_ms % 1000) * 1000000;
if (ts.tv_nsec > 1000000000) {
ts.tv_sec++;
ts.tv_nsec -= 1000000000;
}
return pthread_cond_timedwait(cond, mutex, &ts);
}
#endif
static void rd_kafka_log (const rd_kafka_t *rk, int level,
const char *fac, const char *fmt, ...) {
char buf[2048];
va_list ap;
if (!rd_kafka_log_cb)
return;
va_start(ap, fmt);
vsnprintf(buf, sizeof(buf), fmt, ap);
va_end(ap);
rd_kafka_log_cb(rk, level, fac, buf);
}
#define rd_kafka_dbg(rk,fac,fmt...) rd_kafka_log(rk,LOG_DEBUG,fac,fmt)
void rd_kafka_log_print (const rd_kafka_t *rk, int level,
const char *fac, const char *buf) {
struct timeval tv;
gettimeofday(&tv, NULL);
fprintf(stderr, "%%%i|%u.%03u|%s|%s| %s\n",
level, (int)tv.tv_sec, (int)(tv.tv_usec / 1000),
fac, rk ? rk->rk_broker.name : "", buf);
}
void rd_kafka_log_syslog (const rd_kafka_t *rk, int level,
const char *fac, const char *buf) {
static int initialized = 0;
if (!initialized)
openlog("rdkafka", LOG_PID|LOG_CONS, LOG_USER);
syslog(level, "%s: %s: %s", fac, rk ? rk->rk_broker.name : "", buf);
}
void rd_kafka_set_logger (void (*func) (const rd_kafka_t *rk, int level,
const char *fac, const char *buf)) {
rd_kafka_log_cb = func;
}
static void rd_kafka_destroy0 (rd_kafka_t *rk) {
if (rk->rk_broker.s != -1)
close(rk->rk_broker.s);
if (rk->rk_broker.rsal)
rd_sockaddr_list_destroy(rk->rk_broker.rsal);
switch (rk->rk_type)
{
case RD_KAFKA_CONSUMER:
if (rk->rk_consumer.topic)
free(rk->rk_consumer.topic);
if (rk->rk_conf.consumer.offset_file)
free(rk->rk_conf.consumer.offset_file);
if (rk->rk_consumer.offset_file_fd != -1)
close(rk->rk_consumer.offset_file_fd);
break;
case RD_KAFKA_PRODUCER:
break;
}
free(rk);
}
void rd_kafka_destroy (rd_kafka_t *rk) {
(void)rd_atomic_sub(&rk->rk_refcnt, 1);
rk->rk_terminate = 1;
if (rk->rk_refcnt == 0)
rd_kafka_destroy0(rk);
}
/**
*
* Locality: Kafka thread
*/
static inline void rd_kafka_set_state (rd_kafka_t *rk,
rd_kafka_state_t new_state) {
if (rk->rk_state == new_state)
return;
rk->rk_state = new_state;
gettimeofday(&rk->rk_tv_state_change, NULL);
}
/**
* Failure propagation to application.
* Will tear down connection to broker and trigger a reconnect.
*
* If 'fmt' is NULL nothing will be logged or propagated to the application.
*
* Locality: Kafka thread
*/
static void rd_kafka_fail (rd_kafka_t *rk, const char *fmt, ...) {
va_list ap;
pthread_mutex_lock(&rk->rk_lock);
rk->rk_err.err = errno;
rd_kafka_set_state(rk, RD_KAFKA_STATE_DOWN);
if (rk->rk_broker.s != -1) {
close(rk->rk_broker.s);
rk->rk_broker.s = -1;
}
if (fmt) {
va_start(ap, fmt);
vsnprintf(rk->rk_err.msg, sizeof(rk->rk_err.msg), fmt, ap);
va_end(ap);
rd_kafka_log(rk, LOG_ERR, "FAIL", "%s", rk->rk_err.msg);
/* Send ERR op back to application for processing. */
rd_kafka_op_reply(rk, RD_KAFKA_OP_ERR,
RD_KAFKA_RESP_ERR__FAIL, 0,
strdup(rk->rk_err.msg),
strlen(rk->rk_err.msg), 0);
}
pthread_mutex_unlock(&rk->rk_lock);
}
int rd_kafka_offset_store (rd_kafka_t *rk, uint64_t offset) {
/* File-based */
if (rk->rk_conf.consumer.offset_file) {
char tmp[32];
int len;
int r;
snprintf(tmp, sizeof(tmp), "%"PRIu64"\n", offset);
len = strlen(tmp);
if (ftruncate(rk->rk_consumer.offset_file_fd, 0) == -1 ||
lseek(rk->rk_consumer.offset_file_fd, SEEK_SET, 0) == -1) {
rd_kafka_log(rk, LOG_ERR, "OFFTRUNC",
"truncate/lseek of offset file %s (fd %i) "
"failed: %s",
rk->rk_conf.consumer.offset_file,
rk->rk_consumer.offset_file_fd,
strerror(errno));
return -1;
}
if ((r = write(rk->rk_consumer.offset_file_fd,
tmp, len)) != len) {
rd_kafka_log(rk, LOG_ERR, "OFFWRITE",
"offset %"PRIu64 " write to "
"file %s failed: %s",
offset,
rk->rk_conf.consumer.offset_file,
r == -1 ? strerror(errno) :
"partial write");
return -1;
}
return 0;
} else {
/* No storage defined for permanent offsets */
return 0;
}
}
/**
* Blocking connect attempt.
*
* Locality: Kafka thread
*/
static int rd_kafka_connect (rd_kafka_t *rk) {
rd_sockaddr_inx_t *sinx = rd_sockaddr_list_next(rk->rk_broker.rsal);
if ((rk->rk_broker.s = socket(sinx->sinx_family,
SOCK_STREAM, IPPROTO_TCP)) == -1) {
rd_kafka_fail(rk,
"Failed to create %s socket: %s",
rd_family2str(sinx->sinx_family),
strerror(errno));
return -1;
}
rd_kafka_set_state(rk, RD_KAFKA_STATE_CONNECTING);
if (connect(rk->rk_broker.s, (struct sockaddr *)sinx,
RD_SOCKADDR_INX_LEN(sinx)) == -1) {
/* Avoid duplicate log messages */
if (rk->rk_err.err == errno)
rd_kafka_fail(rk, NULL);
else
rd_kafka_fail(rk,
"Failed to connect to broker at %s: %s",
rd_sockaddr2str(sinx,
RD_SOCKADDR2STR_F_NICE),
strerror(errno));
return -1;
}
rd_kafka_dbg(rk, "CONNECTED", "connected to %s",
rd_sockaddr2str(sinx, RD_SOCKADDR2STR_F_NICE));
rd_kafka_set_state(rk, RD_KAFKA_STATE_UP);
rk->rk_err.err = 0;
return 0;
}
void rd_kafka_op_destroy (rd_kafka_t *rk, rd_kafka_op_t *rko) {
if (rko->rko_topic && rko->rko_flags & RD_KAFKA_OP_F_FREE_TOPIC)
free(rko->rko_topic);
if (rko->rko_payload && rko->rko_flags & RD_KAFKA_OP_F_FREE)
free(rko->rko_payload);
free(rko);
}
static int rd_kafka_send (rd_kafka_t *rk, const struct msghdr *msg) {
int r;
r = sendmsg(rk->rk_broker.s, msg, 0);
if (r == -1) {
rd_kafka_fail(rk, "Send failed: %s", strerror(errno));
return -1;
}
rk->rk_broker.stats.tx_bytes += r;
rk->rk_broker.stats.tx++;
return r;
}
#define RD_KAFKA_SEND_END -1
static void rd_kafka_send_request (rd_kafka_t *rk,
uint16_t msgtype,
struct rd_kafkap_topicpart *topicpart,
...) {
va_list ap;
int sz;
void *ptr;
const int iov_begin = 2; /* Request header + topic&partition */
struct iovec iov[IOV_MAX];
struct msghdr msg = {
msg_iov: iov,
msg_iovlen: iov_begin,
};
struct rd_kafkap_req req = {
rkpr_type: htons(msgtype),
};
int len = 0;
int hdr_len;
/* Request header */
iov[0].iov_base = &req;
iov[0].iov_len = sizeof(req);
iov[1].iov_base = topicpart->rkptp_buf;
iov[1].iov_len = topicpart->rkptp_len;
hdr_len = iov[0].iov_len - sizeof(req.rkpr_len) + iov[1].iov_len;
va_start(ap, topicpart);
while ((sz = va_arg(ap, int)) != RD_KAFKA_SEND_END) {
ptr = va_arg(ap, void *);
iov[msg.msg_iovlen].iov_len = sz;
iov[msg.msg_iovlen++].iov_base = ptr;
len += sz;
if (msg.msg_iovlen == IOV_MAX) {
req.rkpr_len = htonl(hdr_len + len);
req.rkpr_topic_len = htons(topicpart->rkptp_len - 4);
if (rd_kafka_send(rk, &msg) == -1)
break; /* Send failed, socket failure. */
msg.msg_iovlen = iov_begin;
}
}
va_end(ap);
if (msg.msg_iovlen > iov_begin) {
req.rkpr_len = htonl(hdr_len + len);
req.rkpr_topic_len = htons(topicpart->rkptp_len - 4);
rd_kafka_send(rk, &msg);
}
}
/**
* Serialize a topic+partition combo into a temporary buffer.
*/
static struct rd_kafkap_topicpart *
rd_kafka_topicpart_serialize (const char *topic, uint32_t partition) {
static __thread union {
struct rd_kafkap_topicpart topicpart;
char buf[sizeof(struct rd_kafkap_topicpart) +
RD_KAFKA_TOPIC_MAXLEN + sizeof(partition)];
} u;
int len;
len = strlen(topic);
if (len > RD_KAFKA_TOPIC_MAXLEN) {
/* This may look silly, but it saves us from mallocs,
* and it should suffice for most uses, if not, increase it! */
fprintf(stderr,
"RD-KAFKA-FIXME: Topic name (%s) is too long (max %i),"
" will truncate it (increase in rdkafka.h)\n",
topic, RD_KAFKA_TOPIC_MAXLEN);
len = RD_KAFKA_TOPIC_MAXLEN;
}
partition = htonl(partition);
memcpy(u.topicpart.rkptp_buf, topic, len);
memcpy(u.topicpart.rkptp_buf+len, &partition, sizeof(partition));
u.topicpart.rkptp_len = len + sizeof(partition);
return &u.topicpart;
}
/**
* Initialize a queue.
*/
static void rd_kafka_q_init (rd_kafka_q_t *rkq) {
TAILQ_INIT(&rkq->rkq_q);
rkq->rkq_qlen = 0;
pthread_mutex_init(&rkq->rkq_lock, NULL);
pthread_cond_init(&rkq->rkq_cond, NULL);
}
/**
* Enqueue the 'rko' op at the tail of the queue 'rkq'.
*
* Locality: any thread.
*/
static inline void rd_kafka_q_enq (rd_kafka_q_t *rkq, rd_kafka_op_t *rko) {
pthread_mutex_lock(&rkq->rkq_lock);
TAILQ_INSERT_TAIL(&rkq->rkq_q, rko, rko_link);
(void)rd_atomic_add(&rkq->rkq_qlen, 1);
pthread_cond_signal(&rkq->rkq_cond);
pthread_mutex_unlock(&rkq->rkq_lock);
}
/**
* Pop an op from a queue.
*
* Locality: any thread.
*/
static rd_kafka_op_t *rd_kafka_q_pop (rd_kafka_q_t *rkq, int timeout_ms) {
rd_kafka_op_t *rko;
rd_ts_t last;
pthread_mutex_lock(&rkq->rkq_lock);
while (!(rko = TAILQ_FIRST(&rkq->rkq_q)) &&
(timeout_ms == RD_POLL_INFINITE || timeout_ms > 0)) {
if (timeout_ms != RD_POLL_INFINITE) {
last = rd_clock();
if (pthread_cond_timedwait_ms(&rkq->rkq_cond,
&rkq->rkq_lock,
timeout_ms) ==
ETIMEDOUT) {
pthread_mutex_unlock(&rkq->rkq_lock);
return NULL;
}
timeout_ms -= (rd_clock() - last) / 1000;
} else
pthread_cond_wait(&rkq->rkq_cond, &rkq->rkq_lock);
}
if (rko) {
TAILQ_REMOVE(&rkq->rkq_q, rko, rko_link);
(void)rd_atomic_sub(&rkq->rkq_qlen, 1);
}
pthread_mutex_unlock(&rkq->rkq_lock);
return rko;
}
/**
* Send an op back to the application.
*
* Locality: Kafka threads
*/
static void rd_kafka_op_reply0 (rd_kafka_t *rk, rd_kafka_op_t *rko,
rd_kafka_op_type_t type,
rd_kafka_resp_err_t err, uint8_t compression,
void *payload, int len,
uint64_t offset_len) {
rko->rko_type = type;
rko->rko_flags |= RD_KAFKA_OP_F_FREE;
rko->rko_payload = payload;
rko->rko_len = len;
rko->rko_err = err;
rko->rko_compression = compression;
rko->rko_offset = offset_len;
}
/**
* Send an op back to the application.
*
* Locality: Kafka thread
*/
static void rd_kafka_op_reply (rd_kafka_t *rk,
rd_kafka_op_type_t type,
rd_kafka_resp_err_t err, uint8_t compression,
void *payload, int len,
uint64_t offset_len) {
rd_kafka_op_t *rko;
rko = calloc(1, sizeof(*rko));
if (err && !payload) {
/* Provide human readable error string if not provided. */
/* Provide more info for some errors. */
if (err == RD_KAFKA_RESP_ERR_OFFSET_OUT_OF_RANGE)
payload =
strdup(rd_tsprintf("%s (%"PRIu64")",
rd_kafka_err2str(err),
rk->rk_consumer.offset));
else
payload = strdup(rd_kafka_err2str(err));
len = strlen(payload);
}
rd_kafka_op_reply0(rk, rko, type, err, compression,
payload, len, offset_len);
rd_kafka_q_enq(&rk->rk_rep, rko);
}
/**
* Receive and purge data.
*/
static void rd_kafka_recv_null (rd_kafka_t *rk, int total_len) {
char buf[256];
while (total_len > 0) {
int len = total_len < sizeof(buf) ? total_len : sizeof(buf);
int r;
if ((r = recv(rk->rk_broker.s, buf, len, MSG_DONTWAIT)) < 1)
return;
total_len -= r;
}
}
/**
* Read data from socket.
*/
static int rd_kafka_recv0 (rd_kafka_t *rk, const char *what,
void *data, int len, int partial_ok) {
int r;
if ((r = recv(rk->rk_broker.s, data, len,
partial_ok ? 0 : MSG_WAITALL)) != len) {
if (r > 0) {
if (partial_ok)
return r;
else
rd_kafka_fail(rk, "Partial %s response "
"(%i<%i bytes)",
what, r, len);
} else if (r == 0)
rd_kafka_fail(rk, "No more data while expecting %s "
"(%i bytes)", what, len);
else
rd_kafka_fail(rk, "Failed to recv %i bytes for %s: %s",
len, what, strerror(errno));
return -1;
}
return r;
}
/**
* Parse a single message from buf and passes it to the application.
* 'rko' is optional and will be used, without enqueuing, instead of
* creating a new op, if non-NULL.
*
* If 'offset_len' is non-zero it depicts the outer (compressed) message's
* offset length.
*
* Returns -1 on failure or the consumed data length on success.
*/
static int rd_kafka_msg_parse (rd_kafka_t *rk, char *buf, int len,
rd_kafka_op_t *rko, uint64_t offset_len) {
struct rd_kafkap_msg *msg;
char *payload;
/* Transmissions may be terminated mid-message, this is
* a soft-error that should be ignored and the client
* should try again. */
msg = (struct rd_kafkap_msg *)buf;
msg->rkpm_len = ntohl(msg->rkpm_len);
if (msg->rkpm_len < sizeof(*msg) - sizeof(msg->rkpm_len)) {
/* Formatting error. */
rd_kafka_op_reply0(rk, rko, RD_KAFKA_OP_FETCH,
RD_KAFKA_RESP_ERR__BAD_MSG,
0, NULL, 0, 0);
return -1;
}
if (!offset_len)
offset_len = msg->rkpm_len + sizeof(msg->rkpm_len);
if (msg->rkpm_len > 4000000) {
rd_kafka_op_reply0(rk, rko, RD_KAFKA_OP_FETCH,
RD_KAFKA_RESP_ERR__BAD_MSG,
0, NULL, 0, offset_len);
return msg->rkpm_len + sizeof(msg->rkpm_len);
}
msg->rkpm_len -= sizeof(*msg) - sizeof(msg->rkpm_len);
payload = malloc(msg->rkpm_len);
memcpy(payload, msg+1, msg->rkpm_len);
rd_kafka_op_reply0(rk, rko, RD_KAFKA_OP_FETCH, 0,
msg->rkpm_compression,
payload, msg->rkpm_len, offset_len);
return sizeof(*msg) + msg->rkpm_len;
}
/**
* Receive an entire message from the broker.
*
* Returns the number of data reply ops created.
*
* Locality: Kafka thread
*/
static int rd_kafka_recv (rd_kafka_t *rk) {
struct rd_kafkap_resp resp;
int replycnt = 0;
if (rd_kafka_recv0(rk, "response header",
&resp, sizeof(resp), 0) == -1)
return 0;
resp.rkprp_len = ntohl(resp.rkprp_len);
resp.rkprp_error = ntohs(resp.rkprp_error);
/* Error from broker? */
if (resp.rkprp_error != RD_KAFKA_RESP_ERR_NO_ERROR) {
rd_kafka_op_reply(rk, RD_KAFKA_OP_FETCH, resp.rkprp_error, 0,
NULL, 0, 0);
/* Consume remaining buffer */
if (resp.rkprp_len)
rd_kafka_recv_null(rk, resp.rkprp_len - 4);
return 0;
}
if (resp.rkprp_len < sizeof(resp.rkprp_error) ||
resp.rkprp_len > rk->rk_conf.max_msg_size) {
rd_kafka_fail(rk, "Invalid (or too long) response length %lu",
resp.rkprp_len);
return 0;
}
resp.rkprp_len -= sizeof(resp.rkprp_error);
/* Start chewing off messages, send one op back to application
* for each message parsed. */
while (resp.rkprp_len >= sizeof(struct rd_kafkap_msg)) {
struct rd_kafkap_msg msg;
char *buf;
/* Transmissions may be terminated mid-message, this is
* a soft-error that should be ignored and the client
* should try again at a later offset. */
if (rd_kafka_recv0(rk, "message",
&msg, sizeof(msg), 0) == -1)
return replycnt;
msg.rkpm_len = ntohl(msg.rkpm_len);
resp.rkprp_len -= sizeof(msg);
if (msg.rkpm_len < sizeof(msg) - sizeof(msg.rkpm_len))
return replycnt;
if (msg.rkpm_len > rk->rk_conf.max_msg_size) {
rd_kafka_fail(rk, "Invalid (or too long) response "
"message length %lu",
msg.rkpm_len);
return replycnt;
}
msg.rkpm_len -= sizeof(msg) - 4;
if (msg.rkpm_len > resp.rkprp_len) {
/* Partial message, drop it. */
if (resp.rkprp_len)
rd_kafka_recv_null(rk, resp.rkprp_len);
return replycnt;
}
/*
* Message payload
*/
resp.rkprp_len -= msg.rkpm_len;
buf = malloc(msg.rkpm_len);
if (rd_kafka_recv0(rk, "message payload",
buf, msg.rkpm_len, 0) == -1) {
free(buf);
return replycnt;
}
rk->rk_consumer.offset += msg.rkpm_len + sizeof(msg);
rd_kafka_op_reply(rk, RD_KAFKA_OP_FETCH, 0,
msg.rkpm_compression,
buf, msg.rkpm_len, rk->rk_consumer.offset);
replycnt++;
}
/* Drop padding */
if (resp.rkprp_len > 0)
rd_kafka_recv_null(rk, resp.rkprp_len);
return replycnt;
}
/**
* Send PRODUCE message.
*
* Locality: Kafka thread
*/
static void rd_kafka_produce_send (rd_kafka_t *rk, rd_kafka_op_t *rko) {
struct rd_kafkap_msg msg = {
rkpm_len: htonl(sizeof(msg) - sizeof(msg.rkpm_len) + rko->rko_len),
rkpm_magic: RD_KAFKAP_MSG_MAGIC_COMPRESSION_ATTR,
rkpm_compression: RD_KAFKAP_MSG_COMPRESSION_NONE,
rkpm_cksum: htonl(rd_crc32(rko->rko_payload, rko->rko_len)),
};
struct rd_kafkap_produce prod = {
rkpp_msgs_len: htonl(sizeof(msg) + rko->rko_len),
};
rd_kafka_send_request(rk,
RD_KAFKAP_PRODUCE,
rd_kafka_topicpart_serialize(rko->rko_topic,
rko->rko_partition),
sizeof(prod), &prod,
sizeof(msg), &msg,
rko->rko_len, rko->rko_payload,
RD_KAFKA_SEND_END);
}
/**
* Send FETCH message
*
* Locality: Kafka thread
*/
static void rd_kafka_fetch_send (rd_kafka_t *rk, uint64_t offset,
uint32_t max_size) {
struct rd_kafkap_fetch_req freq = {
rkpfr_offset: htobe64(offset),
rkpfr_max_size: htonl(max_size),
};
rd_kafka_send_request(rk,
RD_KAFKAP_FETCH,
rd_kafka_topicpart_serialize(rk->
rk_consumer.topic,
rk->
rk_consumer.
partition),
sizeof(freq), &freq,
RD_KAFKA_SEND_END);
}
/**
* Producer: Wait for PRODUCE events from application.
*
* Locality: Kafka thread
*/
static void rd_kafka_wait_op (rd_kafka_t *rk) {
while (!rk->rk_terminate && rk->rk_state == RD_KAFKA_STATE_UP) {
rd_kafka_op_t *rko =
rd_kafka_q_pop(&rk->rk_op, RD_POLL_INFINITE);
rd_kafka_produce_send(rk, rko);
rd_kafka_op_destroy(rk, rko);
}
}
/**
* Consumer: Wait for IO from broker.
*
* Locality: Kafka thread
*/
static void rd_kafka_consumer_wait_io (rd_kafka_t *rk) {
while (!rk->rk_terminate && rk->rk_state == RD_KAFKA_STATE_UP) {
int r;
if (rd_kafka_replyq_len(rk) <
rk->rk_conf.consumer.replyq_low_thres) {
/* Below low watermark, fetch more messages. */
rd_kafka_fetch_send(rk,
rk->rk_consumer.offset,
rk->rk_conf.consumer.max_size);
} else {
struct pollfd pfd = { fd: rk->rk_broker.s,
events: POLLIN };
/* We have messages, wait for more data from broker,
* if any. */
r = poll(&pfd, 1, rk->rk_conf.consumer.poll_interval);
if (r == -1) { /* Error */
if (errno == EINTR)
continue;
rd_kafka_fail(rk,
"Failed to poll socket %i: %s",
rk->rk_broker.s,
strerror(errno));
break;
} else if (r == 0) /* Timeout */
continue;
}
/* Blocking receive of message. */
r = rd_kafka_recv(rk);
/* No messages received? Wait a while before retrying. */
if (!r)
usleep(rk->rk_conf.consumer.poll_interval * 1000);
}
}
/**
* Kafka thread's main loop.
*
* Locality: Kafka thread.
*/
static void *rd_kafka_thread_main (void *arg) {
rd_kafka_t *rk = arg;
while (!rk->rk_terminate) {
switch (rk->rk_state)
{
case RD_KAFKA_STATE_DOWN:
/* ..connect() will block until done (or failure) */
if (rd_kafka_connect(rk) == -1)
sleep(1); /*Sleep between connection attempts*/
break;
case RD_KAFKA_STATE_CONNECTING:
break;
case RD_KAFKA_STATE_UP:
/* .._wait_*() blocks for as long as the
* state remains UP. */
if (rk->rk_type == RD_KAFKA_PRODUCER)
rd_kafka_wait_op(rk);
else
rd_kafka_consumer_wait_io(rk);
break;
}
}
rd_kafka_destroy(rk);
return NULL;
}
static const char *rd_kafka_type2str (rd_kafka_type_t type) {
static const char *types[] = {
[RD_KAFKA_PRODUCER] = "producer",
[RD_KAFKA_CONSUMER] = "consumer",