-
Notifications
You must be signed in to change notification settings - Fork 3.2k
/
rdkafka_request.c
5630 lines (4707 loc) · 222 KB
/
rdkafka_request.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.
*/
#include <stdarg.h>
#include "rdkafka_int.h"
#include "rdkafka_request.h"
#include "rdkafka_broker.h"
#include "rdkafka_offset.h"
#include "rdkafka_topic.h"
#include "rdkafka_partition.h"
#include "rdkafka_metadata.h"
#include "rdkafka_msgset.h"
#include "rdkafka_idempotence.h"
#include "rdkafka_txnmgr.h"
#include "rdkafka_sasl.h"
#include "rdrand.h"
#include "rdstring.h"
#include "rdunittest.h"
/**
* Kafka protocol request and response handling.
* All of this code runs in the broker thread and uses op queues for
* propagating results back to the various sub-systems operating in
* other threads.
*/
/* RD_KAFKA_ERR_ACTION_.. to string map */
static const char *rd_kafka_actions_descs[] = {
"Permanent", "Ignore", "Refresh", "Retry",
"Inform", "Special", "MsgNotPersisted", "MsgPossiblyPersisted",
"MsgPersisted", NULL,
};
const char *rd_kafka_actions2str(int actions) {
static RD_TLS char actstr[128];
return rd_flags2str(actstr, sizeof(actstr), rd_kafka_actions_descs,
actions);
}
/**
* @brief Decide action(s) to take based on the returned error code.
*
* The optional var-args is a .._ACTION_END terminated list
* of action,error tuples which overrides the general behaviour.
* It is to be read as: for \p error, return \p action(s).
*
* @warning \p request, \p rkbuf and \p rkb may be NULL.
*/
int rd_kafka_err_action(rd_kafka_broker_t *rkb,
rd_kafka_resp_err_t err,
const rd_kafka_buf_t *request,
...) {
va_list ap;
int actions = 0;
int exp_act;
if (!err)
return 0;
/* Match explicitly defined error mappings first. */
va_start(ap, request);
while ((exp_act = va_arg(ap, int))) {
int exp_err = va_arg(ap, int);
if (err == exp_err)
actions |= exp_act;
}
va_end(ap);
/* Explicit error match. */
if (actions) {
if (err && rkb && request)
rd_rkb_dbg(
rkb, BROKER, "REQERR",
"%sRequest failed: %s: explicit actions %s",
rd_kafka_ApiKey2str(request->rkbuf_reqhdr.ApiKey),
rd_kafka_err2str(err),
rd_kafka_actions2str(actions));
return actions;
}
/* Default error matching */
switch (err) {
case RD_KAFKA_RESP_ERR_NO_ERROR:
break;
case RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE:
case RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION:
case RD_KAFKA_RESP_ERR_BROKER_NOT_AVAILABLE:
case RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE:
case RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE:
case RD_KAFKA_RESP_ERR_NOT_COORDINATOR:
case RD_KAFKA_RESP_ERR__WAIT_COORD:
/* Request metadata information update */
actions |= RD_KAFKA_ERR_ACTION_REFRESH |
RD_KAFKA_ERR_ACTION_MSG_NOT_PERSISTED;
break;
case RD_KAFKA_RESP_ERR_KAFKA_STORAGE_ERROR:
/* Request metadata update and retry */
actions |= RD_KAFKA_ERR_ACTION_REFRESH |
RD_KAFKA_ERR_ACTION_RETRY |
RD_KAFKA_ERR_ACTION_MSG_NOT_PERSISTED;
break;
case RD_KAFKA_RESP_ERR__TRANSPORT:
case RD_KAFKA_RESP_ERR__SSL:
case RD_KAFKA_RESP_ERR__TIMED_OUT:
case RD_KAFKA_RESP_ERR_REQUEST_TIMED_OUT:
case RD_KAFKA_RESP_ERR_NOT_ENOUGH_REPLICAS_AFTER_APPEND:
actions |= RD_KAFKA_ERR_ACTION_RETRY |
RD_KAFKA_ERR_ACTION_MSG_POSSIBLY_PERSISTED;
break;
case RD_KAFKA_RESP_ERR_NOT_ENOUGH_REPLICAS:
/* Client-side wait-response/in-queue timeout */
case RD_KAFKA_RESP_ERR__TIMED_OUT_QUEUE:
actions |= RD_KAFKA_ERR_ACTION_RETRY |
RD_KAFKA_ERR_ACTION_MSG_NOT_PERSISTED;
break;
case RD_KAFKA_RESP_ERR__PURGE_INFLIGHT:
actions |= RD_KAFKA_ERR_ACTION_PERMANENT |
RD_KAFKA_ERR_ACTION_MSG_POSSIBLY_PERSISTED;
break;
case RD_KAFKA_RESP_ERR__BAD_MSG:
/* Buffer parse failures are typically a client-side bug,
* treat them as permanent failures. */
actions |= RD_KAFKA_ERR_ACTION_PERMANENT |
RD_KAFKA_ERR_ACTION_MSG_POSSIBLY_PERSISTED;
break;
case RD_KAFKA_RESP_ERR_COORDINATOR_LOAD_IN_PROGRESS:
actions |= RD_KAFKA_ERR_ACTION_RETRY;
break;
case RD_KAFKA_RESP_ERR__DESTROY:
case RD_KAFKA_RESP_ERR_INVALID_SESSION_TIMEOUT:
case RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE:
case RD_KAFKA_RESP_ERR__PURGE_QUEUE:
default:
actions |= RD_KAFKA_ERR_ACTION_PERMANENT |
RD_KAFKA_ERR_ACTION_MSG_NOT_PERSISTED;
break;
}
/* Fatal or permanent errors are not retriable */
if (actions &
(RD_KAFKA_ERR_ACTION_FATAL | RD_KAFKA_ERR_ACTION_PERMANENT))
actions &= ~RD_KAFKA_ERR_ACTION_RETRY;
/* If no request buffer was specified, which might be the case
* in certain error call chains, mask out the retry action. */
if (!request)
actions &= ~RD_KAFKA_ERR_ACTION_RETRY;
else if (request->rkbuf_reqhdr.ApiKey != RD_KAFKAP_Produce)
/* Mask out message-related bits for non-Produce requests */
actions &= ~RD_KAFKA_ERR_ACTION_MSG_FLAGS;
if (err && actions && rkb && request)
rd_rkb_dbg(
rkb, BROKER, "REQERR", "%sRequest failed: %s: actions %s",
rd_kafka_ApiKey2str(request->rkbuf_reqhdr.ApiKey),
rd_kafka_err2str(err), rd_kafka_actions2str(actions));
return actions;
}
/**
* @brief Read a list of topic+partitions+extra from \p rkbuf.
*
* @param rkbuf buffer to read from
* @param fields An array of fields to read from the buffer and set on
* the rktpar object, in the specified order, must end
* with RD_KAFKA_TOPIC_PARTITION_FIELD_END.
*
* @returns a newly allocated list on success, or NULL on parse error.
*/
rd_kafka_topic_partition_list_t *rd_kafka_buf_read_topic_partitions(
rd_kafka_buf_t *rkbuf,
size_t estimated_part_cnt,
const rd_kafka_topic_partition_field_t *fields) {
const int log_decode_errors = LOG_ERR;
int32_t TopicArrayCnt;
rd_kafka_topic_partition_list_t *parts = NULL;
rd_kafka_buf_read_arraycnt(rkbuf, &TopicArrayCnt, RD_KAFKAP_TOPICS_MAX);
parts = rd_kafka_topic_partition_list_new(
RD_MAX(TopicArrayCnt * 4, (int)estimated_part_cnt));
while (TopicArrayCnt-- > 0) {
rd_kafkap_str_t kTopic;
int32_t PartArrayCnt;
char *topic;
rd_kafka_buf_read_str(rkbuf, &kTopic);
rd_kafka_buf_read_arraycnt(rkbuf, &PartArrayCnt,
RD_KAFKAP_PARTITIONS_MAX);
RD_KAFKAP_STR_DUPA(&topic, &kTopic);
while (PartArrayCnt-- > 0) {
int32_t Partition = -1, Epoch = -1234,
CurrentLeaderEpoch = -1234;
int64_t Offset = -1234;
int16_t ErrorCode = 0;
rd_kafka_topic_partition_t *rktpar;
int fi;
/*
* Read requested fields
*/
for (fi = 0;
fields[fi] != RD_KAFKA_TOPIC_PARTITION_FIELD_END;
fi++) {
switch (fields[fi]) {
case RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION:
rd_kafka_buf_read_i32(rkbuf,
&Partition);
break;
case RD_KAFKA_TOPIC_PARTITION_FIELD_OFFSET:
rd_kafka_buf_read_i64(rkbuf, &Offset);
break;
case RD_KAFKA_TOPIC_PARTITION_FIELD_CURRENT_EPOCH:
rd_kafka_buf_read_i32(
rkbuf, &CurrentLeaderEpoch);
break;
case RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH:
rd_kafka_buf_read_i32(rkbuf, &Epoch);
break;
case RD_KAFKA_TOPIC_PARTITION_FIELD_ERR:
rd_kafka_buf_read_i16(rkbuf,
&ErrorCode);
break;
case RD_KAFKA_TOPIC_PARTITION_FIELD_METADATA:
rd_assert(!*"metadata not implemented");
break;
case RD_KAFKA_TOPIC_PARTITION_FIELD_NOOP:
break;
case RD_KAFKA_TOPIC_PARTITION_FIELD_END:
break;
}
}
rktpar = rd_kafka_topic_partition_list_add(parts, topic,
Partition);
/* Use dummy sentinel values that are unlikely to be
* seen from the broker to know if we are to set these
* fields or not. */
if (Offset != -1234)
rktpar->offset = Offset;
if (Epoch != -1234)
rd_kafka_topic_partition_set_leader_epoch(
rktpar, Epoch);
if (CurrentLeaderEpoch != -1234)
rd_kafka_topic_partition_set_current_leader_epoch(
rktpar, CurrentLeaderEpoch);
rktpar->err = ErrorCode;
rd_kafka_buf_skip_tags(rkbuf);
}
rd_kafka_buf_skip_tags(rkbuf);
}
return parts;
err_parse:
if (parts)
rd_kafka_topic_partition_list_destroy(parts);
return NULL;
}
/**
* @brief Write a list of topic+partitions+offsets+extra to \p rkbuf
*
* @returns the number of partitions written to buffer.
*
* @remark The \p parts list MUST be sorted.
*/
int rd_kafka_buf_write_topic_partitions(
rd_kafka_buf_t *rkbuf,
const rd_kafka_topic_partition_list_t *parts,
rd_bool_t skip_invalid_offsets,
rd_bool_t only_invalid_offsets,
const rd_kafka_topic_partition_field_t *fields) {
size_t of_TopicArrayCnt;
size_t of_PartArrayCnt = 0;
int TopicArrayCnt = 0, PartArrayCnt = 0;
int i;
const char *prev_topic = NULL;
int cnt = 0;
rd_assert(!only_invalid_offsets ||
(only_invalid_offsets != skip_invalid_offsets));
/* TopicArrayCnt */
of_TopicArrayCnt = rd_kafka_buf_write_arraycnt_pos(rkbuf);
for (i = 0; i < parts->cnt; i++) {
const rd_kafka_topic_partition_t *rktpar = &parts->elems[i];
int fi;
if (rktpar->offset < 0) {
if (skip_invalid_offsets)
continue;
} else if (only_invalid_offsets)
continue;
if (!prev_topic || strcmp(rktpar->topic, prev_topic)) {
/* Finish previous topic, if any. */
if (of_PartArrayCnt > 0) {
rd_kafka_buf_finalize_arraycnt(
rkbuf, of_PartArrayCnt, PartArrayCnt);
/* Tags for previous topic struct */
rd_kafka_buf_write_tags(rkbuf);
}
/* Topic */
rd_kafka_buf_write_str(rkbuf, rktpar->topic, -1);
TopicArrayCnt++;
prev_topic = rktpar->topic;
/* New topic so reset partition count */
PartArrayCnt = 0;
/* PartitionArrayCnt: updated later */
of_PartArrayCnt =
rd_kafka_buf_write_arraycnt_pos(rkbuf);
}
/*
* Write requested fields
*/
for (fi = 0; fields[fi] != RD_KAFKA_TOPIC_PARTITION_FIELD_END;
fi++) {
switch (fields[fi]) {
case RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION:
rd_kafka_buf_write_i32(rkbuf,
rktpar->partition);
break;
case RD_KAFKA_TOPIC_PARTITION_FIELD_OFFSET:
rd_kafka_buf_write_i64(rkbuf, rktpar->offset);
break;
case RD_KAFKA_TOPIC_PARTITION_FIELD_CURRENT_EPOCH:
rd_kafka_buf_write_i32(
rkbuf,
rd_kafka_topic_partition_get_current_leader_epoch(
rktpar));
break;
case RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH:
rd_kafka_buf_write_i32(
rkbuf,
rd_kafka_topic_partition_get_leader_epoch(
rktpar));
break;
case RD_KAFKA_TOPIC_PARTITION_FIELD_ERR:
rd_kafka_buf_write_i16(rkbuf, rktpar->err);
break;
case RD_KAFKA_TOPIC_PARTITION_FIELD_METADATA:
/* Java client 0.9.0 and broker <0.10.0 can't
* parse Null metadata fields, so as a
* workaround we send an empty string if
* it's Null. */
if (!rktpar->metadata)
rd_kafka_buf_write_str(rkbuf, "", 0);
else
rd_kafka_buf_write_str(
rkbuf, rktpar->metadata,
rktpar->metadata_size);
break;
case RD_KAFKA_TOPIC_PARTITION_FIELD_NOOP:
break;
case RD_KAFKA_TOPIC_PARTITION_FIELD_END:
break;
}
}
if (fi > 1)
/* If there was more than one field written
* then this was a struct and thus needs the
* struct suffix tags written. */
rd_kafka_buf_write_tags(rkbuf);
PartArrayCnt++;
cnt++;
}
if (of_PartArrayCnt > 0) {
rd_kafka_buf_finalize_arraycnt(rkbuf, of_PartArrayCnt,
PartArrayCnt);
/* Tags for topic struct */
rd_kafka_buf_write_tags(rkbuf);
}
rd_kafka_buf_finalize_arraycnt(rkbuf, of_TopicArrayCnt, TopicArrayCnt);
return cnt;
}
/**
* @brief Send FindCoordinatorRequest.
*
* @param coordkey is the group.id for RD_KAFKA_COORD_GROUP,
* and the transactional.id for RD_KAFKA_COORD_TXN
*/
rd_kafka_resp_err_t
rd_kafka_FindCoordinatorRequest(rd_kafka_broker_t *rkb,
rd_kafka_coordtype_t coordtype,
const char *coordkey,
rd_kafka_replyq_t replyq,
rd_kafka_resp_cb_t *resp_cb,
void *opaque) {
rd_kafka_buf_t *rkbuf;
int16_t ApiVersion;
ApiVersion = rd_kafka_broker_ApiVersion_supported(
rkb, RD_KAFKAP_FindCoordinator, 0, 2, NULL);
if (coordtype != RD_KAFKA_COORD_GROUP && ApiVersion < 1)
return RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE;
rkbuf = rd_kafka_buf_new_request(rkb, RD_KAFKAP_FindCoordinator, 1,
1 + 2 + strlen(coordkey));
rd_kafka_buf_write_str(rkbuf, coordkey, -1);
if (ApiVersion >= 1)
rd_kafka_buf_write_i8(rkbuf, (int8_t)coordtype);
rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, 0);
rd_kafka_broker_buf_enq_replyq(rkb, rkbuf, replyq, resp_cb, opaque);
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
/**
* @brief Parses a ListOffsets reply.
*
* Returns the parsed offsets (and errors) in \p offsets which must have been
* initialized by caller.
*
* @returns 0 on success, else an error (\p offsets may be completely or
* partially updated, depending on the nature of the error, and per
* partition error codes should be checked by the caller).
*/
static rd_kafka_resp_err_t
rd_kafka_parse_ListOffsets(rd_kafka_buf_t *rkbuf,
rd_kafka_topic_partition_list_t *offsets) {
const int log_decode_errors = LOG_ERR;
int32_t TopicArrayCnt;
int16_t api_version;
rd_kafka_resp_err_t all_err = RD_KAFKA_RESP_ERR_NO_ERROR;
api_version = rkbuf->rkbuf_reqhdr.ApiVersion;
if (api_version >= 2)
rd_kafka_buf_read_throttle_time(rkbuf);
/* NOTE:
* Broker may return offsets in a different constellation than
* in the original request .*/
rd_kafka_buf_read_i32(rkbuf, &TopicArrayCnt);
while (TopicArrayCnt-- > 0) {
rd_kafkap_str_t ktopic;
int32_t PartArrayCnt;
char *topic_name;
rd_kafka_buf_read_str(rkbuf, &ktopic);
rd_kafka_buf_read_i32(rkbuf, &PartArrayCnt);
RD_KAFKAP_STR_DUPA(&topic_name, &ktopic);
while (PartArrayCnt-- > 0) {
int32_t kpartition;
int16_t ErrorCode;
int32_t OffsetArrayCnt;
int64_t Offset = -1;
int32_t LeaderEpoch = -1;
rd_kafka_topic_partition_t *rktpar;
rd_kafka_buf_read_i32(rkbuf, &kpartition);
rd_kafka_buf_read_i16(rkbuf, &ErrorCode);
if (api_version >= 1) {
int64_t Timestamp;
rd_kafka_buf_read_i64(rkbuf, &Timestamp);
rd_kafka_buf_read_i64(rkbuf, &Offset);
if (api_version >= 4)
rd_kafka_buf_read_i32(rkbuf,
&LeaderEpoch);
} else if (api_version == 0) {
rd_kafka_buf_read_i32(rkbuf, &OffsetArrayCnt);
/* We only request one offset so just grab
* the first one. */
while (OffsetArrayCnt-- > 0)
rd_kafka_buf_read_i64(rkbuf, &Offset);
} else {
RD_NOTREACHED();
}
rktpar = rd_kafka_topic_partition_list_add(
offsets, topic_name, kpartition);
rktpar->err = ErrorCode;
rktpar->offset = Offset;
rd_kafka_topic_partition_set_leader_epoch(rktpar,
LeaderEpoch);
if (ErrorCode && !all_err)
all_err = ErrorCode;
}
}
return all_err;
err_parse:
return rkbuf->rkbuf_err;
}
/**
* @brief Parses and handles ListOffsets replies.
*
* Returns the parsed offsets (and errors) in \p offsets.
* \p offsets must be initialized by the caller.
*
* @returns 0 on success, else an error. \p offsets may be populated on error,
* depending on the nature of the error.
* On error \p actionsp (unless NULL) is updated with the recommended
* error actions.
*/
rd_kafka_resp_err_t
rd_kafka_handle_ListOffsets(rd_kafka_t *rk,
rd_kafka_broker_t *rkb,
rd_kafka_resp_err_t err,
rd_kafka_buf_t *rkbuf,
rd_kafka_buf_t *request,
rd_kafka_topic_partition_list_t *offsets,
int *actionsp) {
int actions;
if (!err)
err = rd_kafka_parse_ListOffsets(rkbuf, offsets);
if (!err)
return RD_KAFKA_RESP_ERR_NO_ERROR;
actions = rd_kafka_err_action(
rkb, err, request, RD_KAFKA_ERR_ACTION_PERMANENT,
RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART,
RD_KAFKA_ERR_ACTION_REFRESH,
RD_KAFKA_RESP_ERR_NOT_LEADER_FOR_PARTITION,
RD_KAFKA_ERR_ACTION_REFRESH,
RD_KAFKA_RESP_ERR_REPLICA_NOT_AVAILABLE,
RD_KAFKA_ERR_ACTION_REFRESH, RD_KAFKA_RESP_ERR_KAFKA_STORAGE_ERROR,
RD_KAFKA_ERR_ACTION_REFRESH, RD_KAFKA_RESP_ERR_OFFSET_NOT_AVAILABLE,
RD_KAFKA_ERR_ACTION_REFRESH | RD_KAFKA_ERR_ACTION_RETRY,
RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE,
RD_KAFKA_ERR_ACTION_REFRESH | RD_KAFKA_ERR_ACTION_RETRY,
RD_KAFKA_RESP_ERR_FENCED_LEADER_EPOCH,
RD_KAFKA_ERR_ACTION_REFRESH | RD_KAFKA_ERR_ACTION_RETRY,
RD_KAFKA_RESP_ERR_UNKNOWN_LEADER_EPOCH,
RD_KAFKA_ERR_ACTION_RETRY, RD_KAFKA_RESP_ERR__TRANSPORT,
RD_KAFKA_ERR_ACTION_RETRY, RD_KAFKA_RESP_ERR_REQUEST_TIMED_OUT,
RD_KAFKA_ERR_ACTION_END);
if (actionsp)
*actionsp = actions;
if (rkb)
rd_rkb_dbg(
rkb, TOPIC, "OFFSET", "OffsetRequest failed: %s (%s)",
rd_kafka_err2str(err), rd_kafka_actions2str(actions));
if (actions & RD_KAFKA_ERR_ACTION_REFRESH) {
char tmp[256];
/* Re-query for leader */
rd_snprintf(tmp, sizeof(tmp), "ListOffsetsRequest failed: %s",
rd_kafka_err2str(err));
rd_kafka_metadata_refresh_known_topics(rk, NULL,
rd_true /*force*/, tmp);
}
if ((actions & RD_KAFKA_ERR_ACTION_RETRY) &&
rd_kafka_buf_retry(rkb, request))
return RD_KAFKA_RESP_ERR__IN_PROGRESS;
return err;
}
/**
* @brief Async maker for ListOffsetsRequest.
*/
static rd_kafka_resp_err_t
rd_kafka_make_ListOffsetsRequest(rd_kafka_broker_t *rkb,
rd_kafka_buf_t *rkbuf,
void *make_opaque) {
const rd_kafka_topic_partition_list_t *partitions =
(const rd_kafka_topic_partition_list_t *)make_opaque;
int i;
size_t of_TopicArrayCnt = 0, of_PartArrayCnt = 0;
const char *last_topic = "";
int32_t topic_cnt = 0, part_cnt = 0;
int16_t ApiVersion;
ApiVersion = rd_kafka_broker_ApiVersion_supported(
rkb, RD_KAFKAP_ListOffsets, 0, 5, NULL);
if (ApiVersion == -1)
return RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE;
/* ReplicaId */
rd_kafka_buf_write_i32(rkbuf, -1);
/* IsolationLevel */
if (ApiVersion >= 2)
rd_kafka_buf_write_i8(rkbuf,
rkb->rkb_rk->rk_conf.isolation_level);
/* TopicArrayCnt */
of_TopicArrayCnt = rd_kafka_buf_write_i32(rkbuf, 0); /* updated later */
for (i = 0; i < partitions->cnt; i++) {
const rd_kafka_topic_partition_t *rktpar =
&partitions->elems[i];
if (strcmp(rktpar->topic, last_topic)) {
/* Finish last topic, if any. */
if (of_PartArrayCnt > 0)
rd_kafka_buf_update_i32(rkbuf, of_PartArrayCnt,
part_cnt);
/* Topic */
rd_kafka_buf_write_str(rkbuf, rktpar->topic, -1);
topic_cnt++;
last_topic = rktpar->topic;
/* New topic so reset partition count */
part_cnt = 0;
/* PartitionArrayCnt: updated later */
of_PartArrayCnt = rd_kafka_buf_write_i32(rkbuf, 0);
}
/* Partition */
rd_kafka_buf_write_i32(rkbuf, rktpar->partition);
part_cnt++;
if (ApiVersion >= 4)
/* CurrentLeaderEpoch */
rd_kafka_buf_write_i32(
rkbuf,
rd_kafka_topic_partition_get_current_leader_epoch(
rktpar));
/* Time/Offset */
rd_kafka_buf_write_i64(rkbuf, rktpar->offset);
if (ApiVersion == 0) {
/* MaxNumberOfOffsets */
rd_kafka_buf_write_i32(rkbuf, 1);
}
}
if (of_PartArrayCnt > 0) {
rd_kafka_buf_update_i32(rkbuf, of_PartArrayCnt, part_cnt);
rd_kafka_buf_update_i32(rkbuf, of_TopicArrayCnt, topic_cnt);
}
rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, 0);
rd_rkb_dbg(rkb, TOPIC, "OFFSET",
"ListOffsetsRequest (v%hd, opv %d) "
"for %" PRId32 " topic(s) and %" PRId32 " partition(s)",
ApiVersion, rkbuf->rkbuf_replyq.version, topic_cnt,
partitions->cnt);
return RD_KAFKA_RESP_ERR_NO_ERROR;
}
/**
* @brief Send ListOffsetsRequest for partitions in \p partitions.
*/
void rd_kafka_ListOffsetsRequest(rd_kafka_broker_t *rkb,
rd_kafka_topic_partition_list_t *partitions,
rd_kafka_replyq_t replyq,
rd_kafka_resp_cb_t *resp_cb,
void *opaque) {
rd_kafka_buf_t *rkbuf;
rd_kafka_topic_partition_list_t *make_parts;
make_parts = rd_kafka_topic_partition_list_copy(partitions);
rd_kafka_topic_partition_list_sort_by_topic(make_parts);
rkbuf = rd_kafka_buf_new_request(
rkb, RD_KAFKAP_ListOffsets, 1,
/* ReplicaId+IsolationLevel+TopicArrayCnt+Topic */
4 + 1 + 4 + 100 +
/* PartArrayCnt */
4 +
/* partition_cnt * Partition+Time+MaxNumOffs */
(make_parts->cnt * (4 + 8 + 4)));
/* Postpone creating the request contents until time to send,
* at which time the ApiVersion is known. */
rd_kafka_buf_set_maker(rkbuf, rd_kafka_make_ListOffsetsRequest,
make_parts,
rd_kafka_topic_partition_list_destroy_free);
rd_kafka_broker_buf_enq_replyq(rkb, rkbuf, replyq, resp_cb, opaque);
}
/**
* @brief OffsetForLeaderEpochResponse handler.
*/
rd_kafka_resp_err_t rd_kafka_handle_OffsetForLeaderEpoch(
rd_kafka_t *rk,
rd_kafka_broker_t *rkb,
rd_kafka_resp_err_t err,
rd_kafka_buf_t *rkbuf,
rd_kafka_buf_t *request,
rd_kafka_topic_partition_list_t **offsets) {
const int log_decode_errors = LOG_ERR;
int16_t ApiVersion;
if (err)
goto err;
ApiVersion = rkbuf->rkbuf_reqhdr.ApiVersion;
if (ApiVersion >= 2)
rd_kafka_buf_read_throttle_time(rkbuf);
const rd_kafka_topic_partition_field_t fields[] = {
RD_KAFKA_TOPIC_PARTITION_FIELD_ERR,
RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION,
ApiVersion >= 1 ? RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH
: RD_KAFKA_TOPIC_PARTITION_FIELD_NOOP,
RD_KAFKA_TOPIC_PARTITION_FIELD_OFFSET,
RD_KAFKA_TOPIC_PARTITION_FIELD_END};
*offsets = rd_kafka_buf_read_topic_partitions(rkbuf, 0, fields);
if (!*offsets)
goto err_parse;
return RD_KAFKA_RESP_ERR_NO_ERROR;
err:
return err;
err_parse:
err = rkbuf->rkbuf_err;
goto err;
}
/**
* @brief Send OffsetForLeaderEpochRequest for partition(s).
*
*/
void rd_kafka_OffsetForLeaderEpochRequest(
rd_kafka_broker_t *rkb,
rd_kafka_topic_partition_list_t *parts,
rd_kafka_replyq_t replyq,
rd_kafka_resp_cb_t *resp_cb,
void *opaque) {
rd_kafka_buf_t *rkbuf;
int16_t ApiVersion;
ApiVersion = rd_kafka_broker_ApiVersion_supported(
rkb, RD_KAFKAP_OffsetForLeaderEpoch, 2, 2, NULL);
/* If the supported ApiVersions are not yet known,
* or this broker doesn't support it, we let this request
* succeed or fail later from the broker thread where the
* version is checked again. */
if (ApiVersion == -1)
ApiVersion = 2;
rkbuf = rd_kafka_buf_new_flexver_request(
rkb, RD_KAFKAP_OffsetForLeaderEpoch, 1, 4 + (parts->cnt * 64),
ApiVersion >= 4 /*flexver*/);
/* Sort partitions by topic */
rd_kafka_topic_partition_list_sort_by_topic(parts);
/* Write partition list */
const rd_kafka_topic_partition_field_t fields[] = {
RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION,
/* CurrentLeaderEpoch */
RD_KAFKA_TOPIC_PARTITION_FIELD_CURRENT_EPOCH,
/* LeaderEpoch */
RD_KAFKA_TOPIC_PARTITION_FIELD_EPOCH,
RD_KAFKA_TOPIC_PARTITION_FIELD_END};
rd_kafka_buf_write_topic_partitions(
rkbuf, parts, rd_false /*include invalid offsets*/,
rd_false /*skip valid offsets */, fields);
rd_kafka_buf_ApiVersion_set(rkbuf, ApiVersion, 0);
/* Let caller perform retries */
rkbuf->rkbuf_max_retries = RD_KAFKA_REQUEST_NO_RETRIES;
rd_kafka_broker_buf_enq_replyq(rkb, rkbuf, replyq, resp_cb, opaque);
}
/**
* Generic handler for OffsetFetch responses.
* Offsets for included partitions will be propagated through the passed
* 'offsets' list.
*
* @param rkbuf response buffer, may be NULL if \p err is set.
* @param update_toppar update toppar's committed_offset
* @param add_part if true add partitions from the response to \p *offsets,
* else just update the partitions that are already
* in \p *offsets.
*/
rd_kafka_resp_err_t
rd_kafka_handle_OffsetFetch(rd_kafka_t *rk,
rd_kafka_broker_t *rkb,
rd_kafka_resp_err_t err,
rd_kafka_buf_t *rkbuf,
rd_kafka_buf_t *request,
rd_kafka_topic_partition_list_t **offsets,
rd_bool_t update_toppar,
rd_bool_t add_part,
rd_bool_t allow_retry) {
const int log_decode_errors = LOG_ERR;
int32_t TopicArrayCnt;
int64_t offset = RD_KAFKA_OFFSET_INVALID;
int16_t ApiVersion;
rd_kafkap_str_t metadata;
int retry_unstable = 0;
int i;
int actions;
int seen_cnt = 0;
if (err)
goto err;
ApiVersion = rkbuf->rkbuf_reqhdr.ApiVersion;
if (ApiVersion >= 3)
rd_kafka_buf_read_throttle_time(rkbuf);
if (!*offsets)
*offsets = rd_kafka_topic_partition_list_new(16);
/* Set default offset for all partitions. */
rd_kafka_topic_partition_list_set_offsets(rkb->rkb_rk, *offsets, 0,
RD_KAFKA_OFFSET_INVALID,
0 /* !is commit */);
rd_kafka_buf_read_arraycnt(rkbuf, &TopicArrayCnt, RD_KAFKAP_TOPICS_MAX);
for (i = 0; i < TopicArrayCnt; i++) {
rd_kafkap_str_t topic;
int32_t PartArrayCnt;
char *topic_name;
int j;
rd_kafka_buf_read_str(rkbuf, &topic);
rd_kafka_buf_read_arraycnt(rkbuf, &PartArrayCnt,
RD_KAFKAP_PARTITIONS_MAX);
RD_KAFKAP_STR_DUPA(&topic_name, &topic);
for (j = 0; j < PartArrayCnt; j++) {
int32_t partition;
rd_kafka_toppar_t *rktp;
rd_kafka_topic_partition_t *rktpar;
int32_t LeaderEpoch = -1;
int16_t err2;
rd_kafka_buf_read_i32(rkbuf, &partition);
rd_kafka_buf_read_i64(rkbuf, &offset);
if (ApiVersion >= 5)
rd_kafka_buf_read_i32(rkbuf, &LeaderEpoch);
rd_kafka_buf_read_str(rkbuf, &metadata);
rd_kafka_buf_read_i16(rkbuf, &err2);
rd_kafka_buf_skip_tags(rkbuf);
rktpar = rd_kafka_topic_partition_list_find(
*offsets, topic_name, partition);
if (!rktpar && add_part)
rktpar = rd_kafka_topic_partition_list_add(
*offsets, topic_name, partition);
else if (!rktpar) {
rd_rkb_dbg(rkb, TOPIC, "OFFSETFETCH",
"OffsetFetchResponse: %s [%" PRId32
"] "
"not found in local list: ignoring",
topic_name, partition);
continue;
}
seen_cnt++;
rktp = rd_kafka_topic_partition_get_toppar(
rk, rktpar, rd_false /*no create on miss*/);
/* broker reports invalid offset as -1 */
if (offset == -1)
rktpar->offset = RD_KAFKA_OFFSET_INVALID;
else
rktpar->offset = offset;
rd_kafka_topic_partition_set_leader_epoch(rktpar,
LeaderEpoch);
rktpar->err = err2;
rd_rkb_dbg(rkb, TOPIC, "OFFSETFETCH",
"OffsetFetchResponse: %s [%" PRId32
"] "
"offset %" PRId64 ", leader epoch %" PRId32
", metadata %d byte(s): %s",
topic_name, partition, offset, LeaderEpoch,
RD_KAFKAP_STR_LEN(&metadata),
rd_kafka_err2name(rktpar->err));
if (update_toppar && !err2 && rktp) {
/* Update toppar's committed offset */
rd_kafka_toppar_lock(rktp);
rktp->rktp_committed_pos =
rd_kafka_topic_partition_get_fetch_pos(
rktpar);
rd_kafka_toppar_unlock(rktp);
}
if (rktpar->err ==
RD_KAFKA_RESP_ERR_UNSTABLE_OFFSET_COMMIT)
retry_unstable++;
if (rktpar->metadata)
rd_free(rktpar->metadata);
if (RD_KAFKAP_STR_IS_NULL(&metadata)) {
rktpar->metadata = NULL;
rktpar->metadata_size = 0;
} else {
rktpar->metadata = RD_KAFKAP_STR_DUP(&metadata);