summaryrefslogtreecommitdiffstats
path: root/fluent-bit/lib/librdkafka-2.1.0/src/rdkafka_offset.c
blob: ffa6a9d52471d9506ae70271ef75a2014f9f6a0a (plain)
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
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
/*
 * librdkafka - Apache Kafka C library
 *
 * Copyright (c) 2012,2013 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.
 */

// FIXME: Revise this documentation:
/**
 * This file implements the consumer offset storage.
 * It currently supports local file storage and broker OffsetCommit storage.
 *
 * Regardless of commit method (file, broker, ..) this is how it works:
 *  - When rdkafka, or the application, depending on if auto.offset.commit
 *    is enabled or not, calls rd_kafka_offset_store() with an offset to store,
 *    all it does is set rktp->rktp_stored_offset to this value.
 *    This can happen from any thread and is locked by the rktp lock.
 *  - The actual commit/write of the offset to its backing store (filesystem)
 *    is performed by the main rdkafka thread and scheduled at the configured
 *    auto.commit.interval.ms interval.
 *  - The write is performed in the main rdkafka thread (in a blocking manner
 *    for file based offsets) and once the write has
 *    succeeded rktp->rktp_committed_offset is updated to the new value.
 *  - If offset.store.sync.interval.ms is configured the main rdkafka thread
 *    will also make sure to fsync() each offset file accordingly. (file)
 */


#include "rdkafka_int.h"
#include "rdkafka_topic.h"
#include "rdkafka_partition.h"
#include "rdkafka_offset.h"
#include "rdkafka_broker.h"
#include "rdkafka_request.h"

#include <stdio.h>
#include <sys/types.h>
#include <fcntl.h>

#ifdef _WIN32
#include <io.h>
#include <share.h>
#include <sys/stat.h>
#include <shlwapi.h>
#endif


/**
 * Convert an absolute or logical offset to string.
 */
const char *rd_kafka_offset2str(int64_t offset) {
        static RD_TLS char ret[16][32];
        static RD_TLS int i = 0;

        i = (i + 1) % 16;

        if (offset >= 0)
                rd_snprintf(ret[i], sizeof(ret[i]), "%" PRId64, offset);
        else if (offset == RD_KAFKA_OFFSET_BEGINNING)
                return "BEGINNING";
        else if (offset == RD_KAFKA_OFFSET_END)
                return "END";
        else if (offset == RD_KAFKA_OFFSET_STORED)
                return "STORED";
        else if (offset == RD_KAFKA_OFFSET_INVALID)
                return "INVALID";
        else if (offset <= RD_KAFKA_OFFSET_TAIL_BASE)
                rd_snprintf(ret[i], sizeof(ret[i]), "TAIL(%lld)",
                            llabs(offset - RD_KAFKA_OFFSET_TAIL_BASE));
        else
                rd_snprintf(ret[i], sizeof(ret[i]), "%" PRId64 "?", offset);

        return ret[i];
}

static void rd_kafka_offset_file_close(rd_kafka_toppar_t *rktp) {
        if (!rktp->rktp_offset_fp)
                return;

        fclose(rktp->rktp_offset_fp);
        rktp->rktp_offset_fp = NULL;
}


#ifndef _WIN32
/**
 * Linux version of open callback providing racefree CLOEXEC.
 */
int rd_kafka_open_cb_linux(const char *pathname,
                           int flags,
                           mode_t mode,
                           void *opaque) {
#ifdef O_CLOEXEC
        return open(pathname, flags | O_CLOEXEC, mode);
#else
        return rd_kafka_open_cb_generic(pathname, flags, mode, opaque);
#endif
}
#endif

/**
 * Fallback version of open_cb NOT providing racefree CLOEXEC,
 * but setting CLOEXEC after file open (if FD_CLOEXEC is defined).
 */
int rd_kafka_open_cb_generic(const char *pathname,
                             int flags,
                             mode_t mode,
                             void *opaque) {
#ifndef _WIN32
        int fd;
        int on = 1;
        fd     = open(pathname, flags, mode);
        if (fd == -1)
                return -1;
#ifdef FD_CLOEXEC
        fcntl(fd, F_SETFD, FD_CLOEXEC, &on);
#endif
        return fd;
#else
        int fd;
        if (_sopen_s(&fd, pathname, flags, _SH_DENYNO, mode) != 0)
                return -1;
        return fd;
#endif
}


static int rd_kafka_offset_file_open(rd_kafka_toppar_t *rktp) {
        rd_kafka_t *rk = rktp->rktp_rkt->rkt_rk;
        int fd;

#ifndef _WIN32
        mode_t mode = 0644;
#else
        mode_t mode = _S_IREAD | _S_IWRITE;
#endif
        if ((fd = rk->rk_conf.open_cb(rktp->rktp_offset_path, O_CREAT | O_RDWR,
                                      mode, rk->rk_conf.opaque)) == -1) {
                rd_kafka_op_err(rktp->rktp_rkt->rkt_rk, RD_KAFKA_RESP_ERR__FS,
                                "%s [%" PRId32
                                "]: "
                                "Failed to open offset file %s: %s",
                                rktp->rktp_rkt->rkt_topic->str,
                                rktp->rktp_partition, rktp->rktp_offset_path,
                                rd_strerror(errno));
                return -1;
        }

        rktp->rktp_offset_fp =
#ifndef _WIN32
            fdopen(fd, "r+");
#else
            _fdopen(fd, "r+");
#endif

        return 0;
}


static int64_t rd_kafka_offset_file_read(rd_kafka_toppar_t *rktp) {
        char buf[22];
        char *end;
        int64_t offset;
        size_t r;

        if (fseek(rktp->rktp_offset_fp, 0, SEEK_SET) == -1) {
                rd_kafka_op_err(rktp->rktp_rkt->rkt_rk, RD_KAFKA_RESP_ERR__FS,
                                "%s [%" PRId32
                                "]: "
                                "Seek (for read) failed on offset file %s: %s",
                                rktp->rktp_rkt->rkt_topic->str,
                                rktp->rktp_partition, rktp->rktp_offset_path,
                                rd_strerror(errno));
                rd_kafka_offset_file_close(rktp);
                return RD_KAFKA_OFFSET_INVALID;
        }

        r = fread(buf, 1, sizeof(buf) - 1, rktp->rktp_offset_fp);
        if (r == 0) {
                rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET",
                             "%s [%" PRId32 "]: offset file (%s) is empty",
                             rktp->rktp_rkt->rkt_topic->str,
                             rktp->rktp_partition, rktp->rktp_offset_path);
                return RD_KAFKA_OFFSET_INVALID;
        }

        buf[r] = '\0';

        offset = strtoull(buf, &end, 10);
        if (buf == end) {
                rd_kafka_op_err(rktp->rktp_rkt->rkt_rk, RD_KAFKA_RESP_ERR__FS,
                                "%s [%" PRId32
                                "]: "
                                "Unable to parse offset in %s",
                                rktp->rktp_rkt->rkt_topic->str,
                                rktp->rktp_partition, rktp->rktp_offset_path);
                return RD_KAFKA_OFFSET_INVALID;
        }


        rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET",
                     "%s [%" PRId32 "]: Read offset %" PRId64
                     " from offset "
                     "file (%s)",
                     rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition,
                     offset, rktp->rktp_offset_path);

        return offset;
}


