-
Notifications
You must be signed in to change notification settings - Fork 3.2k
/
Copy pathrdkafka_buf.h
1524 lines (1309 loc) · 66.8 KB
/
rdkafka_buf.h
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.
*/
#ifndef _RDKAFKA_BUF_H_
#define _RDKAFKA_BUF_H_
#include "rdkafka_int.h"
#include "rdcrc32.h"
#include "rdlist.h"
#include "rdbuf.h"
#include "rdkafka_msgbatch.h"
typedef struct rd_kafka_broker_s rd_kafka_broker_t;
#define RD_KAFKA_HEADERS_IOV_CNT 2
/**
* Temporary buffer with memory aligned writes to accommodate
* effective and platform safe struct writes.
*/
typedef struct rd_tmpabuf_s {
size_t size;
size_t of;
char *buf;
int failed;
rd_bool_t assert_on_fail;
} rd_tmpabuf_t;
/**
* @brief Initialize new tmpabuf of non-final \p size bytes.
*/
static RD_UNUSED void
rd_tmpabuf_new(rd_tmpabuf_t *tab, size_t size, rd_bool_t assert_on_fail) {
tab->buf = NULL;
tab->size = RD_ROUNDUP(size, 8);
tab->of = 0;
tab->failed = 0;
tab->assert_on_fail = assert_on_fail;
}
/**
* @brief Add a new allocation of \p _size bytes,
* rounded up to maximum word size,
* for \p _times times.
*/
#define rd_tmpabuf_add_alloc_times(_tab, _size, _times) \
(_tab)->size += RD_ROUNDUP(_size, 8) * _times
#define rd_tmpabuf_add_alloc(_tab, _size) \
rd_tmpabuf_add_alloc_times(_tab, _size, 1)
/**
* @brief Finalize tmpabuf pre-allocating tab->size bytes.
*/
#define rd_tmpabuf_finalize(_tab) (_tab)->buf = rd_malloc((_tab)->size)
/**
* @brief Free memory allocated by tmpabuf
*/
static RD_UNUSED void rd_tmpabuf_destroy(rd_tmpabuf_t *tab) {
rd_free(tab->buf);
}
/**
* @returns 1 if a previous operation failed.
*/
static RD_UNUSED RD_INLINE int rd_tmpabuf_failed(rd_tmpabuf_t *tab) {
return tab->failed;
}
/**
* @brief Allocate \p size bytes for writing, returning an aligned pointer
* to the memory.
* @returns the allocated pointer (within the tmpabuf) on success or
* NULL if the requested number of bytes + alignment is not available
* in the tmpabuf.
*/
static RD_UNUSED void *
rd_tmpabuf_alloc0(const char *func, int line, rd_tmpabuf_t *tab, size_t size) {
void *ptr;
if (unlikely(tab->failed))
return NULL;
if (unlikely(tab->of + size > tab->size)) {
if (tab->assert_on_fail) {
fprintf(stderr,
"%s: %s:%d: requested size %" PRIusz
" + %" PRIusz " > %" PRIusz "\n",
__FUNCTION__, func, line, tab->of, size,
tab->size);
assert(!*"rd_tmpabuf_alloc: not enough size in buffer");
}
return NULL;
}
ptr = (void *)(tab->buf + tab->of);
tab->of += RD_ROUNDUP(size, 8);
return ptr;
}
#define rd_tmpabuf_alloc(tab, size) \
rd_tmpabuf_alloc0(__FUNCTION__, __LINE__, tab, size)
/**
* @brief Write \p buf of \p size bytes to tmpabuf memory in an aligned fashion.
*
* @returns the allocated and written-to pointer (within the tmpabuf) on success
* or NULL if the requested number of bytes + alignment is not
* available in the tmpabuf.
*/
static RD_UNUSED void *rd_tmpabuf_write0(const char *func,
int line,
rd_tmpabuf_t *tab,
const void *buf,
size_t size) {
void *ptr = rd_tmpabuf_alloc0(func, line, tab, size);
if (likely(ptr && size))
memcpy(ptr, buf, size);
return ptr;
}
#define rd_tmpabuf_write(tab, buf, size) \
rd_tmpabuf_write0(__FUNCTION__, __LINE__, tab, buf, size)
/**
* @brief Wrapper for rd_tmpabuf_write() that takes a nul-terminated string.
*/
static RD_UNUSED char *rd_tmpabuf_write_str0(const char *func,
int line,
rd_tmpabuf_t *tab,
const char *str) {
return rd_tmpabuf_write0(func, line, tab, str, strlen(str) + 1);
}
#define rd_tmpabuf_write_str(tab, str) \
rd_tmpabuf_write_str0(__FUNCTION__, __LINE__, tab, str)
/**
* Response handling callback.
*
* NOTE: Callbacks must check for 'err == RD_KAFKA_RESP_ERR__DESTROY'
* which indicates that some entity is terminating (rd_kafka_t, broker,
* toppar, queue, etc) and the callback may not be called in the
* correct thread. In this case the callback must perform just
* the most minimal cleanup and dont trigger any other operations.
*
* NOTE: rkb, reply and request may be NULL, depending on error situation.
*/
typedef void(rd_kafka_resp_cb_t)(rd_kafka_t *rk,
rd_kafka_broker_t *rkb,
rd_kafka_resp_err_t err,
rd_kafka_buf_t *reply,
rd_kafka_buf_t *request,
void *opaque);
/**
* @brief Sender callback. This callback is used to construct and send (enq)
* a rkbuf on a particular broker.
*/
typedef rd_kafka_resp_err_t(rd_kafka_send_req_cb_t)(rd_kafka_broker_t *rkb,
rd_kafka_op_t *rko,
rd_kafka_replyq_t replyq,
rd_kafka_resp_cb_t *resp_cb,
void *reply_opaque);
/**
* @brief Request maker. A callback that constructs the actual contents
* of a request.
*
* When constructing a request the ApiVersion typically needs to be selected
* which requires the broker's supported ApiVersions to be known, which in
* turn requires the broker connection to be UP.
*
* As a buffer constructor you have two choices:
* a. acquire the broker handle, wait for it to come up, and then construct
* the request buffer, or
* b. acquire the broker handle, enqueue an uncrafted/unmaked
* request on the broker request queue, and when the broker is up
* the make_req_cb will be called for you to construct the request.
*
* From a code complexity standpoint, the latter option is usually the least
* complex and voids the caller to care about any of the broker state.
* Any information that is required to construct the request is passed through
* the make_opaque, which can be automatically freed by the buffer code
* when it has been used, or handled by the caller (in which case it must
* outlive the lifetime of the buffer).
*
* Usage:
*
* 1. Construct an rkbuf with the appropriate ApiKey.
* 2. Make a copy or reference of any data that is needed to construct the
* request, e.g., through rd_kafka_topic_partition_list_copy(). This
* data is passed by the make_opaque.
* 3. Set the make callback by calling rd_kafka_buf_set_maker() and pass
* the make_opaque data and a free function, if needed.
* 4. The callback will eventually be called from the broker thread.
* 5. In the make callback construct the request on the passed rkbuf.
* 6. The request is sent to the broker and the make_opaque is freed.
*
* See rd_kafka_ListOffsetsRequest() in rdkafka_request.c for an example.
*
*/
typedef rd_kafka_resp_err_t(rd_kafka_make_req_cb_t)(rd_kafka_broker_t *rkb,
rd_kafka_buf_t *rkbuf,
void *make_opaque);
/**
* @struct Request and response buffer
*
*/
struct rd_kafka_buf_s { /* rd_kafka_buf_t */
TAILQ_ENTRY(rd_kafka_buf_s) rkbuf_link;
int32_t rkbuf_corrid;
rd_ts_t rkbuf_ts_retry; /* Absolute send retry time */
int rkbuf_flags; /* RD_KAFKA_OP_F */
/** What convenience flags to copy from request to response along
* with the reqhdr. */
#define RD_KAFKA_BUF_FLAGS_RESP_COPY_MASK (RD_KAFKA_OP_F_FLEXVER)
rd_kafka_prio_t rkbuf_prio; /**< Request priority */
rd_buf_t rkbuf_buf; /**< Send/Recv byte buffer */
rd_slice_t rkbuf_reader; /**< Buffer slice reader for rkbuf_buf */
int rkbuf_connid; /* broker connection id (used when buffer
* was partially sent). */
size_t rkbuf_totlen; /* recv: total expected length,
* send: not used */
rd_crc32_t rkbuf_crc; /* Current CRC calculation */
struct rd_kafkap_reqhdr rkbuf_reqhdr; /* Request header.
* These fields are encoded
* and written to output buffer
* on buffer finalization.
* Note:
* The request's
* reqhdr is copied to the
* response's reqhdr as a
* convenience. */
struct rd_kafkap_reshdr rkbuf_reshdr; /* Response header.
* Decoded fields are copied
* here from the buffer
* to provide an ease-of-use
* interface to the header */
int32_t rkbuf_expected_size; /* expected size of message */
rd_kafka_replyq_t rkbuf_replyq; /* Enqueue response on replyq */
rd_kafka_replyq_t rkbuf_orig_replyq; /* Original replyq to be used
* for retries from inside
* the rkbuf_cb() callback
* since rkbuf_replyq will
* have been reset. */
rd_kafka_resp_cb_t *rkbuf_cb; /* Response callback */
struct rd_kafka_buf_s *rkbuf_response; /* Response buffer */
rd_kafka_make_req_cb_t *rkbuf_make_req_cb; /**< Callback to construct
* the request itself.
* Will be used if
* RD_KAFKA_OP_F_NEED_MAKE
* is set. */
void *rkbuf_make_opaque; /**< Opaque passed to rkbuf_make_req_cb.
* Will be freed automatically after use
* by the rkbuf code. */
void (*rkbuf_free_make_opaque_cb)(void *); /**< Free function for
* rkbuf_make_opaque. */
struct rd_kafka_broker_s *rkbuf_rkb; /**< Optional broker object
* with refcnt increased used
* for logging decode errors
* if log_decode_errors is > 0 */
rd_refcnt_t rkbuf_refcnt;
void *rkbuf_opaque;
int rkbuf_max_retries; /**< Maximum retries to attempt. */
int rkbuf_retries; /**< Retries so far. */
int rkbuf_features; /* Required feature(s) that must be
* supported by broker. */
rd_ts_t rkbuf_ts_enq;
rd_ts_t rkbuf_ts_sent; /* Initially: Absolute time of transmission,
* after response: RTT. */
/* Request timeouts:
* rkbuf_ts_timeout is the effective absolute request timeout used
* by the timeout scanner to see if a request has timed out.
* It is set when a request is enqueued on the broker transmit
* queue based on the relative or absolute timeout:
*
* rkbuf_rel_timeout is the per-request-transmit relative timeout,
* this value is reused for each sub-sequent retry of a request.
*
* rkbuf_abs_timeout is the absolute request timeout, spanning
* all retries.
* This value is effectively limited by socket.timeout.ms for
* each transmission, but the absolute timeout for a request's
* lifetime is the absolute value.
*
* Use rd_kafka_buf_set_timeout() to set a relative timeout
* that will be reused on retry,
* or rd_kafka_buf_set_abs_timeout() to set a fixed absolute timeout
* for the case where the caller knows the request will be
* semantically outdated when that absolute time expires, such as for
* session.timeout.ms-based requests.
*
* The decision to retry a request is delegated to the rkbuf_cb
* response callback, which should use rd_kafka_err_action()
* and check the return actions for RD_KAFKA_ERR_ACTION_RETRY to be set
* and then call rd_kafka_buf_retry().
* rd_kafka_buf_retry() will enqueue the request on the rkb_retrybufs
* queue with a backoff time of retry.backoff.ms.
* The rkb_retrybufs queue is served by the broker thread's timeout
* scanner.
* @warning rkb_retrybufs is NOT purged on broker down.
*/
rd_ts_t rkbuf_ts_timeout; /* Request timeout (absolute time). */
rd_ts_t
rkbuf_abs_timeout; /* Absolute timeout for request, including
* retries.
* Mutually exclusive with rkbuf_rel_timeout*/
int rkbuf_rel_timeout; /* Relative timeout (ms), used for retries.
* Defaults to socket.timeout.ms.
* Mutually exclusive with rkbuf_abs_timeout*/
rd_bool_t rkbuf_force_timeout; /**< Force request timeout to be
* remaining abs_timeout regardless
* of socket.timeout.ms. */
int64_t rkbuf_offset; /* Used by OffsetCommit */
rd_list_t *rkbuf_rktp_vers; /* Toppar + Op Version map.
* Used by FetchRequest. */
rd_kafka_resp_err_t rkbuf_err; /* Buffer parsing error code */
union {
struct {
rd_list_t *topics; /* Requested topics (char *) */
rd_list_t *
topic_ids; /* Requested topic ids rd_kafka_Uuid_t */
char *reason; /* Textual reason */
rd_kafka_op_t *rko; /* Originating rko with replyq
* (if any) */
rd_bool_t all_topics; /**< Full/All topics requested */
rd_bool_t cgrp_update; /**< Update cgrp with topic
* status from response. */
rd_bool_t force_racks; /**< Force the returned metadata
* to contain partition to
* rack mapping. */
int *decr; /* Decrement this integer by one
* when request is complete:
* typically points to metadata
* cache's full_.._sent.
* Will be performed with
* decr_lock held. */
mtx_t *decr_lock;
} Metadata;
struct {
rd_kafka_msgbatch_t batch; /**< MessageSet/batch */
} Produce;
struct {
rd_bool_t commit; /**< true = txn commit,
* false = txn abort */
} EndTxn;
} rkbuf_u;
#define rkbuf_batch rkbuf_u.Produce.batch
const char *rkbuf_uflow_mitigation; /**< Buffer read underflow
* human readable mitigation
* string (const memory).
* This is used to hint the
* user why the underflow
* might have occurred, which
* depends on request type. */
};
/**
* @name Read buffer interface
*
* Memory reading helper macros to be used when parsing network responses.
*
* Assumptions:
* - an 'err_parse:' goto-label must be available for error bailouts,
* the error code will be set in rkbuf->rkbuf_err
* - local `int log_decode_errors` variable set to the logging level
* to log parse errors (or 0 to turn off logging).
*/
#define rd_kafka_buf_parse_fail(rkbuf, ...) \
do { \
if (log_decode_errors > 0 && rkbuf->rkbuf_rkb) { \
rd_rkb_log( \
rkbuf->rkbuf_rkb, log_decode_errors, "PROTOERR", \
"Protocol parse failure for %s v%hd%s " \
"at %" PRIusz "/%" PRIusz \
" (%s:%i) " \
"(incorrect broker.version.fallback?)", \
rd_kafka_ApiKey2str(rkbuf->rkbuf_reqhdr.ApiKey), \
rkbuf->rkbuf_reqhdr.ApiVersion, \
(rkbuf->rkbuf_flags & RD_KAFKA_OP_F_FLEXVER \
? "(flex)" \
: ""), \
rd_slice_offset(&rkbuf->rkbuf_reader), \
rd_slice_size(&rkbuf->rkbuf_reader), __FUNCTION__, \
__LINE__); \
rd_rkb_log(rkbuf->rkbuf_rkb, log_decode_errors, \
"PROTOERR", __VA_ARGS__); \
} \
(rkbuf)->rkbuf_err = RD_KAFKA_RESP_ERR__BAD_MSG; \
goto err_parse; \
} while (0)
/**
* @name Fail buffer reading due to buffer underflow.
*/
#define rd_kafka_buf_underflow_fail(rkbuf, wantedlen, ...) \
do { \
if (log_decode_errors > 0 && rkbuf->rkbuf_rkb) { \
char __tmpstr[256]; \
rd_snprintf(__tmpstr, sizeof(__tmpstr), \
": " __VA_ARGS__); \
if (strlen(__tmpstr) == 2) \
__tmpstr[0] = '\0'; \
rd_rkb_log( \
rkbuf->rkbuf_rkb, log_decode_errors, "PROTOUFLOW", \
"Protocol read buffer underflow " \
"for %s v%hd " \
"at %" PRIusz "/%" PRIusz \
" (%s:%i): " \
"expected %" PRIusz \
" bytes > " \
"%" PRIusz " remaining bytes (%s)%s", \
rd_kafka_ApiKey2str(rkbuf->rkbuf_reqhdr.ApiKey), \
rkbuf->rkbuf_reqhdr.ApiVersion, \
rd_slice_offset(&rkbuf->rkbuf_reader), \
rd_slice_size(&rkbuf->rkbuf_reader), __FUNCTION__, \
__LINE__, wantedlen, \
rd_slice_remains(&rkbuf->rkbuf_reader), \
rkbuf->rkbuf_uflow_mitigation \
? rkbuf->rkbuf_uflow_mitigation \
: "incorrect broker.version.fallback?", \
__tmpstr); \
} \
(rkbuf)->rkbuf_err = RD_KAFKA_RESP_ERR__UNDERFLOW; \
goto err_parse; \
} while (0)
/**
* Returns the number of remaining bytes available to read.
*/
#define rd_kafka_buf_read_remain(rkbuf) rd_slice_remains(&(rkbuf)->rkbuf_reader)
/**
* Checks that at least 'len' bytes remain to be read in buffer, else fails.
*/
#define rd_kafka_buf_check_len(rkbuf, len) \
do { \
size_t __len0 = (size_t)(len); \
if (unlikely(__len0 > rd_kafka_buf_read_remain(rkbuf))) { \
rd_kafka_buf_underflow_fail(rkbuf, __len0); \
} \
} while (0)
/**
* Skip (as in read and ignore) the next 'len' bytes.
*/
#define rd_kafka_buf_skip(rkbuf, len) \
do { \
size_t __len1 = (size_t)(len); \
if (__len1 && \
!rd_slice_read(&(rkbuf)->rkbuf_reader, NULL, __len1)) \
rd_kafka_buf_check_len(rkbuf, __len1); \
} while (0)
/**
* Skip (as in read and ignore) up to fixed position \p pos.
*/
#define rd_kafka_buf_skip_to(rkbuf, pos) \
do { \
size_t __len1 = \
(size_t)(pos)-rd_slice_offset(&(rkbuf)->rkbuf_reader); \
if (__len1 && \
!rd_slice_read(&(rkbuf)->rkbuf_reader, NULL, __len1)) \
rd_kafka_buf_check_len(rkbuf, __len1); \
} while (0)
/**
* Read 'len' bytes and copy to 'dstptr'
*/
#define rd_kafka_buf_read(rkbuf, dstptr, len) \
do { \
size_t __len2 = (size_t)(len); \
if (!rd_slice_read(&(rkbuf)->rkbuf_reader, dstptr, __len2)) \
rd_kafka_buf_check_len(rkbuf, __len2); \
} while (0)
/**
* @brief Read \p len bytes at slice offset \p offset and copy to \p dstptr
* without affecting the current reader position.
*/
#define rd_kafka_buf_peek(rkbuf, offset, dstptr, len) \
do { \
size_t __len2 = (size_t)(len); \
if (!rd_slice_peek(&(rkbuf)->rkbuf_reader, offset, dstptr, \
__len2)) \
rd_kafka_buf_check_len(rkbuf, (offset) + (__len2)); \
} while (0)
/**
* Read a 16,32,64-bit integer and store it in 'dstptr'
*/
#define rd_kafka_buf_read_i64(rkbuf, dstptr) \
do { \
int64_t _v; \
int64_t *_vp = dstptr; \
rd_kafka_buf_read(rkbuf, &_v, sizeof(_v)); \
*_vp = be64toh(_v); \
} while (0)
#define rd_kafka_buf_peek_i64(rkbuf, of, dstptr) \
do { \
int64_t _v; \
int64_t *_vp = dstptr; \
rd_kafka_buf_peek(rkbuf, of, &_v, sizeof(_v)); \
*_vp = be64toh(_v); \
} while (0)
#define rd_kafka_buf_read_i32(rkbuf, dstptr) \
do { \
int32_t _v; \
int32_t *_vp = dstptr; \
rd_kafka_buf_read(rkbuf, &_v, sizeof(_v)); \
*_vp = be32toh(_v); \
} while (0)
#define rd_kafka_buf_peek_i32(rkbuf, of, dstptr) \
do { \
int32_t _v; \
int32_t *_vp = dstptr; \
rd_kafka_buf_peek(rkbuf, of, &_v, sizeof(_v)); \
*_vp = be32toh(_v); \
} while (0)
/* Same as .._read_i32 but does a direct assignment.
* dst is assumed to be a scalar, not pointer. */
#define rd_kafka_buf_read_i32a(rkbuf, dst) \
do { \
int32_t _v; \
rd_kafka_buf_read(rkbuf, &_v, 4); \
dst = (int32_t)be32toh(_v); \
} while (0)
#define rd_kafka_buf_read_i16(rkbuf, dstptr) \
do { \
int16_t _v; \
int16_t *_vp = dstptr; \
rd_kafka_buf_read(rkbuf, &_v, sizeof(_v)); \
*_vp = (int16_t)be16toh(_v); \
} while (0)
#define rd_kafka_buf_peek_i16(rkbuf, of, dstptr) \
do { \
int16_t _v; \
int16_t *_vp = dstptr; \
rd_kafka_buf_peek(rkbuf, of, &_v, sizeof(_v)); \
*_vp = be16toh(_v); \
} while (0)
#define rd_kafka_buf_read_i16a(rkbuf, dst) \
do { \
int16_t _v; \
rd_kafka_buf_read(rkbuf, &_v, 2); \
dst = (int16_t)be16toh(_v); \
} while (0)
#define rd_kafka_buf_read_i8(rkbuf, dst) rd_kafka_buf_read(rkbuf, dst, 1)
#define rd_kafka_buf_peek_i8(rkbuf, of, dst) \
rd_kafka_buf_peek(rkbuf, of, dst, 1)
#define rd_kafka_buf_read_bool(rkbuf, dstptr) \
do { \
int8_t _v; \
rd_bool_t *_dst = dstptr; \
rd_kafka_buf_read(rkbuf, &_v, 1); \
*_dst = (rd_bool_t)_v; \
} while (0)
/**
* @brief Read varint and store in int64_t \p dst
*/
#define rd_kafka_buf_read_varint(rkbuf, dstptr) \
do { \
int64_t _v; \
int64_t *_vp = dstptr; \
size_t _r = rd_slice_read_varint(&(rkbuf)->rkbuf_reader, &_v); \
if (unlikely(RD_UVARINT_UNDERFLOW(_r))) \
rd_kafka_buf_underflow_fail(rkbuf, (size_t)0, \
"varint parsing failed"); \
*_vp = _v; \
} while (0)
/**
* @brief Read unsigned varint and store in uint64_t \p dst
*/
#define rd_kafka_buf_read_uvarint(rkbuf, dstptr) \
do { \
uint64_t _v; \
uint64_t *_vp = dstptr; \
size_t _r = \
rd_slice_read_uvarint(&(rkbuf)->rkbuf_reader, &_v); \
if (unlikely(RD_UVARINT_UNDERFLOW(_r))) \
rd_kafka_buf_underflow_fail(rkbuf, (size_t)0, \
"uvarint parsing failed"); \
*_vp = _v; \
} while (0)
/**
* @brief Read Kafka COMPACT_STRING (VARINT+N) or
* standard String representation (2+N).
*
* The kstr data will be updated to point to the rkbuf. */
#define rd_kafka_buf_read_str(rkbuf, kstr) \
do { \
int _klen; \
if ((rkbuf)->rkbuf_flags & RD_KAFKA_OP_F_FLEXVER) { \
uint64_t _uva; \
rd_kafka_buf_read_uvarint(rkbuf, &_uva); \
(kstr)->len = ((int32_t)_uva) - 1; \
_klen = (kstr)->len; \
} else { \
rd_kafka_buf_read_i16a(rkbuf, (kstr)->len); \
_klen = RD_KAFKAP_STR_LEN(kstr); \
} \
if (RD_KAFKAP_STR_IS_NULL(kstr)) \
(kstr)->str = NULL; \
else if (RD_KAFKAP_STR_LEN(kstr) == 0) \
(kstr)->str = ""; \
else if (!((kstr)->str = rd_slice_ensure_contig( \
&rkbuf->rkbuf_reader, _klen))) \
rd_kafka_buf_check_len(rkbuf, _klen); \
} while (0)
/* Read Kafka String representation (2+N) and write it to the \p tmpabuf
* with a trailing nul byte. */
#define rd_kafka_buf_read_str_tmpabuf(rkbuf, tmpabuf, dst) \
do { \
rd_kafkap_str_t _kstr; \
size_t _slen; \
char *_dst; \
rd_kafka_buf_read_str(rkbuf, &_kstr); \
if (RD_KAFKAP_STR_IS_NULL(&_kstr)) { \
dst = NULL; \
break; \
} \
_slen = RD_KAFKAP_STR_LEN(&_kstr); \
if (!(_dst = rd_tmpabuf_write(tmpabuf, _kstr.str, _slen + 1))) \
rd_kafka_buf_parse_fail( \
rkbuf, \
"Not enough room in tmpabuf: " \
"%" PRIusz "+%" PRIusz " > %" PRIusz, \
(tmpabuf)->of, _slen + 1, (tmpabuf)->size); \
_dst[_slen] = '\0'; \
dst = (void *)_dst; \
} while (0)
/**
* Skip a string without flexver.
*/
#define rd_kafka_buf_skip_str_no_flexver(rkbuf) \
do { \
int16_t _slen; \
rd_kafka_buf_read_i16(rkbuf, &_slen); \
rd_kafka_buf_skip(rkbuf, RD_KAFKAP_STR_LEN0(_slen)); \
} while (0)
/**
* Skip a string (generic).
*/
#define rd_kafka_buf_skip_str(rkbuf) \
do { \
if ((rkbuf)->rkbuf_flags & RD_KAFKA_OP_F_FLEXVER) { \
uint64_t _uva; \
rd_kafka_buf_read_uvarint(rkbuf, &_uva); \
rd_kafka_buf_skip( \
rkbuf, RD_KAFKAP_STR_LEN0(((int64_t)_uva) - 1)); \
} else { \
rd_kafka_buf_skip_str_no_flexver(rkbuf); \
} \
} while (0)
/**
* Read Kafka COMPACT_BYTES representation (VARINT+N) or
* standard BYTES representation(4+N).
* The 'kbytes' will be updated to point to rkbuf data.
*/
#define rd_kafka_buf_read_kbytes(rkbuf, kbytes) \
do { \
int32_t _klen; \
if (!(rkbuf->rkbuf_flags & RD_KAFKA_OP_F_FLEXVER)) { \
rd_kafka_buf_read_i32a(rkbuf, _klen); \
} else { \
uint64_t _uva; \
rd_kafka_buf_read_uvarint(rkbuf, &_uva); \
_klen = ((int32_t)_uva) - 1; \
} \
(kbytes)->len = _klen; \
if (RD_KAFKAP_BYTES_IS_NULL(kbytes)) { \
(kbytes)->data = NULL; \
(kbytes)->len = 0; \
} else if (RD_KAFKAP_BYTES_LEN(kbytes) == 0) \
(kbytes)->data = ""; \
else if (!((kbytes)->data = rd_slice_ensure_contig( \
&(rkbuf)->rkbuf_reader, _klen))) \
rd_kafka_buf_check_len(rkbuf, _klen); \
} while (0)
/**
* @brief Read \p size bytes from buffer, setting \p *ptr to the start
* of the memory region.
*/
#define rd_kafka_buf_read_ptr(rkbuf, ptr, size) \
do { \
size_t _klen = size; \
if (!(*(ptr) = (void *)rd_slice_ensure_contig( \
&(rkbuf)->rkbuf_reader, _klen))) \
rd_kafka_buf_check_len(rkbuf, _klen); \
} while (0)
/**
* @brief Read varint-lengted Kafka Bytes representation
*/
#define rd_kafka_buf_read_kbytes_varint(rkbuf, kbytes) \
do { \
int64_t _len2; \
size_t _r = \
rd_slice_read_varint(&(rkbuf)->rkbuf_reader, &_len2); \
if (unlikely(RD_UVARINT_UNDERFLOW(_r))) \
rd_kafka_buf_underflow_fail(rkbuf, (size_t)0, \
"varint parsing failed"); \
(kbytes)->len = (int32_t)_len2; \
if (RD_KAFKAP_BYTES_IS_NULL(kbytes)) { \
(kbytes)->data = NULL; \
(kbytes)->len = 0; \
} else if (RD_KAFKAP_BYTES_LEN(kbytes) == 0) \
(kbytes)->data = ""; \
else if (!((kbytes)->data = rd_slice_ensure_contig( \
&(rkbuf)->rkbuf_reader, (size_t)_len2))) \
rd_kafka_buf_check_len(rkbuf, _len2); \
} while (0)
/**
* @brief Read throttle_time_ms (i32) from response and pass the value
* to the throttle handling code.
*/
#define rd_kafka_buf_read_throttle_time(rkbuf) \
do { \
int32_t _throttle_time_ms; \
rd_kafka_buf_read_i32(rkbuf, &_throttle_time_ms); \
rd_kafka_op_throttle_time((rkbuf)->rkbuf_rkb, \
(rkbuf)->rkbuf_rkb->rkb_rk->rk_rep, \
_throttle_time_ms); \
} while (0)
/**
* @brief Discard all KIP-482 Tags at the current position in the buffer.
*/
#define rd_kafka_buf_skip_tags(rkbuf) \
do { \
uint64_t _tagcnt; \
if (!((rkbuf)->rkbuf_flags & RD_KAFKA_OP_F_FLEXVER)) \
break; \
rd_kafka_buf_read_uvarint(rkbuf, &_tagcnt); \
while (_tagcnt-- > 0) { \
uint64_t _tagtype, _taglen; \
rd_kafka_buf_read_uvarint(rkbuf, &_tagtype); \
rd_kafka_buf_read_uvarint(rkbuf, &_taglen); \
if (_taglen > 0) \
rd_kafka_buf_skip(rkbuf, (size_t)(_taglen)); \
} \
} while (0)
/**
* @brief Read KIP-482 Tags at current position in the buffer using
* the `read_tag` function receiving the `opaque' pointer.
*/
#define rd_kafka_buf_read_tags(rkbuf, read_tag, ...) \
do { \
uint64_t _tagcnt; \
if (!((rkbuf)->rkbuf_flags & RD_KAFKA_OP_F_FLEXVER)) \
break; \
rd_kafka_buf_read_uvarint(rkbuf, &_tagcnt); \
while (_tagcnt-- > 0) { \
uint64_t _tagtype, _taglen; \
rd_kafka_buf_read_uvarint(rkbuf, &_tagtype); \
rd_kafka_buf_read_uvarint(rkbuf, &_taglen); \
int _read_tag_resp = \
read_tag(rkbuf, _tagtype, _taglen, __VA_ARGS__); \
if (_read_tag_resp == -1) \
goto err_parse; \
if (!_read_tag_resp && _taglen > 0) \
rd_kafka_buf_skip(rkbuf, (size_t)(_taglen)); \
} \
} while (0)
/**
* @brief Write \p tagcnt tags at the current position in the buffer.
* Calling \p write_tag to write each one with \p rkbuf , tagtype
* argument and the remaining arguments.
*/
#define rd_kafka_buf_write_tags(rkbuf, write_tag, tags, tagcnt, ...) \
do { \
uint64_t i; \
if (!((rkbuf)->rkbuf_flags & RD_KAFKA_OP_F_FLEXVER)) \
break; \
rd_kafka_buf_write_uvarint(rkbuf, tagcnt); \
for (i = 0; i < tagcnt; i++) { \
size_t of_taglen, prev_buf_len; \
rd_kafka_buf_write_uvarint(rkbuf, tags[i]); \
of_taglen = rd_kafka_buf_write_arraycnt_pos(rkbuf); \
prev_buf_len = (rkbuf)->rkbuf_buf.rbuf_len; \
write_tag(rkbuf, tags[i], __VA_ARGS__); \
rd_kafka_buf_finalize_arraycnt( \
rkbuf, of_taglen, \
(rkbuf)->rkbuf_buf.rbuf_len - prev_buf_len - 1); \
} \
} while (0)
/**
* @brief Write empty tags at the current position in the buffer.
*/
#define rd_kafka_buf_write_tags_empty(rkbuf) \
do { \
if (!((rkbuf)->rkbuf_flags & RD_KAFKA_OP_F_FLEXVER)) \
break; \
rd_kafka_buf_write_i8(rkbuf, 0); \
} while (0)
/**
* @brief Reads an ARRAY or COMPACT_ARRAY count depending on buffer type.
*/
#define rd_kafka_buf_read_arraycnt(rkbuf, arrcnt, maxval) \
do { \
if ((rkbuf)->rkbuf_flags & RD_KAFKA_OP_F_FLEXVER) { \
uint64_t _uva; \
rd_kafka_buf_read_uvarint(rkbuf, &_uva); \
*(arrcnt) = (int32_t)_uva - 1; \
} else { \
rd_kafka_buf_read_i32(rkbuf, arrcnt); \
} \
if (*(arrcnt) < -1 || \
((maxval) != -1 && *(arrcnt) > (maxval))) \
rd_kafka_buf_parse_fail( \
rkbuf, "ApiArrayCnt %" PRId32 " out of range", \
*(arrcnt)); \
} while (0)
/**
* @returns true if buffer has been sent on wire, else 0.
*/
#define rd_kafka_buf_was_sent(rkbuf) ((rkbuf)->rkbuf_flags & RD_KAFKA_OP_F_SENT)
typedef struct rd_kafka_bufq_s {
TAILQ_HEAD(, rd_kafka_buf_s) rkbq_bufs;
rd_atomic32_t rkbq_cnt;
rd_atomic32_t rkbq_msg_cnt;
} rd_kafka_bufq_t;
#define rd_kafka_bufq_cnt(rkbq) rd_atomic32_get(&(rkbq)->rkbq_cnt)
/**
* @brief Set buffer's request timeout to relative \p timeout_ms measured
* from the time the buffer is sent on the underlying socket.
*
* @param now Reuse current time from existing rd_clock() var, else 0.
*
* The relative timeout value is reused upon request retry.
*/
static RD_INLINE void
rd_kafka_buf_set_timeout(rd_kafka_buf_t *rkbuf, int timeout_ms, rd_ts_t now) {
if (!now)
now = rd_clock();
rkbuf->rkbuf_rel_timeout = timeout_ms;
rkbuf->rkbuf_abs_timeout = 0;
}
/**
* @brief Calculate the effective timeout for a request attempt
*/
void rd_kafka_buf_calc_timeout(const rd_kafka_t *rk,
rd_kafka_buf_t *rkbuf,
rd_ts_t now);
/**
* @brief Set buffer's request timeout to relative \p timeout_ms measured
* from \p now.
*
* @param now Reuse current time from existing rd_clock() var, else 0.
* @param force If true: force request timeout to be same as remaining
* abs timeout, regardless of socket.timeout.ms.
* If false: cap each request timeout to socket.timeout.ms.
*
* The remaining time is used as timeout for request retries.
*/
static RD_INLINE void rd_kafka_buf_set_abs_timeout0(rd_kafka_buf_t *rkbuf,
int timeout_ms,
rd_ts_t now,
rd_bool_t force) {
if (!now)
now = rd_clock();
rkbuf->rkbuf_rel_timeout = 0;
rkbuf->rkbuf_abs_timeout = now + ((rd_ts_t)timeout_ms * 1000);
rkbuf->rkbuf_force_timeout = force;
}
#define rd_kafka_buf_set_abs_timeout(rkbuf, timeout_ms, now) \
rd_kafka_buf_set_abs_timeout0(rkbuf, timeout_ms, now, rd_false)
#define rd_kafka_buf_set_abs_timeout_force(rkbuf, timeout_ms, now) \
rd_kafka_buf_set_abs_timeout0(rkbuf, timeout_ms, now, rd_true)
#define rd_kafka_buf_keep(rkbuf) rd_refcnt_add(&(rkbuf)->rkbuf_refcnt)
#define rd_kafka_buf_destroy(rkbuf) \
rd_refcnt_destroywrapper(&(rkbuf)->rkbuf_refcnt, \
rd_kafka_buf_destroy_final(rkbuf))
void rd_kafka_buf_destroy_final(rd_kafka_buf_t *rkbuf);
void rd_kafka_buf_push0(rd_kafka_buf_t *rkbuf,
const void *buf,
size_t len,
int allow_crc_calc,
void (*free_cb)(void *));
#define rd_kafka_buf_push(rkbuf, buf, len, free_cb) \
rd_kafka_buf_push0(rkbuf, buf, len, 1 /*allow_crc*/, free_cb)
rd_kafka_buf_t *rd_kafka_buf_new0(int segcnt, size_t size, int flags);
#define rd_kafka_buf_new(segcnt, size) rd_kafka_buf_new0(segcnt, size, 0)