/**
 * Sync/flush offset file.
 */
static int rd_kafka_offset_file_sync(rd_kafka_toppar_t *rktp) {
        if (!rktp->rktp_offset_fp)
                return 0;

        rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "SYNC",
                     "%s [%" PRId32 "]: offset file sync",
                     rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition);

#ifndef _WIN32
        (void)fflush(rktp->rktp_offset_fp);
        (void)fsync(fileno(rktp->rktp_offset_fp));  // FIXME
#else
        // FIXME
        // FlushFileBuffers(_get_osfhandle(fileno(rktp->rktp_offset_fp)));
#endif
        return 0;
}


/**
 * Write offset to offset file.
 *
 * Locality: toppar's broker thread
 */
static rd_kafka_resp_err_t
rd_kafka_offset_file_commit(rd_kafka_toppar_t *rktp) {
        rd_kafka_topic_t *rkt = rktp->rktp_rkt;
        int attempt;
        rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR;
        int64_t offset          = rktp->rktp_stored_pos.offset;

        for (attempt = 0; attempt < 2; attempt++) {
                char buf[22];
                int len;

                if (!rktp->rktp_offset_fp)
                        if (rd_kafka_offset_file_open(rktp) == -1)
                                continue;

                if (fseek(rktp->rktp_offset_fp, 0, SEEK_SET) == -1) {
                        rd_kafka_op_err(
                            rktp->rktp_rkt->rkt_rk, RD_KAFKA_RESP_ERR__FS,
                            "%s [%" PRId32
                            "]: "
                            "Seek failed on offset file %s: %s",
                            rktp->rktp_rkt->rkt_topic->str,
                            rktp->rktp_partition, rktp->rktp_offset_path,
                            rd_strerror(errno));
                        err = RD_KAFKA_RESP_ERR__FS;
                        rd_kafka_offset_file_close(rktp);
                        continue;
                }

                len = rd_snprintf(buf, sizeof(buf), "%" PRId64 "\n", offset);

                if (fwrite(buf, 1, len, rktp->rktp_offset_fp) < 1) {
                        rd_kafka_op_err(
                            rktp->rktp_rkt->rkt_rk, RD_KAFKA_RESP_ERR__FS,
                            "%s [%" PRId32
                            "]: "
                            "Failed to write offset %" PRId64
                            " to "
                            "offset file %s: %s",
                            rktp->rktp_rkt->rkt_topic->str,
                            rktp->rktp_partition, offset,
                            rktp->rktp_offset_path, rd_strerror(errno));
                        err = RD_KAFKA_RESP_ERR__FS;
                        rd_kafka_offset_file_close(rktp);
                        continue;
                }

                /* Need to flush before truncate to preserve write ordering */
                (void)fflush(rktp->rktp_offset_fp);

                /* Truncate file */
#ifdef _WIN32
                if (_chsize_s(_fileno(rktp->rktp_offset_fp), len) == -1)
                        ; /* Ignore truncate failures */
#else
                if (ftruncate(fileno(rktp->rktp_offset_fp), len) == -1)
                        ; /* Ignore truncate failures */
#endif
                rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET",
                             "%s [%" PRId32 "]: wrote offset %" PRId64
                             " to "
                             "file %s",
                             rktp->rktp_rkt->rkt_topic->str,
                             rktp->rktp_partition, offset,
                             rktp->rktp_offset_path);

                rktp->rktp_committed_pos.offset = offset;

                /* If sync interval is set to immediate we sync right away. */
                if (rkt->rkt_conf.offset_store_sync_interval_ms == 0)
                        rd_kafka_offset_file_sync(rktp);


                return RD_KAFKA_RESP_ERR_NO_ERROR;
        }


        return err;
}



/**
 * Commit a list of offsets asynchronously. Response will be queued on 'replyq'.
 * Optional \p cb will be set on requesting op.
 *
 * Makes a copy of \p offsets (may be NULL for current assignment)
 */
static rd_kafka_resp_err_t
rd_kafka_commit0(rd_kafka_t *rk,
                 const rd_kafka_topic_partition_list_t *offsets,
                 rd_kafka_toppar_t *rktp,
                 rd_kafka_replyq_t replyq,
                 void (*cb)(rd_kafka_t *rk,
                            rd_kafka_resp_err_t err,
                            rd_kafka_topic_partition_list_t *offsets,
                            void *opaque),
                 void *opaque,
                 const char *reason) {
        rd_kafka_cgrp_t *rkcg;
        rd_kafka_op_t *rko;

        if (!(rkcg = rd_kafka_cgrp_get(rk)))
                return RD_KAFKA_RESP_ERR__UNKNOWN_GROUP;

        rko = rd_kafka_op_new(RD_KAFKA_OP_OFFSET_COMMIT);
        rko->rko_u.offset_commit.reason = rd_strdup(reason);
        rko->rko_replyq                 = replyq;
        rko->rko_u.offset_commit.cb     = cb;
        rko->rko_u.offset_commit.opaque = opaque;
        if (rktp)
                rko->rko_rktp = rd_kafka_toppar_keep(rktp);

        if (offsets)
                rko->rko_u.offset_commit.partitions =
                    rd_kafka_topic_partition_list_copy(offsets);

        rd_kafka_q_enq(rkcg->rkcg_ops, rko);

        return RD_KAFKA_RESP_ERR_NO_ERROR;
}



/**
 * NOTE: 'offsets' may be NULL, see official documentation.
 */
rd_kafka_resp_err_t
rd_kafka_commit(rd_kafka_t *rk,
                const rd_kafka_topic_partition_list_t *offsets,
                int async) {
        rd_kafka_cgrp_t *rkcg;
        rd_kafka_resp_err_t err;
        rd_kafka_q_t *repq   = NULL;
        rd_kafka_replyq_t rq = RD_KAFKA_NO_REPLYQ;

        if (!(rkcg = rd_kafka_cgrp_get(rk)))
                return RD_KAFKA_RESP_ERR__UNKNOWN_GROUP;

        if (!async) {
                repq = rd_kafka_q_new(rk);
                rq   = RD_KAFKA_REPLYQ(repq, 0);
        }

        err = rd_kafka_commit0(rk, offsets, NULL, rq, NULL, NULL, "manual");

        if (!err && !async)
                err = rd_kafka_q_wait_result(repq, RD_POLL_INFINITE);

        if (!async)
                rd_kafka_q_destroy_owner(repq);

        return err;
}


rd_kafka_resp_err_t rd_kafka_commit_message(rd_kafka_t *rk,
                                            const rd_kafka_message_t *rkmessage,
                                            int async) {
        rd_kafka_topic_partition_list_t *offsets;
        rd_kafka_topic_partition_t *rktpar;
        rd_kafka_resp_err_t err;

        if (rkmessage->err)
                return RD_KAFKA_RESP_ERR__INVALID_ARG;

        offsets = rd_kafka_topic_partition_list_new(1);
        rktpar  = rd_kafka_topic_partition_list_add(
            offsets, rd_kafka_topic_name(rkmessage->rkt), rkmessage->partition);
        rktpar->offset = rkmessage->offset + 1;

        err = rd_kafka_commit(rk, offsets, async);

        rd_kafka_topic_partition_list_destroy(offsets);

        return err;
}



rd_kafka_resp_err_t
rd_kafka_commit_queue(rd_kafka_t *rk,
                      const rd_kafka_topic_partition_list_t *offsets,
                      rd_kafka_queue_t *rkqu,
                      void (*cb)(rd_kafka_t *rk,
                                 rd_kafka_resp_err_t err,
                                 rd_kafka_topic_partition_list_t *offsets,
                                 void *opaque),
                      void *opaque) {
        rd_kafka_q_t *rkq;
        rd_kafka_resp_err_t err;

        if (!rd_kafka_cgrp_get(rk))
                return RD_KAFKA_RESP_ERR__UNKNOWN_GROUP;

        if (rkqu)
                rkq = rkqu->rkqu_q;
        else
                rkq = rd_kafka_q_new(rk);

        err = rd_kafka_commit0(rk, offsets, NULL, RD_KAFKA_REPLYQ(rkq, 0), cb,
                               opaque, "manual");

        if (!rkqu) {
                rd_kafka_op_t *rko = rd_kafka_q_pop_serve(
                    rkq, RD_POLL_INFINITE, 0, RD_KAFKA_Q_CB_FORCE_RETURN, NULL,
                    NULL);
                if (!rko)
                        err = RD_KAFKA_RESP_ERR__TIMED_OUT;
                else {
                        if (cb)
                                cb(rk, rko->rko_err,
                                   rko->rko_u.offset_commit.partitions, opaque);
                        err = rko->rko_err;
                        rd_kafka_op_destroy(rko);
                }

                if (rkqu)
                        rd_kafka_q_destroy(rkq);
                else
                        rd_kafka_q_destroy_owner(rkq);
        }

        return err;
}



/**
 * Called when a broker commit is done.
 *
 * Locality: toppar handler thread
 * Locks: none
 */
static void
rd_kafka_offset_broker_commit_cb(rd_kafka_t *rk,
                                 rd_kafka_resp_err_t err,
                                 rd_kafka_topic_partition_list_t *offsets,
                                 void *opaque) {
        rd_kafka_toppar_t *rktp;
        rd_kafka_topic_partition_t *rktpar;

        if (offsets->cnt == 0) {
                rd_kafka_dbg(rk, TOPIC, "OFFSETCOMMIT",
                             "No offsets to commit (commit_cb)");
                return;
        }

        rktpar = &offsets->elems[0];

        if (!(rktp =
                  rd_kafka_topic_partition_get_toppar(rk, rktpar, rd_false))) {
                rd_kafka_dbg(rk, TOPIC, "OFFSETCOMMIT",
                             "No local partition found for %s [%" PRId32
                             "] "
                             "while parsing OffsetCommit response "
                             "(offset %" PRId64 ", error \"%s\")",
                             rktpar->topic, rktpar->partition, rktpar->offset,
                             rd_kafka_err2str(rktpar->err));
                return;
        }

        if (!err)
                err = rktpar->err;

        rd_kafka_toppar_offset_commit_result(rktp, err, offsets);

        rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET",
                     "%s [%" PRId32 "]: offset %" PRId64 " %scommitted: %s",
                     rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition,
                     rktpar->offset, err ? "not " : "", rd_kafka_err2str(err));

        rktp->rktp_committing_pos.offset = 0;

        rd_kafka_toppar_lock(rktp);
        if (rktp->rktp_flags & RD_KAFKA_TOPPAR_F_OFFSET_STORE_STOPPING)
                rd_kafka_offset_store_term(rktp, err);
        rd_kafka_toppar_unlock(rktp);

        rd_kafka_toppar_destroy(rktp);
}


/**
 * @locks_required rd_kafka_toppar_lock(rktp) MUST be held.
 */
static rd_kafka_resp_err_t
rd_kafka_offset_broker_commit(rd_kafka_toppar_t *rktp, const char *reason) {
        rd_kafka_topic_partition_list_t *offsets;
        rd_kafka_topic_partition_t *rktpar;

        rd_kafka_assert(rktp->rktp_rkt->rkt_rk, rktp->rktp_cgrp != NULL);
        rd_kafka_assert(rktp->rktp_rkt->rkt_rk,
                        rktp->rktp_flags & RD_KAFKA_TOPPAR_F_OFFSET_STORE);

        rktp->rktp_committing_pos = rktp->rktp_stored_pos;

        offsets = rd_kafka_topic_partition_list_new(1);
        rktpar  = rd_kafka_topic_partition_list_add(
            offsets, rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition);
        rd_kafka_topic_partition_set_from_fetch_pos(rktpar,
                                                    rktp->rktp_committing_pos);

        rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSETCMT",
                     "%.*s [%" PRId32 "]: committing %s: %s",
                     RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic),
                     rktp->rktp_partition,
                     rd_kafka_fetch_pos2str(rktp->rktp_committing_pos), reason);

        rd_kafka_commit0(rktp->rktp_rkt->rkt_rk, offsets, rktp,
                         RD_KAFKA_REPLYQ(rktp->rktp_ops, 0),
                         rd_kafka_offset_broker_commit_cb, NULL, reason);

        rd_kafka_topic_partition_list_destroy(offsets);

        return RD_KAFKA_RESP_ERR__IN_PROGRESS;
}



/**
 * Commit offset to backing store.
 * This might be an async operation.
 *
 * Locality: toppar handler thread
 */
static rd_kafka_resp_err_t rd_kafka_offset_commit(rd_kafka_toppar_t *rktp,
                                                  const char *reason) {
        rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET",
                     "%s [%" PRId32 "]: commit: stored %s > committed %s?",
                     rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition,
                     rd_kafka_fetch_pos2str(rktp->rktp_stored_pos),
                     rd_kafka_fetch_pos2str(rktp->rktp_committed_pos));

        /* Already committed */
        if (rd_kafka_fetch_pos_cmp(&rktp->rktp_stored_pos,
                                   &rktp->rktp_committed_pos) <= 0)
                return RD_KAFKA_RESP_ERR_NO_ERROR;

        /* Already committing (for async ops) */
        if (rd_kafka_fetch_pos_cmp(&rktp->rktp_stored_pos,
                                   &rktp->rktp_committing_pos) <= 0)
                return RD_KAFKA_RESP_ERR__PREV_IN_PROGRESS;

        switch (rktp->rktp_rkt->rkt_conf.offset_store_method) {
        case RD_KAFKA_OFFSET_METHOD_FILE:
                return rd_kafka_offset_file_commit(rktp);
        case RD_KAFKA_OFFSET_METHOD_BROKER:
                return rd_kafka_offset_broker_commit(rktp, reason);
        default:
                /* UNREACHABLE */
                return RD_KAFKA_RESP_ERR__INVALID_ARG;
        }
}



/**
 * Sync offset backing store. This is only used for METHOD_FILE.
 *
 * Locality: rktp's broker thread.
 */
rd_kafka_resp_err_t rd_kafka_offset_sync(rd_kafka_toppar_t *rktp) {
        switch (rktp->rktp_rkt->rkt_conf.offset_store_method) {
        case RD_KAFKA_OFFSET_METHOD_FILE:
                return rd_kafka_offset_file_sync(rktp);
        default:
                return RD_KAFKA_RESP_ERR__INVALID_ARG;
        }
}


/**
 * Store offset.
 * Typically called from application code.
 *
 * NOTE: No locks must be held.
 *
 * @deprecated Use rd_kafka_offsets_store().
 */
rd_kafka_resp_err_t rd_kafka_offset_store(rd_kafka_topic_t *app_rkt,
                                          int32_t partition,
                                          int64_t offset) {
        rd_kafka_topic_t *rkt = rd_kafka_topic_proper(app_rkt);
        rd_kafka_toppar_t *rktp;
        rd_kafka_resp_err_t err;
        rd_kafka_fetch_pos_t pos = {offset + 1, -1 /*no leader epoch known*/};

        /* Find toppar */
        rd_kafka_topic_rdlock(rkt);
        if (!(rktp = rd_kafka_toppar_get(rkt, partition, 0 /*!ua_on_miss*/))) {
                rd_kafka_topic_rdunlock(rkt);
                return RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION;
        }
        rd_kafka_topic_rdunlock(rkt);

        err = rd_kafka_offset_store0(rktp, pos, rd_false /* Don't force */,
                                     RD_DO_LOCK);

        rd_kafka_toppar_destroy(rktp);

        return err;
}


rd_kafka_resp_err_t
rd_kafka_offsets_store(rd_kafka_t *rk,
                       rd_kafka_topic_partition_list_t *offsets) {
        int i;
        int ok_cnt                   = 0;
        rd_kafka_resp_err_t last_err = RD_KAFKA_RESP_ERR_NO_ERROR;

        if (rk->rk_conf.enable_auto_offset_store)
                return RD_KAFKA_RESP_ERR__INVALID_ARG;

        for (i = 0; i < offsets->cnt; i++) {
                rd_kafka_topic_partition_t *rktpar = &offsets->elems[i];
                rd_kafka_toppar_t *rktp;
                rd_kafka_fetch_pos_t pos = {rktpar->offset, -1};

                rktp =
                    rd_kafka_topic_partition_get_toppar(rk, rktpar, rd_false);
                if (!rktp) {
                        rktpar->err = RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION;
                        last_err    = rktpar->err;
                        continue;
                }

                pos.leader_epoch =
                    rd_kafka_topic_partition_get_leader_epoch(rktpar);

                rktpar->err = rd_kafka_offset_store0(
                    rktp, pos, rd_false /* don't force */, RD_DO_LOCK);
                rd_kafka_toppar_destroy(rktp);

                if (rktpar->err)
                        last_err = rktpar->err;
                else
                        ok_cnt++;
        }

        return offsets->cnt > 0 && ok_cnt == 0 ? last_err
                                               : RD_KAFKA_RESP_ERR_NO_ERROR;
}


rd_kafka_error_t *rd_kafka_offset_store_message(rd_kafka_message_t *rkmessage) {
        rd_kafka_toppar_t *rktp;
        rd_kafka_op_t *rko;
        rd_kafka_resp_err_t err;
        rd_kafka_msg_t *rkm = (rd_kafka_msg_t *)rkmessage;
        rd_kafka_fetch_pos_t pos;

        if (rkmessage->err)
                return rd_kafka_error_new(RD_KAFKA_RESP_ERR__INVALID_ARG,
                                          "Message object must not have an "
                                          "error set");

        if (unlikely(!(rko = rd_kafka_message2rko(rkmessage)) ||
                     !(rktp = rko->rko_rktp)))
                return rd_kafka_error_new(RD_KAFKA_RESP_ERR__INVALID_ARG,
                                          "Invalid message object, "
                                          "not a consumed message");

        pos.offset       = rkmessage->offset + 1;
        pos.leader_epoch = rkm->rkm_u.consumer.leader_epoch;
        err = rd_kafka_offset_store0(rktp, pos, rd_false /* Don't force */,
                                     RD_DO_LOCK);

        if (err == RD_KAFKA_RESP_ERR__STATE)
                return rd_kafka_error_new(err, "Partition is not assigned");
        else if (err)
                return rd_kafka_error_new(err, "Failed to store offset: %s",
                                          rd_kafka_err2str(err));

        return NULL;
}



/**
 * Decommissions the use of an offset file for a toppar.
 * The file content will not be touched and the file will not be removed.
 */
static rd_kafka_resp_err_t rd_kafka_offset_file_term(rd_kafka_toppar_t *rktp) {
        rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR;

        /* Sync offset file if the sync is intervalled (> 0) */
        if (rktp->rktp_rkt->rkt_conf.offset_store_sync_interval_ms > 0) {
                rd_kafka_offset_file_sync(rktp);
                rd_kafka_timer_stop(&rktp->rktp_rkt->rkt_rk->rk_timers,
                                    &rktp->rktp_offset_sync_tmr, 1 /*lock*/);
        }


        rd_kafka_offset_file_close(rktp);

        rd_free(rktp->rktp_offset_path);
        rktp->rktp_offset_path = NULL;

        return err;
}

static rd_kafka_op_res_t rd_kafka_offset_reset_op_cb(rd_kafka_t *rk,
                                                     rd_kafka_q_t *rkq,
                                                     rd_kafka_op_t *rko) {
        rd_kafka_toppar_t *rktp = rko->rko_rktp;
        rd_kafka_toppar_lock(rktp);
        rd_kafka_offset_reset(rktp, rko->rko_u.offset_reset.broker_id,
                              rko->rko_u.offset_reset.pos, rko->rko_err, "%s",
                              rko->rko_u.offset_reset.reason);
        rd_kafka_toppar_unlock(rktp);
        return RD_KAFKA_OP_RES_HANDLED;
}

/**
 * @brief Take action when the offset for a toppar is unusable (due to an
 *        error, or offset is logical).
 *
 * @param rktp the toppar
 * @param broker_id Originating broker, if any, else RD_KAFKA_NODEID_UA.
 * @param err_pos a logical offset, or offset corresponding to the error.
 * @param err the error, or RD_KAFKA_RESP_ERR_NO_ERROR if offset is logical.
 * @param fmt a reason string for logging.
 *
 * @locality any. if not main thread, work will be enqued on main thread.
 * @locks_required toppar_lock() MUST be held
 */
void rd_kafka_offset_reset(rd_kafka_toppar_t *rktp,
                           int32_t broker_id,
                           rd_kafka_fetch_pos_t err_pos,
                           rd_kafka_resp_err_t err,
                           const char *fmt,
                           ...) {
        rd_kafka_fetch_pos_t pos = {RD_KAFKA_OFFSET_INVALID, -1};
        const char *extra        = "";
        char reason[512];
        va_list ap;

        va_start(ap, fmt);
        rd_vsnprintf(reason, sizeof(reason), fmt, ap);
        va_end(ap);

        /* Enqueue op for toppar handler thread if we're on the wrong thread. */
        if (!thrd_is_current(rktp->rktp_rkt->rkt_rk->rk_thread)) {
                rd_kafka_op_t *rko =
                    rd_kafka_op_new(RD_KAFKA_OP_OFFSET_RESET | RD_KAFKA_OP_CB);
                rko->rko_op_cb                    = rd_kafka_offset_reset_op_cb;
                rko->rko_err                      = err;
                rko->rko_rktp                     = rd_kafka_toppar_keep(rktp);
                rko->rko_u.offset_reset.broker_id = broker_id;
                rko->rko_u.offset_reset.pos       = err_pos;
                rko->rko_u.offset_reset.reason    = rd_strdup(reason);
                rd_kafka_q_enq(rktp->rktp_ops, rko);
                return;
        }

        if (err_pos.offset == RD_KAFKA_OFFSET_INVALID || err)
                pos.offset = rktp->rktp_rkt->rkt_conf.auto_offset_reset;
        else
                pos.offset = err_pos.offset;

        if (pos.offset == RD_KAFKA_OFFSET_INVALID) {
                /* Error, auto.offset.reset tells us to error out. */
                if (broker_id != RD_KAFKA_NODEID_UA)
                        rd_kafka_consumer_err(
                            rktp->rktp_fetchq, broker_id,
                            RD_KAFKA_RESP_ERR__AUTO_OFFSET_RESET, 0, NULL, rktp,
                            err_pos.offset, "%s: %s (broker %" PRId32 ")",
                            reason, rd_kafka_err2str(err), broker_id);
                else
                        rd_kafka_consumer_err(
                            rktp->rktp_fetchq, broker_id,
                            RD_KAFKA_RESP_ERR__AUTO_OFFSET_RESET, 0, NULL, rktp,
                            err_pos.offset, "%s: %s", reason,
                            rd_kafka_err2str(err));

                rd_kafka_toppar_set_fetch_state(rktp,
                                                RD_KAFKA_TOPPAR_FETCH_NONE);

        } else if (pos.offset == RD_KAFKA_OFFSET_BEGINNING &&
                   rktp->rktp_lo_offset >= 0) {
                /* Use cached log start from last Fetch if available.
                 * Note: The cached end offset (rktp_ls_offset) can't be
                 *       used here since the End offset is a constantly moving
                 *       target as new messages are produced. */
                extra            = "cached BEGINNING offset ";
                pos.offset       = rktp->rktp_lo_offset;
                pos.leader_epoch = -1;
                rd_kafka_toppar_next_offset_handle(rktp, pos);

        } else {
                /* Else query cluster for offset */
                rktp->rktp_query_pos = pos;
                rd_kafka_toppar_set_fetch_state(
                    rktp, RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY);
        }

        /* Offset resets due to error are logged since they might have quite
         * critical impact. For non-errors, or for auto.offset.reset=error,
         * the reason is simply debug-logged. */
        if (!err || err == RD_KAFKA_RESP_ERR__NO_OFFSET ||
            pos.offset == RD_KAFKA_OFFSET_INVALID)
                rd_kafka_dbg(
                    rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET",
                    "%s [%" PRId32 "]: offset reset (at %s, broker %" PRId32
                    ") "
                    "to %s%s: %s: %s",
                    rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition,
                    rd_kafka_fetch_pos2str(err_pos), broker_id, extra,
                    rd_kafka_fetch_pos2str(pos), reason, rd_kafka_err2str(err));
        else
                rd_kafka_log(
                    rktp->rktp_rkt->rkt_rk, LOG_WARNING, "OFFSET",
                    "%s [%" PRId32 "]: offset reset (at %s, broker %" PRId32
                    ") to %s%s: %s: %s",
                    rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition,
                    rd_kafka_fetch_pos2str(err_pos), broker_id, extra,
                    rd_kafka_fetch_pos2str(pos), reason, rd_kafka_err2str(err));

        /* Note: If rktp is not delegated to the leader, then low and high
           offsets will necessarily be cached from the last FETCH request,
           and so this offset query will never occur in that case for
           BEGINNING / END logical offsets. */
        if (rktp->rktp_fetch_state == RD_KAFKA_TOPPAR_FETCH_OFFSET_QUERY)
                rd_kafka_toppar_offset_request(rktp, rktp->rktp_query_pos,
                                               err ? 100 : 0);
}



/**
 * @brief Offset validation retry timer
 */
static void rd_kafka_offset_validate_tmr_cb(rd_kafka_timers_t *rkts,
                                            void *arg) {
        rd_kafka_toppar_t *rktp = arg;

        rd_kafka_toppar_lock(rktp);
        rd_kafka_offset_validate(rktp, "retrying offset validation");
        rd_kafka_toppar_unlock(rktp);
}



/**
 * @brief OffsetForLeaderEpochResponse handler that
 *        pushes the matched toppar's to the next state.
 *
 * @locality rdkafka main thread
 */
static void rd_kafka_toppar_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,
                                                        void *opaque) {
        rd_kafka_topic_partition_list_t *parts = NULL;
        rd_kafka_toppar_t *rktp                = opaque;
        rd_kafka_topic_partition_t *rktpar;
        int64_t end_offset;
        int32_t end_offset_leader_epoch;

        if (err == RD_KAFKA_RESP_ERR__DESTROY) {
                rd_kafka_toppar_destroy(rktp); /* Drop refcnt */
                return;
        }

        err = rd_kafka_handle_OffsetForLeaderEpoch(rk, rkb, err, rkbuf, request,
                                                   &parts);

        rd_kafka_toppar_lock(rktp);

        if (rktp->rktp_fetch_state != RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT)
                err = RD_KAFKA_RESP_ERR__OUTDATED;

        if (unlikely(!err && parts->cnt == 0))
                err = RD_KAFKA_RESP_ERR__UNKNOWN_PARTITION;

        if (!err) {
                err = (&parts->elems[0])->err;
        }

        if (err) {
                int actions;

                rd_rkb_dbg(rkb, FETCH, "OFFSETVALID",
                           "%.*s [%" PRId32
                           "]: OffsetForLeaderEpoch requested failed: %s",
                           RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic),
                           rktp->rktp_partition, rd_kafka_err2str(err));

                if (err == RD_KAFKA_RESP_ERR__UNSUPPORTED_FEATURE) {
                        rd_rkb_dbg(rkb, FETCH, "VALIDATE",
                                   "%.*s [%" PRId32
                                   "]: offset and epoch validation not "
                                   "supported by broker: validation skipped",
                                   RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic),
                                   rktp->rktp_partition);
                        /* Reset the epoch to -1 since it can't be used with
                         * older brokers. */
                        rktp->rktp_next_fetch_start.leader_epoch = -1;
                        rd_kafka_toppar_set_fetch_state(
                            rktp, RD_KAFKA_TOPPAR_FETCH_ACTIVE);
                        goto done;

                } else if (err == RD_KAFKA_RESP_ERR__OUTDATED) {
                        /* Partition state has changed, this response
                         * is outdated. */
                        goto done;
                }

                actions = rd_kafka_err_action(
                    rkb, err, request, RD_KAFKA_ERR_ACTION_REFRESH,
                    RD_KAFKA_RESP_ERR_UNKNOWN_LEADER_EPOCH,
                    RD_KAFKA_ERR_ACTION_REFRESH,
                    RD_KAFKA_RESP_ERR_FENCED_LEADER_EPOCH,
                    RD_KAFKA_ERR_ACTION_REFRESH,
                    RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART,
                    RD_KAFKA_ERR_ACTION_REFRESH,
                    RD_KAFKA_RESP_ERR_OFFSET_NOT_AVAILABLE,
                    RD_KAFKA_ERR_ACTION_REFRESH,
                    RD_KAFKA_RESP_ERR_KAFKA_STORAGE_ERROR,
                    RD_KAFKA_ERR_ACTION_END);


                if (actions & RD_KAFKA_ERR_ACTION_REFRESH)
                        /* Metadata refresh is ongoing, so force it */
                        rd_kafka_topic_leader_query0(rk, rktp->rktp_rkt, 1,
                                                     rd_true /* force */);

                if (actions & RD_KAFKA_ERR_ACTION_RETRY) {
                        /* No need for refcnt on rktp for timer opaque
                         * since the timer resides on the rktp and will be
                         * stopped on toppar remove. */
                        rd_kafka_timer_start_oneshot(
                            &rk->rk_timers, &rktp->rktp_validate_tmr, rd_false,
                            500 * 1000 /* 500ms */,
                            rd_kafka_offset_validate_tmr_cb, rktp);
                        goto done;
                }

                if (!(actions & RD_KAFKA_ERR_ACTION_REFRESH)) {
                        /* Permanent error */
                        rd_kafka_offset_reset(
                            rktp, rd_kafka_broker_id(rkb),
                            RD_KAFKA_FETCH_POS(RD_KAFKA_OFFSET_INVALID,
                                               rktp->rktp_leader_epoch),
                            RD_KAFKA_RESP_ERR__LOG_TRUNCATION,
                            "Unable to validate offset and epoch: %s",
                            rd_kafka_err2str(err));
                }
                goto done;
        }


        rktpar     = &parts->elems[0];
        end_offset = rktpar->offset;
        end_offset_leader_epoch =
            rd_kafka_topic_partition_get_leader_epoch(rktpar);

        if (end_offset < 0 || end_offset_leader_epoch < 0) {
                rd_kafka_offset_reset(
                    rktp, rd_kafka_broker_id(rkb), rktp->rktp_next_fetch_start,
                    RD_KAFKA_RESP_ERR__LOG_TRUNCATION,
                    "No epoch found less or equal to "
                    "%s: broker end offset is %" PRId64
                    " (offset leader epoch %" PRId32
                    ")."
                    " Reset using configured policy.",
                    rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start),
                    end_offset, end_offset_leader_epoch);

        } else if (end_offset < rktp->rktp_next_fetch_start.offset) {

                if (rktp->rktp_rkt->rkt_conf.auto_offset_reset ==
                    RD_KAFKA_OFFSET_INVALID /* auto.offset.reset=error */) {
                        rd_kafka_offset_reset(
                            rktp, rd_kafka_broker_id(rkb),
                            RD_KAFKA_FETCH_POS(RD_KAFKA_OFFSET_INVALID,
                                               rktp->rktp_leader_epoch),
                            RD_KAFKA_RESP_ERR__LOG_TRUNCATION,
                            "Partition log truncation detected at %s: "
                            "broker end offset is %" PRId64
                            " (offset leader epoch %" PRId32
                            "). "
                            "Reset to INVALID.",
                            rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start),
                            end_offset, end_offset_leader_epoch);

                } else {
                        rd_kafka_toppar_unlock(rktp);

                        /* Seek to the updated end offset */
                        rd_kafka_fetch_pos_t fetch_pos =
                            rd_kafka_topic_partition_get_fetch_pos(rktpar);
                        fetch_pos.validated = rd_true;

                        rd_kafka_toppar_op_seek(rktp, fetch_pos,
                                                RD_KAFKA_NO_REPLYQ);

                        rd_kafka_topic_partition_list_destroy(parts);
                        rd_kafka_toppar_destroy(rktp);

                        return;
                }

        } else {
                rd_rkb_dbg(rkb, FETCH, "OFFSETVALID",
                           "%.*s [%" PRId32
                           "]: offset and epoch validation "
                           "succeeded: broker end offset %" PRId64
                           " (offset leader epoch %" PRId32 ")",
                           RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic),
                           rktp->rktp_partition, end_offset,
                           end_offset_leader_epoch);

                rktp->rktp_next_fetch_start.leader_epoch =
                    end_offset_leader_epoch;
                rd_kafka_toppar_set_fetch_state(rktp,
                                                RD_KAFKA_TOPPAR_FETCH_ACTIVE);
        }

done:
        rd_kafka_toppar_unlock(rktp);

        if (parts)
                rd_kafka_topic_partition_list_destroy(parts);
        rd_kafka_toppar_destroy(rktp);
}


static rd_kafka_op_res_t rd_kafka_offset_validate_op_cb(rd_kafka_t *rk,
                                                        rd_kafka_q_t *rkq,
                                                        rd_kafka_op_t *rko) {
        rd_kafka_toppar_t *rktp = rko->rko_rktp;
        rd_kafka_toppar_lock(rktp);
        rd_kafka_offset_validate(rktp, "%s", rko->rko_u.offset_reset.reason);
        rd_kafka_toppar_unlock(rktp);
        return RD_KAFKA_OP_RES_HANDLED;
}

/**
 * @brief Validate partition epoch and offset (KIP-320).
 *
 * @param rktp the toppar
 * @param err Optional error code that triggered the validation.
 * @param fmt a reason string for logging.
 *
 * @locality any. if not main thread, work will be enqued on main thread.
 * @locks_required toppar_lock() MUST be held
 */
void rd_kafka_offset_validate(rd_kafka_toppar_t *rktp, const char *fmt, ...) {
        rd_kafka_topic_partition_list_t *parts;
        rd_kafka_topic_partition_t *rktpar;
        char reason[512];
        va_list ap;

        if (rktp->rktp_rkt->rkt_rk->rk_type != RD_KAFKA_CONSUMER)
                return;

        va_start(ap, fmt);
        rd_vsnprintf(reason, sizeof(reason), fmt, ap);
        va_end(ap);

        /* Enqueue op for toppar handler thread if we're on the wrong thread. */
        if (!thrd_is_current(rktp->rktp_rkt->rkt_rk->rk_thread)) {
                /* Reuse OP_OFFSET_RESET type */
                rd_kafka_op_t *rko =
                    rd_kafka_op_new(RD_KAFKA_OP_OFFSET_RESET | RD_KAFKA_OP_CB);
                rko->rko_op_cb                 = rd_kafka_offset_validate_op_cb;
                rko->rko_rktp                  = rd_kafka_toppar_keep(rktp);
                rko->rko_u.offset_reset.reason = rd_strdup(reason);
                rd_kafka_q_enq(rktp->rktp_ops, rko);
                return;
        }

        if (rktp->rktp_fetch_state != RD_KAFKA_TOPPAR_FETCH_ACTIVE &&
            rktp->rktp_fetch_state !=
                RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT) {
                rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, FETCH, "VALIDATE",
                             "%.*s [%" PRId32
                             "]: skipping offset "
                             "validation in fetch state %s",
                             RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic),
                             rktp->rktp_partition,
                             rd_kafka_fetch_states[rktp->rktp_fetch_state]);
                return;
        }


        if (rktp->rktp_leader_id == -1 || !rktp->rktp_leader ||
            rktp->rktp_leader->rkb_source == RD_KAFKA_INTERNAL) {
                rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, FETCH, "VALIDATE",
                             "%.*s [%" PRId32
                             "]: unable to perform offset "
                             "validation: partition leader not available",
                             RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic),
                             rktp->rktp_partition);

                rd_kafka_toppar_set_fetch_state(rktp,
                                                RD_KAFKA_TOPPAR_FETCH_ACTIVE);
                return;
        }

        /* If the fetch start position does not have an epoch set then
         * there is no point in doing validation.
         * This is the case for epoch-less seek()s or epoch-less
         * committed offsets. */
        if (rktp->rktp_next_fetch_start.leader_epoch == -1) {
                rd_kafka_dbg(
                    rktp->rktp_rkt->rkt_rk, FETCH, "VALIDATE",
                    "%.*s [%" PRId32
                    "]: skipping offset "
                    "validation for %s: no leader epoch set",
                    RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic),
                    rktp->rktp_partition,
                    rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start));
                rd_kafka_toppar_set_fetch_state(rktp,
                                                RD_KAFKA_TOPPAR_FETCH_ACTIVE);
                return;
        }

        rd_kafka_toppar_set_fetch_state(
            rktp, RD_KAFKA_TOPPAR_FETCH_VALIDATE_EPOCH_WAIT);

        /* Construct and send OffsetForLeaderEpochRequest */
        parts  = rd_kafka_topic_partition_list_new(1);
        rktpar = rd_kafka_topic_partition_list_add(
            parts, rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition);
        rd_kafka_topic_partition_set_leader_epoch(
            rktpar, rktp->rktp_next_fetch_start.leader_epoch);
        rd_kafka_topic_partition_set_current_leader_epoch(
            rktpar, rktp->rktp_leader_epoch);
        rd_kafka_toppar_keep(rktp); /* for request opaque */

        rd_rkb_dbg(rktp->rktp_leader, FETCH, "VALIDATE",
                   "%.*s [%" PRId32
                   "]: querying broker for epoch "
                   "validation of %s: %s",
                   RD_KAFKAP_STR_PR(rktp->rktp_rkt->rkt_topic),
                   rktp->rktp_partition,
                   rd_kafka_fetch_pos2str(rktp->rktp_next_fetch_start), reason);

        rd_kafka_OffsetForLeaderEpochRequest(
            rktp->rktp_leader, parts, RD_KAFKA_REPLYQ(rktp->rktp_ops, 0),
            rd_kafka_toppar_handle_OffsetForLeaderEpoch, rktp);
        rd_kafka_topic_partition_list_destroy(parts);
}


/**
 * Escape any special characters in filename 'in' and write escaped
 * string to 'out' (of max size out_size).
 */
static char *mk_esc_filename(const char *in, char *out, size_t out_size) {
        const char *s = in;
        char *o       = out;

        while (*s) {
                const char *esc;
                size_t esclen;

                switch (*s) {
                case '/': /* linux */
                        esc    = "%2F";
                        esclen = strlen(esc);
                        break;
                case ':': /* osx, windows */
                        esc    = "%3A";
                        esclen = strlen(esc);
                        break;
                case '\\': /* windows */
                        esc    = "%5C";
                        esclen = strlen(esc);
                        break;
                default:
                        esc    = s;
                        esclen = 1;
                        break;
                }

                if ((size_t)((o + esclen + 1) - out) >= out_size) {
                        /* No more space in output string, truncate. */
                        break;
                }

                while (esclen-- > 0)
                        *(o++) = *(esc++);

                s++;
        }

        *o = '\0';
        return out;
}


static void rd_kafka_offset_sync_tmr_cb(rd_kafka_timers_t *rkts, void *arg) {
        rd_kafka_toppar_t *rktp = arg;
        rd_kafka_offset_sync(rktp);
}


/**
 * Prepare a toppar for using an offset file.
 *
 * Locality: rdkafka main thread
 * Locks: toppar_lock(rktp) must be held
 */
static void rd_kafka_offset_file_init(rd_kafka_toppar_t *rktp) {
        char spath[4096 + 1]; /* larger than escfile to avoid warning */
        const char *path = rktp->rktp_rkt->rkt_conf.offset_store_path;
        int64_t offset   = RD_KAFKA_OFFSET_INVALID;

        if (rd_kafka_path_is_dir(path)) {
                char tmpfile[1024];
                char escfile[4096];

                /* Include group.id in filename if configured. */
                if (!RD_KAFKAP_STR_IS_NULL(rktp->rktp_rkt->rkt_rk->rk_group_id))
                        rd_snprintf(tmpfile, sizeof(tmpfile),
                                    "%s-%" PRId32 "-%.*s.offset",
                                    rktp->rktp_rkt->rkt_topic->str,
                                    rktp->rktp_partition,
                                    RD_KAFKAP_STR_PR(
                                        rktp->rktp_rkt->rkt_rk->rk_group_id));
                else
                        rd_snprintf(tmpfile, sizeof(tmpfile),
                                    "%s-%" PRId32 ".offset",
                                    rktp->rktp_rkt->rkt_topic->str,
                                    rktp->rktp_partition);

                /* Escape filename to make it safe. */
                mk_esc_filename(tmpfile, escfile, sizeof(escfile));

                rd_snprintf(spath, sizeof(spath), "%s%s%s", path,
                            path[strlen(path) - 1] == '/' ? "" : "/", escfile);

                path = spath;
        }

        rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET",
                     "%s [%" PRId32 "]: using offset file %s",
                     rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition,
                     path);
        rktp->rktp_offset_path = rd_strdup(path);


        /* Set up the offset file sync interval. */
        if (rktp->rktp_rkt->rkt_conf.offset_store_sync_interval_ms > 0)
                rd_kafka_timer_start(
                    &rktp->rktp_rkt->rkt_rk->rk_timers,
                    &rktp->rktp_offset_sync_tmr,
                    rktp->rktp_rkt->rkt_conf.offset_store_sync_interval_ms *
                        1000ll,
                    rd_kafka_offset_sync_tmr_cb, rktp);

        if (rd_kafka_offset_file_open(rktp) != -1) {
                /* Read offset from offset file. */
                offset = rd_kafka_offset_file_read(rktp);
        }

        if (offset != RD_KAFKA_OFFSET_INVALID) {
                /* Start fetching from offset */
                rktp->rktp_stored_pos.offset    = offset;
                rktp->rktp_committed_pos.offset = offset;
                rd_kafka_toppar_next_offset_handle(rktp, rktp->rktp_stored_pos);

        } else {
                /* Offset was not usable: perform offset reset logic */
                rktp->rktp_committed_pos.offset = RD_KAFKA_OFFSET_INVALID;
                rd_kafka_offset_reset(
                    rktp, RD_KAFKA_NODEID_UA,
                    RD_KAFKA_FETCH_POS(RD_KAFKA_OFFSET_INVALID, -1),
                    RD_KAFKA_RESP_ERR__FS, "non-readable offset file");
        }
}



/**
 * Terminate broker offset store
 */
static rd_kafka_resp_err_t
rd_kafka_offset_broker_term(rd_kafka_toppar_t *rktp) {
        return RD_KAFKA_RESP_ERR_NO_ERROR;
}


/**
 * Prepare a toppar for using broker offset commit (broker 0.8.2 or
 * later). When using KafkaConsumer (high-level consumer) this
 * functionality is disabled in favour of the cgrp commits for the
 * entire set of subscriptions.
 */
static void rd_kafka_offset_broker_init(rd_kafka_toppar_t *rktp) {
        if (!rd_kafka_is_simple_consumer(rktp->rktp_rkt->rkt_rk))
                return;
        rd_kafka_offset_reset(rktp, RD_KAFKA_NODEID_UA,
                              RD_KAFKA_FETCH_POS(RD_KAFKA_OFFSET_STORED, -1),
                              RD_KAFKA_RESP_ERR_NO_ERROR,
                              "query broker for offsets");
}


/**
 * Terminates toppar's offset store, this is the finalizing step after
 * offset_store_stop().
 *
 * Locks: rd_kafka_toppar_lock() MUST be held.
 */
void rd_kafka_offset_store_term(rd_kafka_toppar_t *rktp,
                                rd_kafka_resp_err_t err) {
        rd_kafka_resp_err_t err2;

        rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "STORETERM",
                     "%s [%" PRId32 "]: offset store terminating",
                     rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition);

        rktp->rktp_flags &= ~RD_KAFKA_TOPPAR_F_OFFSET_STORE_STOPPING;

        rd_kafka_timer_stop(&rktp->rktp_rkt->rkt_rk->rk_timers,
                            &rktp->rktp_offset_commit_tmr, 1 /*lock*/);

        switch (rktp->rktp_rkt->rkt_conf.offset_store_method) {
        case RD_KAFKA_OFFSET_METHOD_FILE:
                err2 = rd_kafka_offset_file_term(rktp);
                break;
        case RD_KAFKA_OFFSET_METHOD_BROKER:
                err2 = rd_kafka_offset_broker_term(rktp);
                break;
        case RD_KAFKA_OFFSET_METHOD_NONE:
                err2 = RD_KAFKA_RESP_ERR_NO_ERROR;
                break;
        }

        /* Prioritize the input error (probably from commit), fall
         * back on termination error. */
        if (!err)
                err = err2;

        rd_kafka_toppar_fetch_stopped(rktp, err);
}


/**
 * Stop toppar's offset store, committing the final offsets, etc.
 *
 * Returns RD_KAFKA_RESP_ERR_NO_ERROR on success,
 * RD_KAFKA_RESP_ERR__IN_PROGRESS if the term triggered an
 * async operation (e.g., broker offset commit), or
 * any other error in case of immediate failure.
 *
 * The offset layer will call rd_kafka_offset_store_term() when
 * the offset management has been fully stopped for this partition.
 *
 * Locks: rd_kafka_toppar_lock() MUST be held.
 */
rd_kafka_resp_err_t rd_kafka_offset_store_stop(rd_kafka_toppar_t *rktp) {
        rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR;

        if (!(rktp->rktp_flags & RD_KAFKA_TOPPAR_F_OFFSET_STORE))
                goto done;

        rktp->rktp_flags |= RD_KAFKA_TOPPAR_F_OFFSET_STORE_STOPPING;

        rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET",
                     "%s [%" PRId32
                     "]: stopping offset store "
                     "(stored %s, committed %s, EOF offset %" PRId64 ")",
                     rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition,
                     rd_kafka_fetch_pos2str(rktp->rktp_stored_pos),
                     rd_kafka_fetch_pos2str(rktp->rktp_committed_pos),
                     rktp->rktp_offsets_fin.eof_offset);

        /* Store end offset for empty partitions */
        if (rktp->rktp_rkt->rkt_rk->rk_conf.enable_auto_offset_store &&
            rktp->rktp_stored_pos.offset == RD_KAFKA_OFFSET_INVALID &&
            rktp->rktp_offsets_fin.eof_offset > 0)
                rd_kafka_offset_store0(
                    rktp,
                    RD_KAFKA_FETCH_POS(rktp->rktp_offsets_fin.eof_offset,
                                       rktp->rktp_leader_epoch),
                    rd_true /* force */, RD_DONT_LOCK);

        /* Commit offset to backing store.
         * This might be an async operation. */
        if (rd_kafka_is_simple_consumer(rktp->rktp_rkt->rkt_rk) &&
            rd_kafka_fetch_pos_cmp(&rktp->rktp_stored_pos,
                                   &rktp->rktp_committed_pos) > 0)
                err = rd_kafka_offset_commit(rktp, "offset store stop");

        /* If stop is in progress (async commit), return now. */
        if (err == RD_KAFKA_RESP_ERR__IN_PROGRESS)
                return err;

done:
        /* Stop is done */
        rd_kafka_offset_store_term(rktp, err);

        return RD_KAFKA_RESP_ERR_NO_ERROR;
}


static void rd_kafka_offset_auto_commit_tmr_cb(rd_kafka_timers_t *rkts,
                                               void *arg) {
        rd_kafka_toppar_t *rktp = arg;
        rd_kafka_offset_commit(rktp, "auto commit timer");
}

void rd_kafka_offset_query_tmr_cb(rd_kafka_timers_t *rkts, void *arg) {
        rd_kafka_toppar_t *rktp = arg;
        rd_kafka_toppar_lock(rktp);
        rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET",
                     "Topic %s [%" PRId32
                     "]: timed offset query for %s in state %s",
                     rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition,
                     rd_kafka_fetch_pos2str(rktp->rktp_query_pos),
                     rd_kafka_fetch_states[rktp->rktp_fetch_state]);
        rd_kafka_toppar_offset_request(rktp, rktp->rktp_query_pos, 0);
        rd_kafka_toppar_unlock(rktp);
}


/**
 * Initialize toppar's offset store.
 *
 * Locality: toppar handler thread
 */
void rd_kafka_offset_store_init(rd_kafka_toppar_t *rktp) {
        static const char *store_names[] = {"none", "file", "broker"};

        rd_kafka_dbg(rktp->rktp_rkt->rkt_rk, TOPIC, "OFFSET",
                     "%s [%" PRId32 "]: using offset store method: %s",
                     rktp->rktp_rkt->rkt_topic->str, rktp->rktp_partition,
                     store_names[rktp->rktp_rkt->rkt_conf.offset_store_method]);

        /* The committed offset is unknown at this point. */
        rktp->rktp_committed_pos.offset = RD_KAFKA_OFFSET_INVALID;

        /* Set up the commit interval (for simple consumer). */
        if (rd_kafka_is_simple_consumer(rktp->rktp_rkt->rkt_rk) &&
            rktp->rktp_rkt->rkt_conf.auto_commit_interval_ms > 0)
                rd_kafka_timer_start(
                    &rktp->rktp_rkt->rkt_rk->rk_timers,
                    &rktp->rktp_offset_commit_tmr,
                    rktp->rktp_rkt->rkt_conf.auto_commit_interval_ms * 1000ll,
                    rd_kafka_offset_auto_commit_tmr_cb, rktp);

        switch (rktp->rktp_rkt->rkt_conf.offset_store_method) {
        case RD_KAFKA_OFFSET_METHOD_FILE:
                rd_kafka_offset_file_init(rktp);
                break;
        case RD_KAFKA_OFFSET_METHOD_BROKER:
                rd_kafka_offset_broker_init(rktp);
                break;
        case RD_KAFKA_OFFSET_METHOD_NONE:
                break;
        default:
                /* NOTREACHED */
                return;
        }

        rktp->rktp_flags |= RD_KAFKA_TOPPAR_F_OFFSET_STORE;
}


/**
 * Update toppar app_pos and store_offset (if enabled) to the provided
 * offset and epoch.
 */
void rd_kafka_update_app_pos(rd_kafka_t *rk,
                             rd_kafka_toppar_t *rktp,
                             rd_kafka_fetch_pos_t pos,
                             rd_dolock_t do_lock) {

        if (do_lock)
                rd_kafka_toppar_lock(rktp);

        rktp->rktp_app_pos = pos;
        if (rk->rk_conf.enable_auto_offset_store)
                rd_kafka_offset_store0(rktp, pos,
                                       /* force: ignore assignment state */
                                       rd_true, RD_DONT_LOCK);

        if (do_lock)
                rd_kafka_toppar_unlock(rktp);
}