krafka 0.7.0

A pure Rust, async-native Apache Kafka client
Documentation
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
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756
1757
1758
1759
1760
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808
1809
1810
1811
1812
1813
1814
1815
1816
1817
1818
1819
1820
1821
1822
1823
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
1838
1839
1840
1841
1842
1843
1844
1845
1846
1847
1848
1849
1850
1851
1852
1853
1854
1855
1856
1857
1858
1859
1860
1861
1862
1863
1864
1865
1866
1867
1868
1869
1870
1871
1872
1873
1874
1875
1876
1877
1878
1879
1880
1881
1882
1883
1884
1885
1886
1887
1888
1889
1890
1891
1892
1893
1894
1895
1896
1897
1898
1899
1900
1901
1902
1903
1904
1905
1906
1907
1908
1909
1910
1911
1912
1913
1914
1915
1916
1917
1918
1919
1920
1921
1922
1923
1924
1925
1926
1927
1928
1929
1930
1931
1932
1933
1934
1935
1936
1937
1938
1939
1940
1941
1942
1943
1944
1945
1946
1947
1948
1949
1950
1951
1952
1953
1954
1955
1956
1957
1958
1959
1960
1961
1962
1963
1964
1965
1966
1967
1968
1969
1970
1971
1972
1973
1974
1975
1976
1977
1978
1979
1980
1981
1982
1983
1984
1985
1986
1987
1988
1989
1990
1991
1992
1993
1994
1995
1996
1997
1998
1999
2000
2001
2002
2003
2004
2005
2006
2007
2008
2009
2010
2011
2012
2013
2014
2015
2016
2017
2018
2019
2020
2021
2022
2023
2024
2025
2026
2027
2028
2029
2030
2031
2032
2033
2034
2035
2036
2037
2038
2039
2040
2041
2042
2043
2044
2045
2046
2047
2048
2049
2050
2051
2052
2053
2054
2055
2056
2057
2058
2059
2060
2061
2062
2063
2064
2065
2066
2067
2068
2069
2070
2071
2072
2073
2074
2075
2076
2077
2078
2079
2080
2081
2082
2083
2084
2085
2086
2087
2088
2089
2090
2091
2092
2093
2094
2095
2096
2097
2098
2099
2100
2101
2102
2103
2104
2105
2106
2107
2108
2109
2110
2111
2112
2113
2114
2115
2116
2117
2118
2119
2120
2121
2122
2123
2124
2125
2126
2127
2128
2129
2130
2131
2132
2133
2134
2135
2136
2137
2138
2139
2140
2141
2142
2143
2144
2145
2146
2147
2148
2149
2150
2151
2152
2153
2154
2155
2156
2157
2158
2159
2160
2161
2162
2163
2164
2165
2166
2167
2168
2169
2170
2171
2172
2173
2174
2175
2176
2177
2178
2179
2180
2181
2182
2183
2184
2185
2186
2187
2188
2189
2190
2191
2192
2193
2194
2195
2196
2197
2198
2199
2200
2201
2202
2203
2204
2205
2206
2207
2208
2209
2210
2211
2212
2213
2214
2215
2216
2217
2218
2219
2220
2221
2222
2223
2224
2225
2226
2227
2228
2229
2230
2231
2232
2233
2234
2235
2236
2237
2238
2239
2240
2241
2242
2243
2244
2245
2246
2247
2248
2249
2250
2251
2252
2253
2254
2255
2256
2257
2258
2259
2260
2261
2262
2263
2264
2265
2266
2267
2268
2269
2270
2271
2272
2273
2274
2275
2276
2277
2278
2279
2280
2281
2282
2283
2284
2285
2286
2287
2288
2289
2290
2291
2292
2293
2294
2295
2296
2297
2298
2299
2300
2301
2302
2303
2304
2305
2306
2307
2308
2309
2310
2311
2312
2313
2314
2315
2316
2317
2318
2319
2320
2321
2322
2323
2324
2325
2326
2327
2328
2329
2330
2331
2332
2333
2334
2335
2336
2337
2338
2339
2340
2341
2342
2343
2344
2345
2346
2347
2348
2349
2350
2351
2352
2353
2354
2355
2356
2357
2358
2359
2360
2361
2362
2363
2364
2365
2366
2367
2368
2369
2370
2371
2372
2373
2374
2375
2376
2377
2378
2379
2380
2381
2382
2383
2384
2385
2386
2387
2388
2389
2390
2391
2392
2393
2394
2395
2396
2397
2398
2399
2400
2401
2402
2403
2404
2405
2406
2407
2408
2409
2410
2411
2412
2413
2414
2415
2416
2417
2418
2419
2420
2421
2422
2423
2424
2425
2426
2427
2428
2429
2430
2431
2432
2433
2434
2435
2436
2437
2438
2439
2440
2441
2442
2443
2444
2445
2446
2447
2448
2449
2450
2451
2452
2453
2454
2455
2456
2457
2458
2459
2460
2461
2462
2463
2464
2465
2466
2467
2468
2469
2470
2471
2472
2473
2474
2475
2476
2477
2478
2479
2480
2481
2482
2483
2484
2485
2486
2487
2488
2489
2490
2491
2492
2493
2494
2495
2496
2497
2498
2499
2500
2501
2502
2503
2504
2505
2506
2507
2508
2509
2510
2511
2512
2513
2514
2515
2516
2517
2518
2519
2520
2521
2522
2523
2524
2525
2526
2527
2528
2529
2530
2531
2532
2533
2534
2535
2536
2537
2538
2539
2540
2541
2542
2543
2544
2545
2546
2547
2548
2549
2550
2551
2552
2553
2554
2555
2556
2557
2558
2559
2560
2561
2562
2563
2564
2565
2566
2567
2568
2569
2570
2571
2572
2573
2574
2575
2576
2577
2578
2579
2580
2581
2582
2583
2584
2585
2586
2587
2588
2589
2590
2591
2592
2593
2594
2595
2596
2597
2598
2599
2600
2601
2602
2603
2604
2605
2606
2607
2608
2609
2610
2611
2612
2613
2614
2615
2616
2617
2618
2619
2620
2621
2622
2623
2624
2625
2626
2627
2628
2629
2630
2631
2632
2633
2634
2635
2636
2637
2638
2639
2640
2641
2642
2643
2644
2645
2646
2647
2648
2649
2650
2651
2652
2653
2654
2655
2656
2657
2658
2659
2660
2661
2662
2663
2664
2665
2666
2667
2668
2669
2670
2671
2672
2673
2674
2675
2676
2677
2678
2679
2680
2681
2682
2683
2684
2685
2686
2687
2688
2689
2690
2691
2692
2693
2694
2695
2696
2697
2698
2699
2700
2701
2702
2703
2704
2705
2706
2707
2708
2709
2710
2711
2712
2713
2714
2715
2716
2717
2718
2719
2720
2721
2722
2723
2724
2725
2726
2727
2728
2729
2730
2731
2732
2733
2734
2735
2736
2737
2738
2739
2740
2741
2742
2743
2744
2745
2746
2747
2748
2749
2750
2751
2752
2753
2754
2755
2756
2757
2758
2759
2760
2761
2762
2763
2764
2765
2766
2767
2768
2769
2770
2771
2772
2773
2774
2775
2776
2777
2778
2779
2780
2781
2782
2783
2784
2785
2786
2787
2788
2789
2790
2791
2792
2793
2794
2795
2796
2797
2798
2799
2800
2801
2802
2803
2804
2805
2806
2807
2808
2809
2810
2811
2812
2813
2814
2815
2816
2817
2818
2819
2820
2821
2822
2823
2824
2825
2826
2827
2828
2829
2830
2831
2832
2833
2834
2835
2836
2837
2838
2839
2840
2841
2842
2843
2844
2845
2846
2847
2848
2849
2850
2851
2852
2853
2854
2855
2856
2857
2858
2859
2860
2861
2862
2863
2864
2865
2866
2867
2868
2869
2870
2871
2872
2873
2874
2875
2876
2877
2878
2879
2880
2881
2882
2883
2884
2885
2886
2887
2888
2889
2890
2891
2892
2893
2894
2895
2896
2897
2898
2899
2900
2901
2902
2903
2904
2905
2906
2907
2908
2909
2910
2911
2912
2913
2914
2915
2916
2917
2918
2919
2920
2921
2922
2923
2924
2925
2926
2927
2928
2929
2930
2931
2932
2933
2934
2935
2936
2937
2938
2939
2940
2941
2942
2943
2944
2945
2946
2947
2948
2949
2950
2951
2952
2953
2954
2955
2956
2957
2958
2959
2960
2961
2962
2963
2964
2965
2966
2967
2968
2969
2970
2971
2972
2973
2974
2975
2976
2977
2978
2979
2980
2981
2982
2983
2984
2985
2986
2987
2988
2989
2990
2991
2992
2993
2994
2995
2996
2997
2998
2999
3000
3001
3002
3003
3004
3005
3006
3007
3008
3009
3010
3011
3012
3013
3014
3015
3016
3017
3018
3019
3020
3021
3022
3023
3024
3025
3026
3027
3028
3029
3030
3031
3032
3033
3034
3035
3036
3037
3038
3039
3040
3041
3042
3043
3044
3045
3046
3047
3048
3049
3050
3051
3052
3053
3054
3055
3056
3057
3058
3059
3060
3061
3062
3063
3064
3065
3066
3067
3068
3069
3070
3071
3072
3073
3074
3075
3076
3077
3078
3079
3080
3081
3082
3083
3084
3085
3086
3087
3088
3089
3090
3091
3092
3093
3094
3095
3096
3097
3098
3099
3100
3101
3102
3103
3104
3105
3106
3107
3108
3109
3110
3111
3112
3113
3114
3115
3116
3117
3118
3119
3120
3121
3122
3123
3124
3125
3126
3127
3128
3129
3130
3131
3132
3133
3134
3135
3136
3137
3138
3139
3140
3141
3142
3143
3144
3145
3146
3147
3148
3149
3150
3151
3152
3153
3154
3155
3156
3157
3158
3159
3160
3161
3162
3163
3164
3165
3166
3167
3168
3169
3170
3171
3172
3173
3174
3175
3176
3177
3178
3179
3180
3181
3182
3183
3184
3185
3186
3187
3188
3189
3190
3191
3192
3193
3194
3195
3196
3197
3198
3199
3200
3201
3202
3203
3204
3205
3206
3207
3208
3209
3210
3211
3212
3213
3214
3215
3216
3217
3218
3219
3220
3221
3222
3223
3224
3225
3226
3227
3228
3229
3230
3231
3232
3233
3234
3235
3236
3237
3238
3239
3240
3241
3242
3243
3244
3245
3246
3247
3248
3249
3250
3251
3252
3253
3254
3255
3256
3257
3258
3259
3260
3261
3262
3263
3264
3265
3266
3267
3268
3269
3270
3271
3272
3273
3274
3275
3276
3277
3278
3279
3280
3281
3282
3283
3284
3285
3286
3287
3288
3289
3290
3291
3292
3293
3294
3295
3296
3297
3298
3299
3300
3301
3302
3303
3304
3305
3306
3307
3308
3309
3310
3311
3312
3313
3314
3315
3316
3317
3318
3319
3320
3321
3322
3323
3324
3325
3326
3327
3328
3329
3330
3331
3332
3333
3334
3335
3336
3337
3338
3339
3340
3341
3342
3343
3344
3345
3346
3347
3348
3349
3350
3351
3352
3353
3354
3355
3356
3357
3358
3359
3360
3361
3362
3363
3364
3365
3366
3367
3368
3369
3370
3371
3372
3373
3374
3375
3376
3377
3378
3379
3380
3381
3382
3383
3384
3385
3386
3387
3388
3389
3390
3391
3392
3393
3394
3395
3396
3397
3398
3399
3400
3401
3402
3403
3404
3405
3406
3407
3408
3409
3410
3411
3412
3413
3414
3415
3416
3417
3418
3419
3420
3421
3422
3423
3424
3425
3426
3427
3428
3429
3430
3431
3432
3433
3434
3435
3436
3437
3438
3439
3440
3441
3442
3443
3444
3445
3446
3447
3448
3449
3450
3451
3452
3453
3454
3455
3456
3457
3458
3459
3460
3461
3462
3463
3464
3465
3466
3467
3468
3469
3470
3471
3472
3473
3474
3475
3476
3477
3478
3479
3480
3481
3482
3483
3484
3485
3486
3487
3488
3489
3490
3491
3492
3493
3494
3495
3496
3497
3498
3499
3500
3501
3502
3503
3504
3505
3506
3507
3508
3509
3510
3511
3512
3513
3514
3515
3516
3517
3518
3519
3520
3521
3522
3523
3524
3525
3526
3527
3528
3529
3530
3531
3532
3533
3534
3535
3536
3537
3538
3539
3540
3541
3542
3543
3544
3545
3546
3547
3548
3549
3550
3551
3552
3553
3554
3555
3556
3557
3558
3559
3560
3561
3562
3563
3564
3565
3566
3567
3568
3569
3570
3571
3572
3573
3574
3575
3576
3577
3578
3579
3580
3581
3582
3583
3584
3585
3586
3587
3588
3589
3590
3591
3592
3593
3594
3595
3596
3597
3598
3599
3600
3601
3602
3603
3604
3605
3606
3607
3608
3609
3610
3611
3612
3613
3614
3615
3616
3617
3618
3619
3620
3621
3622
3623
3624
3625
3626
3627
3628
3629
3630
3631
3632
3633
3634
3635
3636
3637
3638
3639
3640
3641
3642
3643
3644
3645
3646
3647
3648
3649
3650
3651
3652
3653
3654
3655
3656
3657
3658
3659
3660
3661
3662
3663
3664
3665
3666
3667
3668
3669
3670
3671
3672
3673
3674
3675
3676
3677
3678
3679
3680
3681
3682
3683
3684
3685
3686
3687
3688
3689
3690
3691
3692
3693
3694
3695
3696
3697
3698
3699
3700
3701
3702
3703
3704
3705
3706
3707
3708
3709
3710
3711
3712
3713
3714
3715
3716
3717
3718
3719
3720
3721
3722
3723
3724
3725
3726
3727
3728
3729
3730
3731
3732
3733
3734
3735
3736
3737
3738
3739
3740
3741
3742
3743
3744
3745
3746
3747
3748
3749
3750
3751
3752
3753
3754
3755
3756
3757
3758
3759
3760
3761
3762
3763
3764
3765
3766
3767
3768
3769
3770
3771
3772
3773
3774
3775
3776
3777
3778
3779
3780
3781
3782
3783
3784
3785
3786
3787
3788
3789
3790
3791
3792
3793
3794
3795
3796
3797
3798
3799
3800
3801
3802
3803
3804
3805
3806
3807
3808
3809
3810
3811
3812
3813
3814
3815
3816
3817
3818
3819
3820
3821
3822
3823
3824
3825
3826
3827
3828
3829
3830
3831
3832
3833
3834
3835
3836
3837
3838
3839
3840
3841
3842
3843
3844
3845
3846
3847
3848
3849
3850
3851
3852
3853
3854
3855
3856
3857
3858
3859
3860
3861
3862
3863
3864
3865
3866
3867
3868
3869
3870
3871
3872
3873
3874
3875
3876
3877
3878
3879
3880
3881
3882
3883
3884
3885
3886
3887
3888
3889
3890
3891
3892
3893
3894
3895
3896
3897
3898
3899
3900
3901
3902
3903
3904
3905
3906
3907
3908
3909
3910
3911
3912
3913
3914
3915
3916
3917
3918
3919
3920
3921
3922
3923
3924
3925
3926
3927
3928
3929
3930
3931
3932
3933
3934
3935
3936
3937
3938
3939
3940
3941
3942
3943
3944
3945
3946
3947
3948
3949
3950
3951
3952
3953
3954
3955
3956
3957
3958
3959
3960
3961
3962
3963
3964
3965
3966
3967
3968
3969
3970
3971
3972
3973
3974
3975
3976
3977
3978
3979
3980
3981
3982
3983
3984
3985
3986
3987
3988
3989
3990
3991
3992
3993
3994
3995
3996
3997
3998
3999
4000
4001
4002
4003
4004
4005
4006
4007
4008
4009
4010
4011
4012
4013
4014
4015
4016
4017
4018
4019
4020
4021
4022
4023
4024
4025
4026
4027
4028
4029
4030
4031
4032
4033
4034
4035
4036
4037
4038
4039
4040
4041
4042
4043
4044
4045
4046
4047
4048
4049
4050
4051
4052
4053
4054
4055
4056
4057
4058
4059
4060
4061
4062
4063
4064
4065
4066
4067
4068
4069
4070
4071
4072
4073
4074
4075
4076
4077
4078
4079
4080
4081
4082
4083
4084
4085
4086
4087
4088
4089
4090
4091
4092
4093
4094
4095
4096
4097
4098
4099
4100
4101
4102
4103
4104
4105
4106
4107
4108
4109
4110
4111
4112
4113
4114
4115
4116
4117
4118
4119
4120
4121
4122
4123
4124
4125
4126
4127
4128
4129
4130
4131
4132
4133
4134
4135
4136
4137
4138
4139
4140
4141
4142
4143
4144
4145
4146
4147
4148
4149
4150
4151
4152
4153
4154
4155
4156
4157
4158
4159
4160
4161
4162
4163
4164
4165
4166
4167
4168
4169
4170
4171
4172
4173
4174
4175
4176
4177
4178
4179
4180
4181
4182
4183
4184
4185
4186
4187
4188
4189
4190
4191
4192
4193
4194
4195
4196
4197
4198
4199
4200
4201
4202
4203
4204
4205
4206
4207
4208
4209
4210
4211
4212
4213
4214
4215
4216
4217
4218
4219
4220
4221
4222
4223
4224
4225
4226
4227
4228
4229
4230
4231
4232
4233
4234
4235
4236
4237
4238
4239
4240
4241
4242
4243
4244
4245
4246
4247
4248
4249
4250
4251
4252
4253
4254
4255
4256
4257
4258
4259
4260
4261
4262
4263
4264
4265
4266
4267
4268
4269
4270
4271
4272
4273
4274
4275
4276
4277
4278
4279
4280
4281
4282
4283
4284
4285
4286
4287
4288
4289
4290
4291
4292
4293
4294
4295
4296
4297
4298
4299
4300
4301
4302
4303
4304
4305
4306
4307
4308
4309
4310
4311
4312
4313
4314
4315
4316
4317
4318
4319
4320
4321
4322
4323
4324
4325
4326
4327
4328
4329
4330
4331
4332
4333
4334
4335
4336
4337
4338
4339
4340
4341
4342
4343
4344
4345
4346
4347
4348
4349
4350
4351
4352
4353
4354
4355
4356
4357
4358
4359
4360
4361
4362
4363
4364
4365
4366
4367
4368
4369
4370
4371
4372
4373
4374
4375
4376
4377
4378
4379
4380
4381
4382
4383
4384
4385
4386
4387
4388
4389
4390
4391
4392
4393
4394
4395
4396
4397
4398
4399
4400
4401
4402
4403
4404
4405
4406
4407
4408
4409
4410
4411
4412
4413
4414
4415
4416
4417
4418
4419
4420
4421
4422
4423
4424
4425
4426
4427
4428
4429
4430
4431
4432
4433
4434
4435
4436
4437
4438
4439
4440
4441
4442
4443
4444
4445
4446
4447
4448
4449
4450
4451
4452
4453
4454
4455
4456
4457
4458
4459
4460
4461
4462
4463
4464
4465
4466
4467
4468
4469
4470
4471
4472
4473
4474
4475
4476
4477
4478
4479
4480
4481
4482
4483
4484
4485
4486
4487
4488
4489
4490
4491
4492
4493
4494
4495
4496
4497
4498
4499
4500
4501
4502
4503
4504
4505
4506
4507
4508
4509
4510
4511
4512
4513
4514
4515
4516
4517
4518
4519
4520
4521
4522
4523
4524
4525
4526
4527
4528
4529
4530
4531
4532
4533
4534
4535
4536
4537
4538
4539
4540
4541
4542
4543
4544
4545
4546
4547
4548
4549
4550
4551
4552
4553
4554
4555
4556
4557
4558
4559
4560
4561
4562
4563
4564
4565
4566
4567
4568
4569
4570
4571
4572
4573
4574
4575
4576
4577
4578
4579
4580
4581
4582
4583
4584
4585
4586
4587
4588
4589
4590
4591
4592
4593
4594
4595
4596
4597
4598
4599
4600
4601
4602
4603
4604
4605
4606
4607
4608
4609
4610
4611
4612
4613
4614
4615
4616
4617
4618
4619
4620
4621
4622
4623
4624
4625
4626
4627
4628
4629
4630
4631
4632
4633
4634
4635
4636
4637
4638
4639
4640
4641
4642
4643
4644
4645
4646
4647
4648
4649
4650
4651
4652
4653
4654
4655
4656
4657
4658
4659
4660
4661
4662
4663
4664
4665
4666
4667
4668
4669
4670
4671
4672
4673
4674
4675
4676
4677
4678
4679
4680
4681
4682
4683
4684
4685
4686
4687
4688
4689
4690
4691
4692
4693
4694
4695
4696
4697
4698
4699
4700
4701
4702
4703
4704
4705
4706
4707
4708
4709
4710
4711
4712
4713
4714
4715
4716
4717
4718
4719
4720
4721
4722
4723
4724
4725
4726
4727
4728
4729
4730
4731
4732
4733
4734
4735
4736
4737
4738
4739
4740
4741
4742
4743
4744
4745
4746
4747
4748
4749
4750
4751
4752
4753
4754
4755
4756
4757
4758
4759
4760
4761
4762
4763
4764
4765
4766
4767
4768
4769
4770
4771
4772
4773
4774
4775
4776
4777
4778
4779
4780
4781
4782
4783
4784
4785
4786
4787
4788
4789
4790
4791
4792
4793
4794
4795
4796
4797
4798
4799
4800
4801
4802
4803
4804
4805
4806
4807
4808
4809
4810
4811
4812
4813
4814
4815
4816
4817
4818
4819
4820
4821
4822
4823
4824
4825
4826
4827
4828
4829
4830
4831
4832
4833
4834
4835
4836
4837
4838
4839
4840
4841
4842
4843
4844
4845
4846
4847
4848
4849
4850
4851
4852
4853
4854
4855
4856
4857
4858
4859
4860
4861
4862
4863
4864
4865
4866
4867
4868
4869
4870
4871
4872
4873
4874
4875
4876
4877
4878
4879
4880
4881
4882
4883
4884
4885
4886
4887
4888
4889
4890
4891
4892
4893
4894
4895
4896
4897
4898
4899
4900
4901
4902
4903
4904
4905
4906
4907
4908
4909
4910
4911
4912
4913
4914
4915
4916
4917
4918
4919
4920
4921
4922
4923
4924
4925
4926
4927
4928
4929
4930
4931
4932
4933
4934
4935
4936
4937
4938
4939
4940
4941
4942
4943
4944
4945
4946
4947
4948
4949
4950
4951
4952
4953
4954
4955
4956
4957
4958
4959
4960
4961
4962
4963
4964
4965
4966
4967
4968
4969
4970
4971
4972
4973
4974
4975
4976
4977
4978
4979
4980
4981
4982
4983
4984
4985
4986
4987
4988
4989
4990
4991
4992
4993
4994
4995
4996
4997
4998
4999
5000
5001
5002
5003
5004
5005
5006
5007
5008
5009
5010
5011
5012
5013
5014
5015
5016
5017
5018
5019
5020
5021
5022
5023
5024
5025
5026
5027
5028
5029
5030
5031
5032
5033
5034
5035
5036
5037
5038
5039
5040
5041
5042
5043
5044
5045
5046
5047
5048
5049
5050
5051
5052
5053
5054
5055
5056
5057
5058
5059
5060
5061
5062
5063
5064
5065
5066
5067
5068
5069
5070
5071
5072
5073
5074
5075
5076
5077
5078
5079
5080
5081
5082
5083
5084
5085
5086
5087
5088
5089
5090
5091
5092
5093
5094
5095
5096
5097
5098
5099
5100
5101
5102
5103
5104
5105
5106
5107
5108
5109
5110
5111
5112
5113
5114
5115
5116
5117
5118
5119
5120
5121
5122
5123
5124
5125
5126
5127
5128
5129
5130
5131
5132
5133
5134
5135
5136
5137
5138
5139
5140
5141
5142
5143
5144
5145
5146
5147
5148
5149
5150
5151
5152
5153
5154
5155
5156
5157
5158
5159
5160
5161
5162
5163
5164
5165
5166
5167
5168
5169
5170
5171
5172
5173
5174
5175
5176
5177
5178
5179
5180
5181
5182
5183
5184
5185
5186
5187
5188
5189
5190
5191
5192
5193
5194
5195
5196
5197
5198
5199
5200
5201
5202
5203
5204
5205
5206
5207
5208
5209
5210
5211
5212
5213
5214
5215
5216
5217
5218
5219
5220
5221
5222
5223
5224
5225
5226
5227
5228
5229
5230
5231
5232
5233
5234
5235
5236
5237
5238
5239
5240
5241
5242
5243
5244
5245
5246
5247
5248
5249
5250
5251
5252
5253
5254
5255
5256
5257
5258
5259
5260
5261
5262
5263
5264
5265
5266
5267
5268
5269
5270
5271
5272
5273
5274
5275
5276
5277
5278
5279
5280
5281
5282
5283
5284
5285
5286
5287
5288
5289
5290
5291
5292
5293
5294
5295
5296
5297
5298
5299
5300
5301
5302
5303
5304
5305
5306
5307
5308
5309
5310
5311
5312
5313
5314
5315
5316
5317
5318
5319
5320
5321
5322
5323
5324
5325
5326
5327
5328
5329
5330
5331
5332
5333
5334
5335
5336
5337
5338
5339
5340
5341
5342
5343
5344
5345
5346
5347
5348
5349
5350
5351
5352
5353
5354
5355
5356
5357
5358
5359
5360
5361
5362
5363
5364
5365
5366
5367
5368
5369
5370
5371
5372
5373
5374
5375
5376
5377
5378
5379
5380
5381
5382
5383
5384
5385
5386
5387
5388
5389
5390
5391
5392
5393
5394
5395
5396
5397
5398
5399
5400
5401
5402
5403
5404
5405
5406
5407
5408
5409
5410
5411
5412
5413
5414
5415
5416
5417
5418
5419
5420
5421
5422
5423
5424
5425
5426
5427
5428
5429
5430
5431
5432
5433
5434
5435
5436
5437
5438
5439
5440
5441
5442
5443
5444
5445
5446
5447
5448
5449
5450
5451
5452
5453
5454
5455
5456
5457
5458
5459
5460
5461
5462
5463
5464
5465
5466
5467
5468
5469
5470
5471
5472
5473
5474
5475
5476
5477
5478
5479
5480
5481
5482
5483
5484
5485
5486
5487
5488
5489
5490
5491
5492
5493
5494
5495
5496
5497
5498
5499
5500
5501
5502
5503
5504
5505
5506
5507
5508
5509
5510
5511
5512
//! Kafka consumer implementation.
//!
//! This module provides:
//! - Async consumer for receiving messages
//! - Consumer groups with rebalancing
//! - Offset management (auto and manual commit)
//! - Partition assignment strategies
//!
//! # Delivery Semantics
//!
//! Krafka provides **at-least-once** delivery semantics by default, which is the
//! standard Kafka consumer behavior:
//!
//! 1. Messages are delivered to the application via `poll()`
//! 2. Offsets are committed after processing (auto-commit or manual)
//! 3. If the consumer crashes after processing but before commit, messages may
//!    be redelivered on restart
//!
//! This is the safest default as it ensures no message loss. For use cases that
//! cannot tolerate duplicates, applications should implement idempotent processing.
//!
//! ## Controlling Commit Behavior
//!
//! - **Auto-commit** (default): Offsets are committed periodically in the background
//! - **Manual commit**: Disable auto-commit and call `commit()` explicitly
//!
//! For at-most-once semantics (where message loss is acceptable but duplicates are not),
//! commit offsets before processing:
//!
//! ```ignore
//! let records = consumer.poll(Duration::from_secs(1)).await?;
//! consumer.commit().await?;  // Commit BEFORE processing
//! for record in records {
//!     process(record);  // If this crashes, message is lost
//! }
//! ```

mod config;
mod fetch_session;
mod group;
mod offset;
mod record;
mod stream;

pub mod compacted;

pub use compacted::{
    CompactedTable, CompactedTopicConsumer, CompactedTopicConsumerBuilder, TableChange,
};
pub use config::{
    AutoOffsetReset, ConsumerConfig, ConsumerConfigBuilder, GroupProtocol, IsolationLevel,
    PartitionAssignmentStrategy,
};
pub use group::{
    ConsumerGroup, ConsumerRebalanceListener, CooperativeStickyAssignor, GroupCoordinator,
    GroupMember, GroupState, HeartbeatController, HeartbeatStatus, MemberAssignment,
    NoOpRebalanceListener, PartitionAssignor, RangeAssignor, RoundRobinAssignor,
};
pub use offset::{OffsetAndMetadata, OffsetStore, ResetOffset};
pub use record::{ConsumerRecord, ConsumerRecords, TopicPartition};
pub use stream::ConsumerStream;

use std::collections::{HashMap, HashSet};
use std::sync::Arc;
use std::time::{Duration, Instant};

use parking_lot::Mutex as SyncMutex;
use tokio::sync::RwLock;
use tracing::{debug, error, info, warn};

use crate::auth::AuthConfig;
use crate::error::{KrafkaError, Result};
use crate::metadata::{BrokerInfo, ClusterMetadata, TopicInfo};
use crate::metrics::{ConnectionMetrics, ConsumerMetrics};
use crate::network::{ConnectionConfig, ConnectionPool};
use crate::protocol::{
    ApiKey, FetchPartitionRequest, FetchRequest, FetchResponse, FetchTopicRequest,
    ListOffsetsRequest, ListOffsetsRequestPartition, ListOffsetsRequestTopic, ListOffsetsResponse,
    RecordBatch, VersionedDecode, VersionedEncode, validate_topic_name, validate_topic_names,
    versions,
};
use crate::{Offset, PartitionId};

/// Maximum number of cooperative rebalance rejoin rounds before deferring
/// to the next poll cycle. Cascading membership changes in large groups
/// may require multiple rounds; this cap prevents unbounded looping.
const MAX_COOPERATIVE_ROUNDS: usize = 5;

use fetch_session::FetchSessionCache;

// ── LOCK ORDER ──────────────────────────────────────────────────────────────
//
// `Consumer` holds several synchronization primitives. Two rules prevent
// deadlocks and executor stalls:
//
//   1. Acquire locks in the order listed below. Never acquire an
//      earlier-numbered lock while holding a later-numbered one.
//   2. If you hold an **async** lock (`tokio::sync::{RwLock, Mutex}`) across
//      an `.await`, the awaited operation must not (transitively) acquire
//      any lock numbered **lower** than the one held. In practice: do not
//      call back into `Consumer` methods while holding a lock. Release with
//      `drop(...)` or a scoped `{ … }` block first.
//
// Order:
//   1. `subscriptions`         (async — read across `poll()` network paths)
//   2. `assignments`           (async — read across `poll()` network paths)
//   3. `offsets`               (async — read across commit/fetch RPC paths)
//   4. `paused`                (async — read across `poll()` network paths)
//   5. `partition_state`       (async — per-partition fetch-derived caches)
//   6. `recv_buffer`           (sync — `parking_lot::Mutex`, pure mutation)
//   7. `fetch_sessions`        (sync — `parking_lot::Mutex`, pure mutation;
//                               ALWAYS release before fetch RPC send/recv)
//   8. `last_auto_commit`      (sync — `parking_lot::Mutex<Instant>`)
//
// The sync (`parking_lot`) primitives are chosen only for critical sections
// with NO `.await` inside. They can still block a Tokio worker thread if
// contended, and some sections are O(n) (e.g. `recv_buffer.retain`), so do
// not assume they are always nanosecond-scale. Keep them short, avoid
// contention, and always release them before async work, network I/O, or
// callbacks back into `Consumer`. Do not convert any async lock above
// without first auditing every call site for `.await` under the lock.
//
// Per-partition caches (`high_watermark`, `log_start_offset`,
// `preferred_replica`, `offset_retry_backoff`) were previously four separate
// `RwLock<HashMap<_, _>>` fields. They are consolidated into a single
// `partition_state` map so that revocation / reset / close paths cannot leave
// a partition partially populated. Adding a new per-partition cache? Add a
// field to `PartitionState`; do not introduce another `RwLock`.

/// Per-topic-partition state cached locally by the consumer.
///
/// All fields are populated from fetch responses or consumer-protocol
/// feedback. Grouping them under a single lock (`Consumer::partition_state`)
/// guarantees that revocation and reset paths cannot leave a partition in a
/// partially-populated state — a silent bug class that existed when each
/// cache lived under its own `RwLock`.
///
/// `Default` returns an all-`None` state, suitable for `entry().or_default()`
/// on first insert.
#[derive(Default)]
struct PartitionState {
    /// Latest known high watermark (log-end offset), from `FetchResponse`.
    /// `None` until first observed in a fetch response for this partition
    /// (the broker reports it on every response with `high_watermark >= 0`,
    /// including empty and error responses).
    high_watermark: Option<Offset>,
    /// Latest known log start offset, from `FetchResponse`.
    /// `None` until first observed in a fetch response for this partition
    /// (reported in Fetch v5+ whenever `log_start_offset >= 0`, including
    /// empty and error responses).
    log_start_offset: Option<Offset>,
    /// KIP-392 preferred read replica and its expiry time.
    ///
    /// When a broker returns a `preferred_read_replica` in a fetch response,
    /// subsequent fetches for that partition are routed to the indicated
    /// replica until the entry expires (after `metadata_max_age`).
    /// `None` means the leader should be used (the default).
    preferred_replica: Option<(crate::BrokerId, Instant)>,
    /// Next allowed retry time and current exponential backoff interval for
    /// offset-resolution failures. `None` once the partition is successfully
    /// resolved or was never retried. Prevents retry storms when offset
    /// resolution fails persistently (e.g. broker unavailable).
    offset_retry_backoff: Option<(Instant, Duration)>,
}

/// Cluster metadata snapshot returned by [`Consumer::fetch_metadata`].
#[non_exhaustive]
#[derive(Debug, Clone)]
pub struct FetchMetadataResult {
    /// All brokers known to the cluster.
    pub brokers: Vec<BrokerInfo>,
    /// Topics returned in the metadata snapshot.
    ///
    /// When [`Consumer::fetch_metadata`] is called with `None`, this contains
    /// all cached topics. When called with `Some(topic)`, this contains the
    /// matching topic if found, or is empty if that topic was not found in the
    /// cluster.
    pub topics: Vec<TopicInfo>,
}

/// A Kafka consumer.
pub struct Consumer {
    /// Consumer configuration.
    config: ConsumerConfig,
    /// Cluster metadata.
    metadata: Arc<ClusterMetadata>,
    /// Connection pool.
    pool: Arc<ConnectionPool>,
    /// Subscribed topics.
    subscriptions: RwLock<HashSet<String>>,
    /// Assigned partitions.
    assignments: RwLock<HashMap<String, Vec<PartitionId>>>,
    /// Current offsets.
    offsets: RwLock<HashMap<(String, PartitionId), Offset>>,
    /// Paused partitions.
    paused: RwLock<HashSet<(String, PartitionId)>>,
    /// Whether the consumer is closed.
    closed: std::sync::atomic::AtomicBool,
    /// Group coordinator for full group protocol support.
    group_coordinator: Option<Arc<GroupCoordinator>>,
    /// Consumer metrics.
    metrics: Arc<ConsumerMetrics>,
    /// Rebalance listener.
    rebalance_listener: Arc<dyn ConsumerRebalanceListener>,
    /// Consumer interceptor.
    interceptor: Arc<dyn crate::interceptor::ConsumerInterceptor>,
    /// Last auto-commit time (for auto-commit timer).
    ///
    /// Held only for a single read/write of an `Instant` — no `.await` under
    /// the lock — so a sync `parking_lot::Mutex` is the correct primitive.
    last_auto_commit: SyncMutex<Instant>,
    /// Buffer for records returned by `recv()`.
    /// `poll()` may return multiple records; `recv()` buffers the rest here.
    ///
    /// All call sites mutate (`pop_front` / `extend` / `clear`) or read
    /// `len()` without crossing an `.await`, so a sync `parking_lot::Mutex`
    /// is used instead of a tokio async lock. The old `RwLock` had no
    /// concurrent readers in practice — every access took the write side.
    recv_buffer: SyncMutex<std::collections::VecDeque<ConsumerRecord>>,
    /// Per-broker fetch session cache (KIP-227).
    ///
    /// Every critical section is pure sync (session bookkeeping: build
    /// request, update from response, reset on error). The actual fetch
    /// RPC `send_request().await` is always performed **after** the lock is
    /// released — never while it is held — so a sync `parking_lot::Mutex`
    /// is the correct primitive. If you add a new use site, keep the
    /// critical section straight-line-sync.
    fetch_sessions: SyncMutex<FetchSessionCache>,
    /// Consolidated per-partition state: high watermark, log start offset,
    /// preferred replica (KIP-392), and offset-retry backoff. A single lock
    /// replaces what was previously four separate `RwLock<HashMap>` fields;
    /// see the `LOCK ORDER` comment above and [`PartitionState`] for details.
    partition_state: RwLock<HashMap<(String, PartitionId), PartitionState>>,
}

/// Compute aggregate lag from offset and high-watermark caches.
///
/// Returns `(total_lag, max_lag)` where `total_lag` is the sum across all
/// partitions (using `saturating_add`) and `max_lag` is the per-partition
/// maximum. Only partitions present in both maps contribute.
///
/// **Staleness caveat**: high watermarks are refreshed from each fetch
/// response (including empty and error responses) for partitions the
/// consumer polls. Partitions that are not being polled — or whose broker
/// is unreachable — will retain a stale watermark, so the reported lag
/// becomes increasingly inaccurate the longer fetches are skipped. Lag
/// should be treated as *eventually consistent*. For precise lag values,
/// issue a `ListOffsets` RPC externally.
fn compute_aggregate_lag(
    offsets: &HashMap<(String, PartitionId), Offset>,
    partition_state: &HashMap<(String, PartitionId), PartitionState>,
) -> (u64, u64) {
    let mut total_lag: u64 = 0;
    let mut max_lag: u64 = 0;
    for (key, state) in partition_state {
        if let (Some(watermark), Some(&position)) = (state.high_watermark, offsets.get(key)) {
            let partition_lag = (watermark - position).max(0) as u64;
            total_lag = total_lag.saturating_add(partition_lag);
            max_lag = max_lag.max(partition_lag);
        }
    }
    (total_lag, max_lag)
}

/// Result of routing assigned partitions to brokers for fetching.
struct FetchRoutingPlan {
    /// Partitions grouped by target broker ID.
    partitions_by_broker: HashMap<crate::BrokerId, Vec<(String, PartitionId)>>,
    /// Preferred replica entries that have expired and should be removed.
    expired_preferred: Vec<(String, PartitionId)>,
    /// Partitions that have neither a known leader nor a valid preferred
    /// replica and will not be fetched this round.
    skipped: Vec<(String, PartitionId)>,
}

/// Build a per-broker fetch plan from pre-filtered partition keys,
/// preferred replicas, and leader information.
///
/// `non_paused_keys` should contain only assigned, non-paused partitions
/// (the caller is responsible for filtering). For each key the function
/// checks whether a preferred replica exists and is not expired. If so,
/// the partition is routed to that replica, regardless of whether a leader
/// is known. If there is no valid preferred replica, the function falls
/// back to the leader if one is known; otherwise the partition is skipped.
///
/// This is a pure function extracted from `Consumer::poll()` so that the
/// routing logic can be unit-tested without a live broker.
fn build_fetch_routing_plan(
    non_paused_keys: Vec<(String, PartitionId)>,
    partition_state: &HashMap<(String, PartitionId), PartitionState>,
    leaders: &HashMap<(String, PartitionId), crate::BrokerId>,
    now: Instant,
) -> FetchRoutingPlan {
    let mut partitions_by_broker: HashMap<crate::BrokerId, Vec<(String, PartitionId)>> =
        HashMap::new();
    let mut expired_preferred: Vec<(String, PartitionId)> = Vec::new();
    let mut skipped: Vec<(String, PartitionId)> = Vec::new();

    for key in non_paused_keys {
        // Check for a valid (non-expired) preferred replica
        let target_broker = match partition_state.get(&key).and_then(|s| s.preferred_replica) {
            Some((replica_id, expiry)) if now < expiry => Some(replica_id),
            Some(_) => {
                expired_preferred.push(key.clone());
                None
            }
            None => None,
        };

        let broker_id = match target_broker {
            Some(id) => id,
            None => {
                if let Some(&leader_id) = leaders.get(&key) {
                    leader_id
                } else {
                    skipped.push(key);
                    continue;
                }
            }
        };

        partitions_by_broker.entry(broker_id).or_default().push(key);
    }

    FetchRoutingPlan {
        partitions_by_broker,
        expired_preferred,
        skipped,
    }
}

/// Group a flat `&[(&str, PartitionId)]` slice into a
/// `HashMap<String, Vec<PartitionId>>` keyed by topic name, preserving
/// insertion order within each topic's partition list.
///
/// This is the pure grouping step shared by [`Consumer::offsets_for_times`]
/// and its unit test so that the test exercises the real logic.
fn group_topic_partitions(partitions: &[(&str, PartitionId)]) -> HashMap<String, Vec<PartitionId>> {
    let mut grouped: HashMap<String, Vec<PartitionId>> = HashMap::new();
    for &(topic, partition) in partitions {
        // Use get_mut to avoid allocating a String key on every lookup;
        // only insert (and therefore allocate) when the topic is first seen.
        if let Some(v) = grouped.get_mut(topic) {
            v.push(partition);
        } else {
            grouped.insert(topic.to_string(), vec![partition]);
        }
    }
    grouped
}

/// Apply a [`ListOffsetsResponse`] into a per-partition result map.
///
/// For each partition in the response:
/// - `error_code == None` → inserts `Ok(offset)`.
/// - any other error code → inserts `Err(KrafkaError::broker(...))` and logs
///   a `warn!`.
///
/// Partitions not mentioned in the response are left unchanged, so the
/// pre-populated `Err("no leader found …")` sentinel that `resolve_list_offsets`
/// inserts before calling this function is preserved for any partition the
/// broker did not report.
fn apply_list_offsets_response(
    response: &ListOffsetsResponse,
    result: &mut HashMap<(String, PartitionId), Result<Offset>>,
) {
    for topic_resp in &response.topics {
        for part_resp in &topic_resp.partitions {
            let key = (topic_resp.name.clone(), part_resp.partition_index);
            if part_resp.error_code.is_ok() {
                result.insert(key, Ok(part_resp.offset));
            } else {
                warn!(
                    "ListOffsets error for {}-{}: {:?}",
                    topic_resp.name, part_resp.partition_index, part_resp.error_code
                );
                result.insert(
                    key,
                    Err(KrafkaError::broker(
                        part_resp.error_code,
                        format!(
                            "ListOffsets error for {}-{}",
                            topic_resp.name, part_resp.partition_index
                        ),
                    )),
                );
            }
        }
    }
}

/// Compute revoked partitions as `old - new`.
///
/// Used by eager rebalance cleanup to preserve local state for partitions that
/// remain assigned to the same consumer after the rebalance completes.
fn revoked_partitions_diff(
    old: &HashMap<String, Vec<PartitionId>>,
    new: &HashMap<String, Vec<PartitionId>>,
) -> Vec<TopicPartition> {
    let new_sets: HashMap<&String, HashSet<PartitionId>> = new
        .iter()
        .map(|(topic, partitions)| (topic, partitions.iter().copied().collect()))
        .collect();
    let mut result = Vec::new();
    for (topic, partitions) in old {
        let new_set = new_sets.get(topic);
        for &partition in partitions {
            let gone = new_set.is_none_or(|assigned| !assigned.contains(&partition));
            if gone {
                result.push(TopicPartition::new(topic, partition));
            }
        }
    }
    result
}

impl Consumer {
    /// Create a new consumer builder.
    pub fn builder() -> ConsumerBuilder {
        ConsumerBuilder::default()
    }

    /// Create a new consumer with the given configuration.
    async fn new(config: ConsumerConfig) -> Result<Self> {
        let mut pool_config_builder = ConnectionConfig::builder()
            .client_id(&config.client_id)
            .request_timeout(config.request_timeout);

        if let Some(ref auth) = config.auth {
            pool_config_builder = pool_config_builder.auth(auth.clone());
        }

        #[cfg(feature = "socks5")]
        if let Some(ref proxy) = config.proxy {
            pool_config_builder = pool_config_builder.proxy(proxy.clone());
        }

        let mut pool_config = pool_config_builder.build();
        pool_config.init_tls().await?;

        let pool = Arc::new(ConnectionPool::new(pool_config));
        pool.start_idle_evictor();

        let bootstrap_servers = crate::util::parse_bootstrap_servers(&config.bootstrap_servers)?;

        let metadata = Arc::new({
            let mut meta =
                ClusterMetadata::new(bootstrap_servers, pool.clone(), config.metadata_max_age)
                    .with_recovery_strategy(config.metadata_recovery_strategy)
                    .with_rebootstrap_trigger(config.metadata_recovery_rebootstrap_trigger);
            if let Some(ttl) = config.metadata_topic_cache_ttl {
                meta = meta.with_topic_cache_ttl(ttl);
            } else {
                meta = meta.with_topic_cache_ttl_disabled();
            }
            meta
        });

        // Initial metadata fetch
        metadata.refresh().await?;

        // Create group coordinator if group_id is specified
        let group_coordinator = if let Some(ref group_id) = config.group_id {
            Some(Arc::new(
                GroupCoordinator::new(
                    group_id.clone(),
                    pool.clone(),
                    metadata.clone(),
                    config.session_timeout,
                    config.heartbeat_interval,
                    config.max_poll_interval, // rebalance_timeout matches Java client's max.poll.interval.ms
                )
                .with_assignor_strategy(config.partition_assignment_strategy)
                .with_group_instance_id(config.group_instance_id.clone())
                .with_isolation_level(config.isolation_level.to_i8())
                .with_group_protocol(config.group_protocol),
            ))
        } else {
            None
        };

        let metrics = Arc::new(ConsumerMetrics::default());

        info!(
            "Consumer initialized with {} brokers{}",
            metadata.brokers().len(),
            if let Some(ref gid) = config.group_id {
                format!(", group_id='{gid}'")
            } else {
                String::new()
            }
        );

        Ok(Self {
            config,
            metadata,
            pool,
            subscriptions: RwLock::new(HashSet::new()),
            assignments: RwLock::new(HashMap::new()),
            offsets: RwLock::new(HashMap::new()),
            paused: RwLock::new(HashSet::new()),
            closed: std::sync::atomic::AtomicBool::new(false),
            group_coordinator,
            metrics,
            rebalance_listener: Arc::new(NoOpRebalanceListener),
            interceptor: Arc::new(crate::interceptor::NoOpConsumerInterceptor),
            last_auto_commit: SyncMutex::new(Instant::now()),
            recv_buffer: SyncMutex::new(std::collections::VecDeque::new()),
            fetch_sessions: SyncMutex::new(FetchSessionCache::new()),
            partition_state: RwLock::new(HashMap::new()),
        })
    }

    /// Subscribe to topics.
    ///
    /// Replaces the current subscription with the given topics (matching
    /// the Kafka Java client's replace semantics).
    pub async fn subscribe(&self, topics: &[&str]) -> Result<()> {
        // H6: reject empty / oversize topic names at ingress so they cannot
        // reach the panicking `KafkaString::encode` path via the MetadataRequest
        // / Heartbeat / subscription payload.
        validate_topic_names(topics.iter().copied())?;

        // Scope the write lock so it is dropped before network I/O
        {
            let mut subscriptions = self.subscriptions.write().await;
            subscriptions.clear();
            for topic in topics {
                subscriptions.insert((*topic).to_string());
            }
        }

        // Refresh metadata for subscribed topics
        self.metadata.refresh_for_topics(Some(topics)).await?;

        // If we have a group coordinator, join the group
        if let Some(ref coordinator) = self.group_coordinator {
            let topic_strings: Vec<String> = topics.iter().map(|s| s.to_string()).collect();
            let mut topics_sorted = topic_strings.clone();
            topics_sorted.sort();

            if coordinator.is_consumer_protocol() {
                // KIP-848: defer to poll(), which handles incremental
                // assignment via the background heartbeat task.  subscribe()
                // only updates the subscription; the next heartbeat will carry
                // the new topic list to the coordinator.

                // Detect topic changes while active — trigger rejoin so the
                // next poll sends a full heartbeat with the new subscription.
                {
                    let state = coordinator.state().await;
                    if state == GroupState::Stable {
                        let mut old_sorted = coordinator.subscribed_topics().await;
                        old_sorted.sort();
                        if old_sorted != topics_sorted {
                            coordinator.trigger_rejoin().await;
                        }
                    }
                }

                coordinator.set_subscribed_topics(topic_strings).await;
            } else if coordinator.is_cooperative() {
                // Cooperative (KIP-429): defer the join/sync to poll(), which
                // implements the full two-phase rebalance protocol (revocations,
                // on_partitions_revoked callback, second rejoin). subscribe()
                // only updates the subscription metadata; poll() will detect
                // needs_rejoin() and drive the cooperative flow.

                // Detect topic changes while Stable — mark for rejoin.
                {
                    let state = coordinator.state().await;
                    if state == GroupState::Stable {
                        let mut old_sorted = coordinator.subscribed_topics().await;
                        old_sorted.sort();
                        if old_sorted != topics_sorted {
                            coordinator.set_preparing_rebalance().await;
                        }
                    }
                }

                coordinator.set_subscribed_topics(topic_strings).await;
            } else {
                // Eager: join immediately in subscribe() — single-phase is correct.

                // Snapshot old assignment before the join. If a JoinGroup/SyncGroup
                // occurs, we must revoke the old partitions (eager = revoke all)
                // to clean up per-partition state and notify the listener.
                let old_assignments = self.assignments.read().await.clone();

                let (assignment, joined) =
                    coordinator.ensure_active_membership(&topic_strings).await?;

                if joined {
                    // An actual JoinGroup/SyncGroup occurred (first join or topic change).

                    // Eager revocation: notify the listener for the full previous
                    // assignment, but only clean up partitions that were actually
                    // revoked by the new assignment so retained partitions keep
                    // their local pause/offset/fetch state.
                    if !old_assignments.is_empty() {
                        let revoked: Vec<TopicPartition> = old_assignments
                            .iter()
                            .flat_map(|(t, ps)| ps.iter().map(move |&p| TopicPartition::new(t, p)))
                            .collect();
                        self.safe_on_partitions_revoked(&revoked);

                        let revoked_tuples: Vec<(String, PartitionId)> =
                            revoked_partitions_diff(&old_assignments, &assignment.partitions)
                                .into_iter()
                                .map(|tp| (tp.topic, tp.partition))
                                .collect();
                        self.apply_partition_revocations(&revoked_tuples).await;
                    }

                    self.metrics.rebalances.inc();
                }

                // Update our assignments based on the group assignment
                {
                    let mut assignments = self.assignments.write().await;
                    assignments.clear();
                    for (topic, partitions) in &assignment.partitions {
                        assignments.insert(topic.clone(), partitions.clone());
                    }
                }

                if joined {
                    // Notify listener of assignment (matches Java client behavior:
                    // ConsumerRebalanceListener.onPartitionsAssigned is invoked on every
                    // successful rebalance, including the very first one).
                    let assigned: Vec<TopicPartition> = assignment
                        .partitions
                        .iter()
                        .flat_map(|(t, ps)| ps.iter().map(move |&p| TopicPartition::new(t, p)))
                        .collect();
                    self.safe_on_partitions_assigned(&assigned);

                    // Update assigned_partitions metric
                    self.metrics.assigned_partitions.set(assigned.len() as u64);

                    // Fetch committed offsets for our assigned partitions
                    self.fetch_and_apply_committed_offsets(&assignment.partitions)
                        .await?;
                }
            }

            debug!("Subscribed to topics via group coordinator: {:?}", topics);
        } else {
            // Assign all partitions (simple assignment without group coordination)
            let mut assignments = self.assignments.write().await;
            for topic in topics {
                if let Some(topic_info) = self.metadata.topic(topic) {
                    let partitions: Vec<_> =
                        topic_info.partitions.iter().map(|p| p.partition).collect();
                    assignments.insert((*topic).to_string(), partitions);
                }
            }
            let assigned_snapshot = assignments.clone();
            drop(assignments);

            // Update metric for standalone partition count
            let count: usize = assigned_snapshot.values().map(|p| p.len()).sum();
            self.metrics.assigned_partitions.set(count as u64);

            // Apply auto_offset_reset for non-group consumers.
            // Without this, all partitions default to offset 0 regardless of
            // the configured auto_offset_reset policy.
            self.apply_auto_offset_reset(&assigned_snapshot).await?;

            debug!("Subscribed to topics: {:?}", topics);
        }

        Ok(())
    }

    /// Apply per-partition cleanup for revoked partitions.
    ///
    /// Removes revoked entries from `assignments`, `offsets`, `paused`,
    /// `recv_buffer`, and `partition_state` (the consolidated cache that
    /// holds high watermark, log start offset, preferred replica, and
    /// offset-retry backoff). Fetch sessions are NOT reset here —
    /// `build_request()` automatically computes `forgotten_topics` diffs
    /// from the updated assignment, preserving KIP-227 incremental fetch
    /// benefits. Called by all cooperative revocation paths.
    async fn apply_partition_revocations(&self, revoked: &[(String, PartitionId)]) {
        // Build per-topic set of revoked partition IDs for O(T * P) removal
        // instead of O(R * P) when many partitions of the same topic are revoked.
        let revoked_by_topic: HashMap<&str, HashSet<PartitionId>> = {
            let mut m: HashMap<&str, HashSet<PartitionId>> = HashMap::new();
            for (topic, partition) in revoked {
                m.entry(topic.as_str()).or_default().insert(*partition);
            }
            m
        };

        // Precompute owned keys once to avoid repeated String clones in each
        // removal loop below.
        let revoked_keys: Vec<(String, PartitionId)> =
            revoked.iter().map(|(t, p)| (t.clone(), *p)).collect();

        // Remove from assignments
        {
            let mut assignments = self.assignments.write().await;
            for (topic, revoked_parts) in &revoked_by_topic {
                if let Some(parts) = assignments.get_mut(*topic) {
                    parts.retain(|p| !revoked_parts.contains(p));
                    if parts.is_empty() {
                        assignments.remove(*topic);
                    }
                }
            }
        }
        // Remove offsets for revoked partitions
        {
            let mut offsets = self.offsets.write().await;
            for key in &revoked_keys {
                offsets.remove(key);
            }
        }
        // Discard buffered records from revoked partitions
        {
            let revoked_set: HashSet<(&str, PartitionId)> =
                revoked_keys.iter().map(|(t, p)| (t.as_str(), *p)).collect();
            let mut buf = self.recv_buffer.lock();
            buf.retain(|r| !revoked_set.contains(&(r.topic.as_str(), r.partition)));
            self.metrics.buffered_records.set(buf.len() as u64);
        }
        // Clear paused state for revoked partitions
        {
            let mut paused = self.paused.write().await;
            for key in &revoked_keys {
                paused.remove(key);
            }
            self.metrics.paused_partitions.set(paused.len() as u64);
        }
        // Clear all per-partition fetch-derived state (high watermark, log
        // start offset, preferred replica, offset-retry backoff) in a single
        // lock acquisition. This replaces four independent `RwLock` writes
        // that previously had to be kept in sync by hand.
        {
            let mut partition_state = self.partition_state.write().await;
            for key in &revoked_keys {
                partition_state.remove(key);
            }
        }
        // Recompute lag metrics from remaining caches so revoked
        // partitions no longer contribute to exported values.
        self.recompute_lag_metrics().await;
    }

    /// Finalize a cooperative rebalance: compute newly-assigned diff, update
    /// assignments, fire `on_partitions_assigned`, fetch committed offsets for
    /// new partitions, and record owned partitions in the sticky assignor.
    async fn finalize_cooperative_assignment(
        &self,
        coordinator: &GroupCoordinator,
        assignment: &MemberAssignment,
        old_assignments: &HashMap<String, Vec<PartitionId>>,
    ) -> Result<()> {
        // Build HashSet index for O(1) membership checks.
        let old_sets: HashMap<&String, HashSet<PartitionId>> = old_assignments
            .iter()
            .map(|(t, ps)| (t, ps.iter().copied().collect()))
            .collect();

        // Determine newly assigned partitions (new - old)
        let mut newly_assigned = Vec::new();
        for (topic, partitions) in &assignment.partitions {
            let old_set = old_sets.get(topic);
            for &p in partitions {
                let is_new = old_set.is_none_or(|os| !os.contains(&p));
                if is_new {
                    newly_assigned.push(TopicPartition::new(topic, p));
                }
            }
        }

        // Update to final assignment
        {
            let mut assignments = self.assignments.write().await;
            assignments.clear();
            for (topic, partitions) in &assignment.partitions {
                assignments.insert(topic.clone(), partitions.clone());
            }
        }

        // Notify listener with the full post-rebalance assignment,
        // not just the diff. Always fire, even when the assignment
        // is empty (e.g., more consumers than partitions).
        let full_assigned: Vec<TopicPartition> = assignment
            .partitions
            .iter()
            .flat_map(|(t, ps)| ps.iter().map(move |&p| TopicPartition::new(t, p)))
            .collect();
        self.rebalance_listener
            .on_partitions_assigned(&full_assigned);
        self.metrics
            .assigned_partitions
            .set(full_assigned.len() as u64);

        // Fetch committed offsets for newly assigned partitions only
        // (retained partitions already have tracked offsets).
        if !newly_assigned.is_empty() {
            let new_parts = Self::group_partitions_by_topic(&newly_assigned);
            self.fetch_and_apply_committed_offsets(&new_parts).await?;
        }

        // Record final assignment so the next rebalance's
        // join_group metadata reports correct owned partitions.
        let member_id = coordinator.member_id().await;
        coordinator.record_owned_partitions(&member_id, assignment);

        Ok(())
    }

    /// Clear all per-partition state after an eager revocation or unsubscribe/close.
    ///
    /// Resets fetch sessions, offsets, buffered records, paused set, and the
    /// consolidated [`PartitionState`] map (high watermark, log start offset,
    /// preferred replica, offset-retry backoff), then zeros the lag metrics.
    async fn clear_partition_state(&self) {
        self.fetch_sessions.lock().reset_all();
        self.offsets.write().await.clear();
        self.recv_buffer.lock().clear();
        self.paused.write().await.clear();
        self.partition_state.write().await.clear();
        self.metrics.buffered_records.set(0);
        self.metrics.paused_partitions.set(0);
        self.metrics.lag.set(0);
        self.metrics.lag_max.set(0);
    }

    // ── Panic-safe rebalance listener wrappers ──────────────────────────
    //
    // User-provided `ConsumerRebalanceListener` callbacks are wrapped in
    // `catch_unwind` to prevent a panicking listener from unwinding through
    // the consumer task, which would skip `leave_group()` and other cleanup.
    // This mirrors the interceptor subsystem's panic isolation pattern.

    /// Invoke `on_partitions_assigned` on the rebalance listener, catching panics.
    fn safe_on_partitions_assigned(&self, partitions: &[TopicPartition]) {
        if let Err(_panic) = std::panic::catch_unwind(std::panic::AssertUnwindSafe(|| {
            self.rebalance_listener.on_partitions_assigned(partitions);
        })) {
            tracing::error!(
                partition_count = partitions.len(),
                "ConsumerRebalanceListener::on_partitions_assigned panicked"
            );
        }
    }

    /// Invoke `on_partitions_revoked` on the rebalance listener, catching panics.
    fn safe_on_partitions_revoked(&self, partitions: &[TopicPartition]) {
        if let Err(_panic) = std::panic::catch_unwind(std::panic::AssertUnwindSafe(|| {
            self.rebalance_listener.on_partitions_revoked(partitions);
        })) {
            tracing::error!(
                partition_count = partitions.len(),
                "ConsumerRebalanceListener::on_partitions_revoked panicked"
            );
        }
    }

    /// Invoke `on_partitions_lost` on the rebalance listener, catching panics.
    fn safe_on_partitions_lost(&self, partitions: &[TopicPartition]) {
        if let Err(_panic) = std::panic::catch_unwind(std::panic::AssertUnwindSafe(|| {
            self.rebalance_listener.on_partitions_lost(partitions);
        })) {
            tracing::error!(
                partition_count = partitions.len(),
                "ConsumerRebalanceListener::on_partitions_lost panicked"
            );
        }
    }

    /// Recompute lag and lag_max gauges from cached offsets and high watermarks.
    ///
    /// Call after any mutation of `self.offsets` or the `high_watermark` field
    /// of `self.partition_state` so the exported metrics always reflect the
    /// current consumer position. Acquires read locks in documented order:
    /// offsets → partition_state.
    ///
    /// This performs an O(partitions) full scan via [`compute_aggregate_lag`].
    /// An incremental (delta-based) approach was considered but rejected:
    /// the typical partition count per consumer (tens to low thousands) makes
    /// the scan complete in microseconds, while incremental bookkeeping would
    /// add complexity and drift risk for negligible gain. Callers on the hot
    /// path (e.g. `poll()`) already guard calls behind a change-detection flag.
    async fn recompute_lag_metrics(&self) {
        let offsets = self.offsets.read().await;
        let partition_state = self.partition_state.read().await;
        let (total_lag, max_lag) = compute_aggregate_lag(&offsets, &partition_state);
        self.metrics.lag.set(total_lag);
        self.metrics.lag_max.set(max_lag);
    }

    /// Send an inline heartbeat, invoke the revocation callback, apply
    /// partition revocations, and update the metric + sticky-assignor state.
    ///
    /// Returns `true` if an inline heartbeat signalled session invalidation
    /// and poll() should return early.
    async fn apply_revocation_round(
        &self,
        coordinator: &Arc<GroupCoordinator>,
        revoked_tuples: &[(String, PartitionId)],
        revoked_tps: &[TopicPartition],
    ) -> Result<bool> {
        // Send an inline heartbeat before invoking the user callback
        // to avoid session timeout if the callback is slow.
        match coordinator.send_heartbeat().await {
            Ok(status) if coordinator.handle_inline_heartbeat_status(status).await => {
                return Ok(true);
            }
            Err(e) => {
                warn!("Pre-revocation heartbeat failed: {}", e);
            }
            _ => {}
        }
        // Commit offsets for the partitions we are about to lose so the
        // new owner sees up-to-date committed positions.
        if self.config.enable_auto_commit
            && let Err(e) = self.commit().await
        {
            if e.is_retriable() {
                warn!(
                    "Auto-commit before cooperative revocation failed (retriable): {}",
                    e
                );
            } else {
                error!(
                    "Auto-commit before cooperative revocation failed (fatal): {}",
                    e
                );
            }
        }
        self.safe_on_partitions_revoked(revoked_tps);
        self.apply_partition_revocations(revoked_tuples).await;

        // Update metric and owned-partition state in a single lock
        // acquisition. The metric is set eagerly so it stays accurate
        // even if a subsequent Phase 2 round returns early.
        let member_id = coordinator.member_id().await;
        let current = self.assignments.read().await;
        let count: usize = current.values().map(|ps| ps.len()).sum();
        self.metrics.assigned_partitions.set(count as u64);
        let owned = MemberAssignment {
            partitions: current.clone(),
        };
        drop(current);
        coordinator.record_owned_partitions(&member_id, &owned);

        Ok(false)
    }

    /// Handle group rebalance and inline heartbeat during poll.
    ///
    /// Returns `true` if poll() should return an empty result immediately
    /// (e.g., cooperative rebalance requires another poll cycle).
    async fn handle_group_rebalance(&self) -> Result<bool> {
        let Some(ref coordinator) = self.group_coordinator else {
            return Ok(false);
        };

        if coordinator.needs_rejoin().await {
            let topics: Vec<String> = self.subscriptions.read().await.iter().cloned().collect();
            if !topics.is_empty() {
                coordinator.set_subscribed_topics(topics.clone()).await;

                if coordinator.is_consumer_protocol() {
                    // KIP-848: when the consumer needs to (re)join — initial
                    // join (Unjoined), post-fencing rejoin, or subscription
                    // change — send a full heartbeat with all fields and
                    // (re)start the background task.  When the heartbeat task
                    // delivered a normal assignment update (Stable, same
                    // topics), ensure_active_membership is a no-op.
                    coordinator.ensure_active_membership(&topics).await?;
                    self.handle_kip848_rebalance(coordinator).await?;
                } else if coordinator.is_cooperative() {
                    if self.handle_cooperative_rebalance(coordinator).await? {
                        return Ok(true);
                    }
                } else {
                    self.handle_eager_rebalance(coordinator, &topics).await?;
                }
            }
        }

        // Check if inline heartbeat is needed.
        // Skip for KIP-848 — the background ConsumerGroupHeartbeat task handles
        // heartbeats; sending classic Heartbeat requests would use the wrong API.
        if !coordinator.is_consumer_protocol() && coordinator.is_heartbeat_overdue().await {
            match coordinator.send_heartbeat().await {
                Ok(status) if coordinator.handle_inline_heartbeat_status(status).await => {
                    debug!("Heartbeat indicated rejoin needed");
                }
                Err(e) => {
                    warn!("Inline heartbeat failed: {}", e);
                }
                _ => {}
            }
        }

        Ok(false)
    }

    /// Handle cooperative incremental rebalance (KIP-429).
    ///
    /// Returns `true` if poll() should return an empty result immediately,
    /// which happens when an inline heartbeat signals rejoin or when the
    /// cooperative round limit is exceeded.
    async fn handle_cooperative_rebalance(
        &self,
        coordinator: &Arc<GroupCoordinator>,
    ) -> Result<bool> {
        // Phase 1: join+sync to get new target assignment
        let (new_assignment, to_revoke) = coordinator.perform_cooperative_join_and_sync().await?;

        if !to_revoke.is_empty() {
            // Revoke only the diff — keep consuming unaffected partitions
            let revoked: Vec<TopicPartition> = to_revoke
                .iter()
                .map(|(t, p)| TopicPartition::new(t, *p))
                .collect();
            if self
                .apply_revocation_round(coordinator, &to_revoke, &revoked)
                .await?
            {
                return Ok(true);
            }
            self.metrics.rebalances.inc();

            // Phase 2: rejoin to finalize after revocations.
            // In rare cases (concurrent topic changes, racing rebalances),
            // additional revocations may be needed. Loop with a bound.
            coordinator.trigger_rejoin().await;
            let mut final_assignment = MemberAssignment::empty();
            for round in 0..MAX_COOPERATIVE_ROUNDS {
                let (assignment, extra_revoke) =
                    coordinator.perform_cooperative_join_and_sync().await?;
                final_assignment = assignment;

                if extra_revoke.is_empty() {
                    break;
                }

                // Process additional revocations (including final round)
                let extra_revoked: Vec<TopicPartition> = extra_revoke
                    .iter()
                    .map(|(t, p)| TopicPartition::new(t, *p))
                    .collect();
                if self
                    .apply_revocation_round(coordinator, &extra_revoke, &extra_revoked)
                    .await?
                {
                    return Ok(true);
                }

                if round == MAX_COOPERATIVE_ROUNDS - 1 {
                    warn!(
                        "Cooperative rebalance exceeded {} rounds with pending revocations; \
                         this may indicate cascading membership changes. \
                         Deferring assignment to next poll cycle.",
                        MAX_COOPERATIVE_ROUNDS
                    );
                    // Start heartbeat to avoid session timeout while we
                    // defer the additional cooperative rebalance round
                    // to the next poll cycle. Do NOT apply final_assignment
                    // since it still required another rejoin. Set state
                    // directly instead of trigger_rejoin() to avoid
                    // killing the heartbeat task via Rejoin command.
                    coordinator.start_heartbeat_task().await;
                    coordinator.set_preparing_rebalance().await;
                    // Note: rebalances metric was already incremented
                    // at Phase 1 entry; do not double-count here.
                    // assigned_partitions metric was already updated
                    // after apply_partition_revocations above.
                    return Ok(true);
                }

                coordinator.trigger_rejoin().await;
            }

            // Finalize cooperative assignment: update assignments,
            // fire on_partitions_assigned, fetch offsets, record owned.
            let old_assignments = self.assignments.read().await.clone();
            self.finalize_cooperative_assignment(coordinator, &final_assignment, &old_assignments)
                .await?;
        } else {
            // No revocations — assignment is final in one round
            let old_assignments = self.assignments.read().await.clone();

            // Build HashSet index of new partitions for O(1) lookups.
            let new_sets: HashMap<&String, HashSet<PartitionId>> = new_assignment
                .partitions
                .iter()
                .map(|(t, ps)| (t, ps.iter().copied().collect()))
                .collect();

            // Determine partitions removed in this rebalance
            // (e.g., reassigned to another member, topic deleted).
            // This is a clean cooperative revocation, not an unclean
            // loss, so use on_partitions_revoked (not on_partitions_lost).
            let mut revoked_parts: Vec<TopicPartition> = Vec::new();
            for (topic, partitions) in &old_assignments {
                let new_set = new_sets.get(topic);
                for &p in partitions {
                    let gone = new_set.is_none_or(|ns| !ns.contains(&p));
                    if gone {
                        revoked_parts.push(TopicPartition::new(topic, p));
                    }
                }
            }
            if !revoked_parts.is_empty() {
                self.safe_on_partitions_revoked(&revoked_parts);
                let revoked_tuples: Vec<(String, PartitionId)> = revoked_parts
                    .iter()
                    .map(|tp| (tp.topic.clone(), tp.partition))
                    .collect();
                self.apply_partition_revocations(&revoked_tuples).await;
            }

            self.metrics.rebalances.inc();

            // Finalize cooperative assignment: update assignments,
            // fire on_partitions_assigned, fetch offsets, record owned.
            self.finalize_cooperative_assignment(coordinator, &new_assignment, &old_assignments)
                .await?;
        }

        Ok(false)
    }

    /// Handle KIP-848 server-side assignment: diff-based callbacks.
    ///
    /// The KIP-848 background heartbeat task stores the new assignment in
    /// `GroupCoordinator.assignment` and signals rebalance. This method reads
    /// the current assignment, computes the diff against the Consumer's local
    /// assignments, fires revocation/assignment callbacks for changed
    /// partitions, and fetches committed offsets for newly added ones.
    async fn handle_kip848_rebalance(&self, coordinator: &Arc<GroupCoordinator>) -> Result<()> {
        let new_assignment = coordinator.assignment().await;
        let old_assignments = self.assignments.read().await.clone();

        // Build HashSets for O(n) diffing instead of Vec::contains.
        let old_sets: HashMap<&String, HashSet<PartitionId>> = old_assignments
            .iter()
            .map(|(t, ps)| (t, ps.iter().copied().collect()))
            .collect();
        let new_sets: HashMap<&String, HashSet<PartitionId>> = new_assignment
            .partitions
            .iter()
            .map(|(t, ps)| (t, ps.iter().copied().collect()))
            .collect();

        // Compute revoked partitions: in old but not in new.
        let mut revoked: Vec<TopicPartition> = Vec::new();
        for (topic, old_set) in &old_sets {
            let new_set = new_sets.get(*topic);
            for &p in old_set {
                let retained = new_set.is_some_and(|ns| ns.contains(&p));
                if !retained {
                    revoked.push(TopicPartition::new(*topic, p));
                }
            }
        }

        // Compute newly assigned partitions: in new but not in old.
        let mut assigned: Vec<TopicPartition> = Vec::new();
        for (topic, new_set) in &new_sets {
            let old_set = old_sets.get(*topic);
            for &p in new_set {
                let was_assigned = old_set.is_some_and(|os| os.contains(&p));
                if !was_assigned {
                    assigned.push(TopicPartition::new(*topic, p));
                }
            }
        }

        if revoked.is_empty() && assigned.is_empty() {
            // No actual change — the heartbeat task may have signalled
            // rebalance for state reasons (e.g. first assignment).
            // Still need to ensure our local assignments are in sync.
            if old_assignments.is_empty() && !new_assignment.partitions.is_empty() {
                // First assignment: treat all partitions as newly assigned.
                for (topic, parts) in &new_assignment.partitions {
                    for &p in parts {
                        assigned.push(TopicPartition::new(topic, p));
                    }
                }
            } else if !old_assignments.is_empty() {
                // Had partitions before, diff shows no movement — nothing to do.
                return Ok(());
            }
            // Remaining case: old_assignments is empty.  Either
            //   (a) new is also empty  — first heartbeat with an empty
            //       assignment (more consumers than partitions), or
            //   (b) new is non-empty   — handled by the branch above.
            // For (a) we fall through so on_partitions_assigned fires,
            // matching cooperative/eager paths which always invoke the
            // callback on the initial assignment.
        }

        // Fire revocation callback and clean up per-partition state.
        if !revoked.is_empty() {
            // KIP-848 §revocation: commit offsets for the partitions we are
            // about to lose before invoking the user callback, so the new
            // owner sees up-to-date committed positions. The old assignments
            // are still active at this point, so `commit()` includes them.
            if self.config.enable_auto_commit
                && let Err(e) = self.commit().await
            {
                if e.is_retriable() {
                    warn!(
                        "Auto-commit before KIP-848 revocation failed (retriable): {}",
                        e
                    );
                } else {
                    error!(
                        "Auto-commit before KIP-848 revocation failed (fatal): {}",
                        e
                    );
                }
            }
            self.safe_on_partitions_revoked(&revoked);
            let revoked_tuples: Vec<(String, PartitionId)> = revoked
                .iter()
                .map(|tp| (tp.topic.clone(), tp.partition))
                .collect();
            self.apply_partition_revocations(&revoked_tuples).await;
        }

        // Update assignments to the new state.
        {
            let mut assignments = self.assignments.write().await;
            assignments.clear();
            for (topic, partitions) in &new_assignment.partitions {
                assignments.insert(topic.clone(), partitions.clone());
            }
        }

        self.metrics.rebalances.inc();

        // Fire assignment callback with the full post-rebalance assignment
        // (consistent with the cooperative/eager paths in this crate).
        // Always fire, even when the assignment is empty or only revocations
        // occurred, so listeners can react to the post-rebalance state.
        let full_assignment: Vec<TopicPartition> = new_assignment
            .partitions
            .iter()
            .flat_map(|(topic, partitions)| {
                partitions
                    .iter()
                    .copied()
                    .map(move |partition| TopicPartition::new(topic, partition))
            })
            .collect();
        self.safe_on_partitions_assigned(&full_assignment);

        let count: usize = new_assignment.partitions.values().map(|ps| ps.len()).sum();
        self.metrics.assigned_partitions.set(count as u64);

        // Fetch committed offsets only for newly assigned partitions.
        if !assigned.is_empty() {
            let new_parts = Self::group_partitions_by_topic(&assigned);
            self.fetch_and_apply_committed_offsets(&new_parts).await?;
        }

        // KIP-848 §revocation-ack: after processing revocations, send an
        // immediate heartbeat with the updated owned partitions so the
        // coordinator can proceed with the rebalance.
        if !revoked.is_empty() {
            coordinator.acknowledge_revocation().await;
        }

        Ok(())
    }

    /// Handle eager rebalance: revoke all partitions, then reassign from scratch.
    async fn handle_eager_rebalance(
        &self,
        coordinator: &Arc<GroupCoordinator>,
        topics: &[String],
    ) -> Result<()> {
        let old_assignments = self.assignments.read().await.clone();
        if !old_assignments.is_empty() {
            let revoked: Vec<TopicPartition> = old_assignments
                .iter()
                .flat_map(|(t, ps)| ps.iter().map(move |&p| TopicPartition::new(t, p)))
                .collect();
            // Commit offsets for all partitions before the eager revoke-all,
            // so the group has up-to-date committed positions.
            if self.config.enable_auto_commit
                && let Err(e) = self.commit().await
            {
                if e.is_retriable() {
                    warn!(
                        "Auto-commit before eager revocation failed (retriable): {}",
                        e
                    );
                } else {
                    error!("Auto-commit before eager revocation failed (fatal): {}", e);
                }
            }
            self.safe_on_partitions_revoked(&revoked);
            self.clear_partition_state().await;

            // Clear assignments immediately after revocation so that
            // if ensure_active_membership fails below, the next poll
            // won't re-fire on_partitions_revoked for already-revoked
            // partitions. Matches the Java client's behavior of
            // clearing subscription state after the eager revoke phase.
            self.assignments.write().await.clear();
            self.metrics.assigned_partitions.set(0);
        }

        self.metrics.rebalances.inc();

        // `joined` is always true here: handle_group_rebalance gates on
        // needs_rejoin(), so ensure_active_membership always performs a
        // full JoinGroup/SyncGroup.
        let (assignment, _joined) = coordinator.ensure_active_membership(topics).await?;

        // Update our assignments
        let mut assignments = self.assignments.write().await;
        assignments.clear();
        for (topic, partitions) in &assignment.partitions {
            assignments.insert(topic.clone(), partitions.clone());
        }
        drop(assignments);

        // Notify listener of newly assigned partitions
        let assigned: Vec<TopicPartition> = assignment
            .partitions
            .iter()
            .flat_map(|(t, ps)| ps.iter().map(move |&p| TopicPartition::new(t, p)))
            .collect();
        self.safe_on_partitions_assigned(&assigned);
        self.metrics.assigned_partitions.set(assigned.len() as u64);

        // Fetch committed offsets for new assignment
        self.fetch_and_apply_committed_offsets(&assignment.partitions)
            .await?;

        Ok(())
    }

    /// Group topic-partitions into a map keyed by topic name.
    fn group_partitions_by_topic(
        partitions: &[TopicPartition],
    ) -> HashMap<String, Vec<PartitionId>> {
        let mut map: HashMap<String, Vec<PartitionId>> = HashMap::new();
        for tp in partitions {
            map.entry(tp.topic.clone()).or_default().push(tp.partition);
        }
        map
    }

    /// Fetch committed offsets and apply auto_offset_reset for partitions without committed offsets.
    ///
    /// Called after group assignment to initialize partition offsets:
    /// 1. Fetch committed offsets from the group coordinator
    /// 2. For partitions with no committed offset, apply the configured auto_offset_reset policy
    async fn fetch_and_apply_committed_offsets(
        &self,
        assigned: &HashMap<String, Vec<PartitionId>>,
    ) -> Result<()> {
        let coordinator = match self.group_coordinator {
            Some(ref c) => c,
            None => return Ok(()),
        };

        // Fetch committed offsets
        let committed = coordinator.fetch_committed_offsets(assigned).await?;

        // Determine which partitions are missing committed offsets
        let mut need_reset: Vec<(String, PartitionId)> = Vec::new();
        let mut offsets = self.offsets.write().await;

        // Log the initial offsets state before processing committed offsets
        debug!("fetch_and_apply: existing offsets: {:?}", *offsets);

        for (topic, partitions) in assigned {
            for &partition in partitions {
                let key = (topic.clone(), partition);

                // Respect user-set offsets (e.g., from seek() in on_partitions_assigned).
                // If the caller already positioned this partition, do not overwrite.
                if offsets.contains_key(&key) {
                    debug!(
                        "Keeping existing offset for {}-{} (user-set or prior)",
                        topic, partition
                    );
                    continue;
                }

                let committed_val = committed.get(&key);
                if let Some(&offset) = committed_val
                    && offset >= 0
                {
                    debug!(
                        "Using committed offset {} for {}-{}",
                        offset, topic, partition
                    );
                    offsets.insert(key, offset);
                    continue;
                }
                // No committed offset or negative (unknown)
                debug!(
                    "No committed offset for {}-{} (committed={:?}), will auto-reset",
                    topic, partition, committed_val
                );
                need_reset.push(key);
            }
        }

        if need_reset.is_empty() {
            return Ok(());
        }

        // Apply auto_offset_reset
        if let Some(timestamp) = self.config.auto_offset_reset.to_offset() {
            // Group partitions by topic for list_offsets call
            let mut reset_partitions: HashMap<String, Vec<PartitionId>> = HashMap::new();
            for (topic, partition) in &need_reset {
                reset_partitions
                    .entry(topic.clone())
                    .or_default()
                    .push(*partition);
            }

            let resolved = coordinator
                .list_offsets(&reset_partitions, timestamp)
                .await?;

            for (key, offset) in &resolved {
                offsets.insert(key.clone(), *offset);
            }

            // Fallback: if the group coordinator's list_offsets silently
            // dropped some partitions (partition-level errors), resolve
            // them individually via the direct ListOffsets v1 path.
            for (topic, partition) in &need_reset {
                let key = (topic.clone(), *partition);
                if !resolved.contains_key(&key) && !offsets.contains_key(&key) {
                    debug!(
                        "Falling back to direct ListOffsets for {}-{} \
                         (coordinator path returned no result)",
                        topic, partition
                    );
                    // Release offsets lock temporarily for the network call
                    drop(offsets);
                    match self.resolve_list_offset(topic, *partition, timestamp).await {
                        Ok(offset) => {
                            offsets = self.offsets.write().await;
                            offsets.insert(key, offset);
                        }
                        Err(e) => {
                            warn!(
                                "Fallback offset resolution failed for {}-{}: {}",
                                topic, partition, e
                            );
                            offsets = self.offsets.write().await;
                        }
                    }
                }
            }
        } else {
            // AutoOffsetReset::None — fail if no committed offset
            let missing: Vec<String> = need_reset.iter().map(|(t, p)| format!("{t}-{p}")).collect();
            return Err(KrafkaError::invalid_state(format!(
                "no committed offset for partitions and auto.offset.reset=none: {}",
                missing.join(", ")
            )));
        }

        // Drop the write lock before recomputing lag metrics to avoid
        // deadlocking with the read lock that recompute_lag_metrics acquires.
        drop(offsets);

        self.recompute_lag_metrics().await;
        Ok(())
    }

    /// Assign specific partitions manually.
    ///
    /// Manual assignment and group subscription are mutually exclusive.
    /// This method returns an error if a group coordinator is active.
    pub async fn assign(&self, topic: &str, partitions: Vec<PartitionId>) -> Result<()> {
        // H6: reject empty / oversize topic names at ingress so they cannot
        // reach the panicking `KafkaString::encode` path via MetadataRequest /
        // FetchRequest / OffsetFetchRequest.
        validate_topic_name(topic)?;

        if self.group_coordinator.is_some() {
            return Err(KrafkaError::invalid_state(
                "cannot use manual partition assignment with consumer group subscription",
            ));
        }

        // Refresh metadata so we can resolve partition leaders for offset lookup
        self.metadata.refresh_for_topics(Some(&[topic])).await?;

        let topic_owned = topic.to_string();

        let mut assignments = self.assignments.write().await;
        assignments.insert(topic_owned.clone(), partitions.clone());

        let mut subscriptions = self.subscriptions.write().await;
        subscriptions.insert(topic_owned.clone());
        drop(subscriptions);
        drop(assignments);

        // Apply auto_offset_reset for manually assigned partitions
        let mut assigned = HashMap::new();
        debug!("Assigned partitions for {}: {:?}", topic, partitions);
        assigned.insert(topic_owned, partitions);
        self.apply_auto_offset_reset(&assigned).await?;

        Ok(())
    }

    /// Apply auto_offset_reset policy for partitions that have no tracked offset.
    ///
    /// This resolves initial offsets based on the configured `auto_offset_reset`
    /// policy (Earliest, Latest, or None). Used by both group and non-group
    /// consumers during partition assignment.
    async fn apply_auto_offset_reset(
        &self,
        assigned: &HashMap<String, Vec<PartitionId>>,
    ) -> Result<()> {
        // Collect partitions that don't already have a tracked offset
        let need_reset: Vec<(String, PartitionId)> = {
            let offsets = self.offsets.read().await;
            let mut need = Vec::new();
            for (topic, partitions) in assigned {
                for &p in partitions {
                    let key = (topic.clone(), p);
                    if !offsets.contains_key(&key) {
                        need.push(key);
                    }
                }
            }
            need
        };

        if need_reset.is_empty() {
            return Ok(());
        }

        if let Some(timestamp) = self.config.auto_offset_reset.to_offset() {
            let reset_pairs: Vec<(&str, PartitionId)> =
                need_reset.iter().map(|(t, p)| (t.as_str(), *p)).collect();
            let batch = group_topic_partitions(&reset_pairs);

            let resolved = self.resolve_list_offsets(&batch, timestamp).await;
            let mut offsets = self.offsets.write().await;
            for (key, result) in &resolved {
                if let Ok(offset) = result {
                    offsets.insert(key.clone(), *offset);
                }
            }
            drop(offsets);

            // Log any partitions that weren't resolved (no leader, broker error, etc.)
            for key in &need_reset {
                if resolved.get(key).is_none_or(|r| r.is_err()) {
                    warn!(
                        "Failed to resolve offset for {}-{}, will retry on next poll",
                        key.0, key.1
                    );
                }
            }
        } else {
            // AutoOffsetReset::None — fail if no offset
            let missing = need_reset
                .iter()
                .map(|(t, p)| format!("{t}-{p}"))
                .collect::<Vec<_>>()
                .join(", ");
            return Err(KrafkaError::invalid_state(format!(
                "no offset for partitions and auto.offset.reset=none: {missing}"
            )));
        }

        self.recompute_lag_metrics().await;
        Ok(())
    }

    /// Seek to a specific offset.
    pub async fn seek(&self, topic: &str, partition: PartitionId, offset: Offset) -> Result<()> {
        {
            let mut offsets = self.offsets.write().await;
            offsets.insert((topic.to_string(), partition), offset);
        }
        self.recompute_lag_metrics().await;
        debug!("Seek to offset {} for {}-{}", offset, topic, partition);
        Ok(())
    }

    /// Seek to the beginning.
    pub async fn seek_to_beginning(&self, topic: &str, partition: PartitionId) -> Result<()> {
        self.seek(topic, partition, 0).await
    }

    /// Seek to the end (latest offset).
    ///
    /// Sets the consumer position to the high watermark, so subsequent polls
    /// will only return new messages produced after this call.
    ///
    /// This resolves the actual latest offset via a ListOffsets RPC to the
    /// partition leader. The Kafka Fetch API does not interpret special offset
    /// values like -1; those are only meaningful in the ListOffsets API.
    pub async fn seek_to_end(&self, topic: &str, partition: PartitionId) -> Result<()> {
        // Resolve the actual latest offset via ListOffsets (timestamp=-1 means latest)
        let offset = self.resolve_list_offset(topic, partition, -1).await?;
        self.seek(topic, partition, offset).await
    }

    /// Look up the earliest offset whose message timestamp is greater than or
    /// equal to the given timestamp, for each listed `(topic, partition)`.
    ///
    /// Uses the ListOffsets API. Requests are batched by leader broker so each
    /// broker receives at most one RPC.
    ///
    /// Every input partition appears in the returned map:
    /// - `Ok(offset)` — the broker returned a valid offset (`-1` means no
    ///   message exists at or after the timestamp for that partition).
    /// - `Err(e)` — a partition-level broker error (e.g. `NotLeaderForPartition`)
    ///   or a transport failure prevented resolution for this partition.
    ///
    /// # Example
    ///
    /// ```ignore
    /// let results = consumer
    ///     .offsets_for_times(&[("orders", 0), ("orders", 1)], 1_700_000_000_000)
    ///     .await;
    /// for ((topic, partition), result) in &results {
    ///     match result {
    ///         Ok(offset) => println!("{topic}-{partition}: {offset}"),
    ///         Err(e) => eprintln!("{topic}-{partition}: error {e}"),
    ///     }
    /// }
    /// ```
    pub async fn offsets_for_times(
        &self,
        partitions: &[(&str, PartitionId)],
        timestamp: i64,
    ) -> HashMap<(String, PartitionId), Result<Offset>> {
        // Validate all topic names up front; surface invalid ones as per-partition
        // Err entries rather than letting them reach protocol encoding.
        let mut result: HashMap<(String, PartitionId), Result<Offset>> = HashMap::new();
        let mut valid: Vec<(&str, PartitionId)> = Vec::with_capacity(partitions.len());
        for &(topic, partition) in partitions {
            match validate_topic_name(topic) {
                Ok(()) => valid.push((topic, partition)),
                Err(e) => {
                    result.insert((topic.to_string(), partition), Err(e));
                }
            }
        }
        if !valid.is_empty() {
            let grouped = group_topic_partitions(&valid);
            result.extend(self.resolve_list_offsets(&grouped, timestamp).await);
        }
        result
    }

    /// Look up the earliest offset whose message timestamp is greater than or
    /// equal to the given timestamp, for every partition of a single topic.
    ///
    /// Convenience wrapper around [`Consumer::offsets_for_times`] that resolves
    /// the topic's partitions from metadata so callers don't have to list
    /// them. Always refreshes topic metadata before deriving the partition
    /// list so the results reflect the latest leader assignments (the refresh
    /// is skipped by the metadata layer if cached metadata is still fresh).
    ///
    /// Returns `Err` if the topic cannot be found after the metadata refresh.
    /// On success, each `PartitionId` maps to `Ok(offset)` or `Err(e)` —
    /// see [`Consumer::offsets_for_times`] for per-partition semantics.
    pub async fn offsets_for_times_for_topic(
        &self,
        topic: &str,
        timestamp: i64,
    ) -> Result<HashMap<PartitionId, Result<Offset>>> {
        validate_topic_name(topic)?;
        self.metadata.refresh_for_topics(Some(&[topic])).await?;
        let info = self
            .metadata
            .topic(topic)
            .ok_or_else(|| KrafkaError::invalid_state(format!("topic not found: {topic}")))?;

        // Build the grouped map directly — topic name is already validated and
        // the partition list comes from trusted metadata, so bypass the
        // per-entry validation loop inside `offsets_for_times`.
        let mut grouped: HashMap<String, Vec<PartitionId>> = HashMap::new();
        grouped.insert(
            topic.to_string(),
            info.partitions.iter().map(|p| p.partition).collect(),
        );
        let results = self.resolve_list_offsets(&grouped, timestamp).await;

        Ok(results
            .into_iter()
            .map(|((_, p), result)| (p, result))
            .collect())
    }

    /// Fetch the low (log start) and high (latest) watermarks for a partition.
    ///
    /// Issues two ListOffsets RPCs to the partition leader — one for the
    /// earliest offset (`timestamp = -2`) and one for the latest
    /// (`timestamp = -1`) — and returns `(low, high)`. Both RPCs are issued
    /// concurrently.
    pub async fn fetch_watermarks(
        &self,
        topic: &str,
        partition: PartitionId,
    ) -> Result<(Offset, Offset)> {
        validate_topic_name(topic)?;
        let (low, high) = tokio::join!(
            self.resolve_list_offset(topic, partition, -2),
            self.resolve_list_offset(topic, partition, -1),
        );
        Ok((low?, high?))
    }

    /// Return a snapshot of cluster metadata (brokers and topics).
    ///
    /// If `topic` is `Some`, only that topic is returned and the metadata
    /// layer is asked to refresh that topic first (the network call is
    /// skipped if cached metadata is still fresh). If `None`, a snapshot of
    /// all currently cached topics is returned without triggering a refresh
    /// (cached data may be partial or stale).
    pub async fn fetch_metadata(&self, topic: Option<&str>) -> Result<FetchMetadataResult> {
        if let Some(name) = topic {
            validate_topic_name(name)?;
            self.metadata.refresh_for_topics(Some(&[name])).await?;
        }

        let brokers = self.metadata.brokers();
        let topics = match topic {
            Some(name) => self
                .metadata
                .topic(name)
                .map(|t| vec![t])
                .unwrap_or_default(),
            None => self.metadata.topics(),
        };

        Ok(FetchMetadataResult { brokers, topics })
    }

    /// Resolve an offset timestamp via the ListOffsets API.
    ///
    /// `timestamp` should be:
    /// - `-1` for the latest offset (high watermark)
    /// - `-2` for the earliest available offset
    async fn resolve_list_offset(
        &self,
        topic: &str,
        partition: PartitionId,
        timestamp: i64,
    ) -> Result<Offset> {
        let mut partitions = HashMap::new();
        let topic_owned = topic.to_string();
        partitions.insert(topic_owned.clone(), vec![partition]);
        let mut results = self.resolve_list_offsets(&partitions, timestamp).await;
        results
            .remove(&(topic_owned, partition))
            .unwrap_or_else(|| {
                Err(KrafkaError::protocol(format!(
                    "no offset returned for {topic}-{partition}"
                )))
            })
    }

    /// Resolve offsets for multiple partitions in batched ListOffsets RPCs,
    /// grouped by leader broker so each broker receives at most one request.
    ///
    /// Every requested partition appears in the returned map:
    /// - `Ok(offset)` — broker returned a valid offset.
    /// - `Err(e)` — the partition could not be resolved (no leader, connection
    ///   failure, or a partition-level broker error).
    async fn resolve_list_offsets(
        &self,
        partitions: &HashMap<String, Vec<PartitionId>>,
        timestamp: i64,
    ) -> HashMap<(String, PartitionId), Result<Offset>> {
        if partitions.is_empty() {
            return HashMap::new();
        }

        // Pre-populate every partition with a default error; replaced on success.
        let mut result: HashMap<(String, PartitionId), Result<Offset>> = partitions
            .iter()
            .flat_map(|(topic, parts)| {
                let topic = topic.clone();
                parts.iter().map(move |&p| {
                    let msg = format!("no leader found for {topic}-{p}");
                    ((topic.clone(), p), Err(KrafkaError::invalid_state(msg)))
                })
            })
            .collect();

        // Group partitions by leader broker
        let mut by_leader: HashMap<crate::BrokerId, Vec<(String, PartitionId)>> = HashMap::new();
        let mut leaderless: Vec<(String, PartitionId)> = Vec::new();
        for (topic, parts) in partitions {
            for &p in parts {
                if let Some(leader) = self.metadata.leader(topic, p) {
                    by_leader
                        .entry(leader)
                        .or_default()
                        .push((topic.clone(), p));
                } else {
                    leaderless.push((topic.clone(), p));
                }
            }
        }

        // Retry leaderless partitions after a metadata refresh
        if !leaderless.is_empty() {
            // Deduplicate topics to avoid redundant refresh work when multiple
            // partitions of the same topic are leaderless.
            let topic_set: HashSet<&str> = leaderless.iter().map(|(t, _)| t.as_str()).collect();
            let topics: Vec<&str> = topic_set.into_iter().collect();
            if let Err(err) = self.metadata.refresh_for_topics(Some(&topics)).await {
                warn!(
                    "Failed to refresh metadata for leaderless topics {:?}: {}",
                    topics, err
                );
            }
            for (topic, partition) in leaderless {
                if let Some(leader) = self.metadata.leader(&topic, partition) {
                    by_leader
                        .entry(leader)
                        .or_default()
                        .push((topic, partition));
                } else {
                    warn!(
                        "No leader for {}-{} after metadata refresh",
                        topic, partition
                    );
                    // result[(topic, partition)] retains its default Err
                }
            }
        }

        for (&leader_id, leader_partitions) in &by_leader {
            // Group into ListOffsetsRequest topics
            let mut topics_map: HashMap<String, Vec<ListOffsetsRequestPartition>> = HashMap::new();
            for (topic, partition) in leader_partitions {
                topics_map
                    .entry(topic.clone())
                    .or_default()
                    .push(ListOffsetsRequestPartition {
                        partition_index: *partition,
                        // ListOffsets v1/v2 do not serialize current_leader_epoch; use sentinel.
                        current_leader_epoch: -1,
                        timestamp,
                    });
            }

            let topics: Vec<ListOffsetsRequestTopic> = topics_map
                .into_iter()
                .map(|(name, parts)| ListOffsetsRequestTopic {
                    name,
                    partitions: parts,
                })
                .collect();

            let request = ListOffsetsRequest {
                replica_id: -1,
                isolation_level: self.config.isolation_level.to_i8(),
                topics,
                timeout_ms: None,
            };

            // Get a connection to this broker by leader ID.
            // On any broker-level failure, mark all its partitions as Err.
            let broker_info = match self.metadata.broker(leader_id) {
                Some(b) => b,
                None => {
                    warn!("Broker {} not found in metadata, skipping", leader_id);
                    let err = KrafkaError::invalid_state(format!(
                        "broker {leader_id} not found in metadata"
                    ));
                    for (topic, partition) in leader_partitions {
                        result.insert((topic.clone(), *partition), Err(err.clone()));
                    }
                    continue;
                }
            };
            let conn = match self
                .pool
                .get_connection_by_id(leader_id, broker_info.address())
                .await
            {
                Ok(c) => c,
                Err(e) => {
                    warn!("Failed to connect to broker {}: {}, skipping", leader_id, e);
                    for (topic, partition) in leader_partitions {
                        result.insert((topic.clone(), *partition), Err(e.clone()));
                    }
                    continue;
                }
            };

            // Negotiate ListOffsets version — require v1+ (MIN).
            let list_version = match conn
                .negotiate_api_version(
                    ApiKey::ListOffsets,
                    versions::LIST_OFFSETS_MAX,
                    versions::LIST_OFFSETS_MIN,
                )
                .await
            {
                Some(v) => v,
                None => {
                    let err = KrafkaError::protocol(format!(
                        "no mutually supported ListOffsets API version for broker {leader_id}"
                    ));
                    warn!("{err}");
                    for (topic, partition) in leader_partitions {
                        result.insert((topic.clone(), *partition), Err(err.clone()));
                    }
                    continue;
                }
            };

            let response = match conn
                .send_request(ApiKey::ListOffsets, list_version, |buf| {
                    request.encode_versioned(list_version, buf)
                })
                .await
            {
                Ok(r) => r,
                Err(e) => {
                    warn!(
                        "ListOffsets v{} request failed for broker {}: {}, skipping",
                        list_version, leader_id, e
                    );
                    for (topic, partition) in leader_partitions {
                        result.insert((topic.clone(), *partition), Err(e.clone()));
                    }
                    continue;
                }
            };

            let mut buf = response;
            let list_response = match ListOffsetsResponse::decode_versioned(list_version, &mut buf)
            {
                Ok(r) => r,
                Err(e) => {
                    warn!(
                        "Failed to decode ListOffsets v{} response from broker {}: {}, skipping",
                        list_version, leader_id, e
                    );
                    let err = KrafkaError::protocol(format!(
                        "failed to decode ListOffsets response from broker {leader_id}: {e}"
                    ));
                    for (topic, partition) in leader_partitions {
                        result.insert((topic.clone(), *partition), Err(err.clone()));
                    }
                    continue;
                }
            };

            apply_list_offsets_response(&list_response, &mut result);
        }

        result
    }

    /// Poll for new records.
    ///
    /// This is the main method for consuming messages.
    ///
    /// # Example
    ///
    /// ```rust,no_run
    /// # use krafka::consumer::Consumer;
    /// # async fn example() -> Result<(), krafka::error::KrafkaError> {
    /// let consumer = Consumer::builder()
    ///     .bootstrap_servers("localhost:9092")
    ///     .group_id("my-group")
    ///     .build()
    ///     .await?;
    ///
    /// consumer.subscribe(&["my-topic"]).await?;
    ///
    /// loop {
    ///     let records = consumer.poll(std::time::Duration::from_secs(1)).await?;
    ///     for record in records {
    ///         println!("Received: {:?}", record);
    ///     }
    /// }
    /// # }
    /// ```
    pub async fn poll(&self, timeout: Duration) -> Result<Vec<ConsumerRecord>> {
        if self.closed.load(std::sync::atomic::Ordering::SeqCst) {
            return Err(KrafkaError::invalid_state("consumer is closed"));
        }

        let _poll_timer = self.metrics.poll_latency.start();
        self.metrics.polls.inc();

        // Auto-commit timer: commit if interval has elapsed
        if self.config.enable_auto_commit && self.group_coordinator.is_some() {
            let should_commit = {
                let last = self.last_auto_commit.lock();
                last.elapsed() >= self.config.auto_commit_interval
            };
            if should_commit {
                match self.commit().await {
                    Ok(()) => {
                        *self.last_auto_commit.lock() = Instant::now();
                    }
                    Err(e) => {
                        warn!("Auto-commit failed: {}", e);
                    }
                }
            }
        }

        // Handle group rebalance if needed
        if self.handle_group_rebalance().await? {
            return Ok(vec![]);
        }

        let assignments = self.assignments.read().await;
        if assignments.is_empty() {
            self.metrics.empty_polls.inc();
            return Ok(Vec::new());
        }

        // Buffer cap: skip fetching when the recv() buffer has accumulated
        // too many unconsumed records.  Auto-commit and rebalance handling
        // above still run so the consumer remains healthy in the group.
        if self.config.max_buffered_records > 0 {
            let buffered = self.recv_buffer.lock().len();
            if buffered >= self.config.max_buffered_records as usize {
                debug!(
                    buffered = buffered,
                    max = self.config.max_buffered_records,
                    "Buffer cap reached, skipping fetch"
                );
                self.metrics.empty_polls.inc();
                return Ok(Vec::new());
            }
        }

        // Retry offset resolution for partitions that are missing tracked offsets.
        // This fulfils the "will retry on next poll" contract when initial offset
        // resolution fails (e.g., due to a transient ListOffsets error or a
        // rejoin that left some partitions without offsets).
        // Exponential backoff prevents retry storms under sustained failures.
        {
            let now = Instant::now();
            let missing: Vec<(String, PartitionId)> = {
                let offsets = self.offsets.read().await;
                let partition_state = self.partition_state.read().await;
                assignments
                    .iter()
                    .flat_map(|(topic, partitions)| {
                        partitions.iter().filter_map(|&p| {
                            let key = (topic.clone(), p);

                            if offsets.contains_key(&key) {
                                return None;
                            }

                            // Only include if backoff period has elapsed
                            match partition_state
                                .get(&key)
                                .and_then(|s| s.offset_retry_backoff)
                            {
                                None => Some(key),
                                Some((next_retry, _)) if now >= next_retry => Some(key),
                                _ => None,
                            }
                        })
                    })
                    .collect()
            };

            if !missing.is_empty() {
                debug!(
                    "Retrying offset resolution for {} partition(s) without tracked offsets",
                    missing.len()
                );
                let mut reset_partitions: HashMap<String, Vec<PartitionId>> = HashMap::new();
                for (topic, partition) in &missing {
                    reset_partitions
                        .entry(topic.clone())
                        .or_default()
                        .push(*partition);
                }

                // Use group coordinator path if available, otherwise direct path
                if let Some(ref coordinator) = self.group_coordinator {
                    if let Some(timestamp) = self.config.auto_offset_reset.to_offset() {
                        match coordinator.list_offsets(&reset_partitions, timestamp).await {
                            Ok(resolved) => {
                                let mut offsets = self.offsets.write().await;
                                for (key, offset) in &resolved {
                                    offsets.insert(key.clone(), *offset);
                                }
                                drop(offsets);

                                // Fallback for partitions the coordinator path
                                // silently dropped (partition-level errors).
                                for (topic, partition) in &missing {
                                    if !resolved.contains_key(&(topic.clone(), *partition)) {
                                        debug!(
                                            "Poll retry: falling back to direct ListOffsets for {}-{}",
                                            topic, partition
                                        );
                                        if let Ok(offset) = self
                                            .resolve_list_offset(topic, *partition, timestamp)
                                            .await
                                        {
                                            let mut offsets = self.offsets.write().await;
                                            offsets.insert((topic.clone(), *partition), offset);
                                        }
                                    }
                                }
                            }
                            Err(e) => {
                                warn!("Offset resolution retry via coordinator failed: {}", e);
                                // Fall back to direct path for all missing partitions
                                for (topic, partition) in &missing {
                                    if let Ok(offset) =
                                        self.resolve_list_offset(topic, *partition, timestamp).await
                                    {
                                        let mut offsets = self.offsets.write().await;
                                        offsets.insert((topic.clone(), *partition), offset);
                                    }
                                }
                            }
                        }
                    }
                } else if let Err(e) = self.apply_auto_offset_reset(&reset_partitions).await {
                    warn!("Auto-offset-reset failed for missing partitions: {e}");
                }

                // Recompute lag after resolving offsets for missing partitions
                self.recompute_lag_metrics().await;

                // Apply exponential backoff for partitions that are still
                // unresolved after the retry attempt. Clear backoff for
                // partitions that were successfully resolved.
                {
                    let offsets = self.offsets.read().await;
                    let mut partition_state = self.partition_state.write().await;
                    for (topic, partition) in &missing {
                        let key = (topic.clone(), *partition);
                        if offsets.contains_key(&key) {
                            // Successfully resolved — clear backoff.
                            if let Some(state) = partition_state.get_mut(&key) {
                                state.offset_retry_backoff = None;
                            }
                        } else {
                            // Still unresolved — compute next backoff interval.
                            // Start at 100ms, double each time, cap at 30s.
                            let base = Duration::from_millis(100);
                            let max = Duration::from_secs(30);
                            let entry = partition_state.entry(key).or_default();
                            let prev_wait = entry
                                .offset_retry_backoff
                                .map(|(_, d)| d)
                                .unwrap_or(Duration::ZERO);
                            let next_wait = (prev_wait * 2).max(base).min(max);
                            entry.offset_retry_backoff =
                                Some((Instant::now() + next_wait, next_wait));
                        }
                    }
                }
            }
        }

        let paused = self.paused.read().await;

        // Collect non-paused partition keys (one topic clone per partition)
        // and resolve leaders so the pure routing helper doesn't need async
        // metadata access.
        let mut non_paused_keys: Vec<(String, PartitionId)> = Vec::new();
        let mut leaders: HashMap<(String, PartitionId), crate::BrokerId> = HashMap::new();
        for (topic, partitions) in assignments.iter() {
            for &partition in partitions {
                let key = (topic.clone(), partition);
                if paused.contains(&key) {
                    continue;
                }
                if let Some(leader_id) = self.metadata.leader(topic, partition) {
                    leaders.insert(key.clone(), leader_id);
                }
                non_paused_keys.push(key);
            }
        }

        let now = Instant::now();
        let partition_state_read = self.partition_state.read().await;

        let plan = build_fetch_routing_plan(non_paused_keys, &partition_state_read, &leaders, now);

        // Release read lock before potentially acquiring write lock
        drop(partition_state_read);

        // Warn only for partitions that are truly skipped (no leader AND no
        // valid preferred replica). This avoids log spam during transient
        // metadata gaps when a preferred replica is still available.
        for (topic, partition) in &plan.skipped {
            warn!(
                "No leader or preferred replica for {topic}-{partition}, skipping in batch fetch"
            );
        }

        // Clear expired preferred-replica entries so they don't accumulate.
        // Only the `preferred_replica` field is cleared — other per-partition
        // caches (high watermark, log start offset, retry backoff) are kept.
        if !plan.expired_preferred.is_empty() {
            let mut partition_state = self.partition_state.write().await;
            for key in &plan.expired_preferred {
                if let Some(state) = partition_state.get_mut(key) {
                    state.preferred_replica = None;
                }
            }
        }

        drop(paused);
        drop(assignments);

        let mut all_records = Vec::new();
        let mut all_offset_updates: Vec<((String, PartitionId), Offset)> = Vec::new();
        let mut all_hw_updates: Vec<((String, PartitionId), Offset)> = Vec::new();

        // Fetch from each broker (one request per broker, containing all its partitions)
        for (broker_id, topic_partitions) in plan.partitions_by_broker {
            match self
                .batch_fetch_from_broker(broker_id, &topic_partitions, timeout)
                .await
            {
                Ok((records, offset_updates, hw_updates)) => {
                    all_records.extend(records);
                    all_offset_updates.extend(offset_updates);
                    all_hw_updates.extend(hw_updates);
                }
                Err(e) => {
                    self.metrics.record_error();
                    warn!("Batch fetch from broker {} failed: {}", broker_id, e);
                    // Clear preferred replica mappings for all partitions that
                    // were being fetched from this broker.  If the broker was
                    // actually the leader the entries won't exist (no-op), but
                    // if it was a preferred replica this avoids routing to a
                    // dead broker for up to metadata_max_age.
                    let mut partition_state = self.partition_state.write().await;
                    for tp in &topic_partitions {
                        if let Some(state) = partition_state.get_mut(tp) {
                            state.preferred_replica = None;
                        }
                    }
                }
            }
        }

        // Enforce max_poll_records
        // Negative values are treated as unlimited (no truncation)
        // Only advance offsets for records actually delivered.
        // When truncating, recompute offset updates from delivered records only.
        if self.config.max_poll_records > 0 {
            let max = self.config.max_poll_records as usize;
            if all_records.len() > max {
                all_records.truncate(max);
                // Recompute offset updates from the truncated set: for each
                // (topic, partition), the new offset is max(record.offset) + 1
                // only for records that survived truncation.
                let mut delivered_offsets: HashMap<(String, PartitionId), Offset> = HashMap::new();
                for r in &all_records {
                    let key = (r.topic.clone(), r.partition);
                    let entry = delivered_offsets.entry(key).or_insert(r.offset);
                    if r.offset > *entry {
                        *entry = r.offset;
                    }
                }
                all_offset_updates = delivered_offsets
                    .into_iter()
                    .map(|(key, offset)| (key, offset.saturating_add(1)))
                    .collect();
            }
        }

        // Commit the offset updates (deferred from batch_fetch_from_broker until after max_poll_records handling)
        let offsets_changed = !all_offset_updates.is_empty();
        if offsets_changed {
            let mut offsets = self.offsets.write().await;
            for (key, new_offset) in all_offset_updates {
                offsets.insert(key, new_offset);
            }
        }

        // Update high watermarks
        let hw_changed = !all_hw_updates.is_empty();
        if hw_changed {
            let mut partition_state = self.partition_state.write().await;
            for (key, watermark) in all_hw_updates {
                partition_state.entry(key).or_default().high_watermark = Some(watermark);
            }
        }

        // Recompute lag metrics whenever offsets or watermarks changed
        if offsets_changed || hw_changed {
            self.recompute_lag_metrics().await;
        }

        // Record metrics
        if all_records.is_empty() {
            self.metrics.empty_polls.inc();
        } else {
            let bytes: u64 = all_records
                .iter()
                .map(|r| r.value.as_ref().map(|v| v.len() as u64).unwrap_or(0))
                .sum();
            self.metrics.record_receive(all_records.len() as u64, bytes);
        }

        // Invoke consumer interceptor after fetching records
        if !all_records.is_empty() {
            crate::interceptor::safe_on_consume(&*self.interceptor, &all_records);
        }

        Ok(all_records)
    }

    /// Batch fetch from a single broker for multiple topic-partitions.
    ///
    /// This is more efficient than individual fetches because it sends a single
    /// network request for all partitions led by the same broker.
    async fn batch_fetch_from_broker(
        &self,
        broker_id: crate::BrokerId,
        topic_partitions: &[(String, PartitionId)],
        timeout: Duration,
    ) -> Result<(
        Vec<ConsumerRecord>,
        Vec<((String, PartitionId), Offset)>,
        Vec<((String, PartitionId), Offset)>,
    )> {
        if topic_partitions.is_empty() {
            return Ok((Vec::new(), Vec::new(), Vec::new()));
        }

        self.metrics.record_fetch();
        let _fetch_timer = self.metrics.fetch_latency.start();

        // Get connection to this broker
        let broker = self
            .metadata
            .broker(broker_id)
            .ok_or_else(|| KrafkaError::invalid_state(format!("broker {} not found", broker_id)))?;
        let conn = self
            .pool
            .get_connection_by_id(broker_id, broker.address())
            .await?;

        // Group by topic for the request structure
        let mut topics_map: HashMap<String, Vec<PartitionId>> = HashMap::new();
        for (topic, partition) in topic_partitions {
            topics_map
                .entry(topic.clone())
                .or_default()
                .push(*partition);
        }

        // Build fetch request with all topic-partitions.
        // Acquire the offsets read lock once for the entire build instead of
        // per-partition to reduce lock acquire/release overhead.
        let offsets_snapshot = self.offsets.read().await;
        let mut fetch_topics = Vec::with_capacity(topics_map.len());
        for (topic, partitions) in &topics_map {
            let mut fetch_partitions = Vec::with_capacity(partitions.len());
            for &partition in partitions {
                // Skip partitions with no tracked offset rather than
                // defaulting to 0, which defeats the auto_offset_reset fix.
                let offset = match offsets_snapshot.get(&(topic.clone(), partition)).copied() {
                    Some(o) => o,
                    None => {
                        warn!(
                            "No offset for {}-{}, skipping fetch (will retry offset resolution)",
                            topic, partition
                        );
                        continue;
                    }
                };
                // Get leader epoch from metadata for fencing stale reads
                let leader_epoch = self.metadata.leader_epoch(topic, partition).unwrap_or(-1);
                fetch_partitions.push(FetchPartitionRequest {
                    partition,
                    current_leader_epoch: leader_epoch,
                    fetch_offset: offset,
                    last_fetched_epoch: -1,
                    log_start_offset: -1,
                    partition_max_bytes: self.config.max_partition_fetch_bytes,
                    replica_directory_id: None,
                    high_watermark: None,
                });
            }
            fetch_topics.push(FetchTopicRequest {
                topic: topic.clone(),
                topic_id: None,
                partitions: fetch_partitions,
            });
        }
        // Drop the read lock before the network call.
        drop(offsets_snapshot);

        // Negotiate fetch API version — prefer v11 (KIP-392 closest-replica
        // fetching), fall back through v7 (sessions) to v4.
        // We implement encode/decode for v4, v7-v10, and v11.
        // v5/v6 are unsupported (different request wire format).
        // Prefer the highest version we implement:
        //   v11 — rack_id for closest-replica routing (KIP-392)
        //   v9/v10 — current_leader_epoch for leader fencing (KIP-320;
        //            v10 shares the same request wire format as v9)
        // When client_rack is not set, cap at v10 (highest version without
        // rack_id) so we still get epoch fencing without sending an
        // unnecessary rack_id.
        let preferred_version = if self.config.client_rack.is_some() {
            11
        } else {
            10
        };
        let fetch_version = conn
            .negotiate_api_version(ApiKey::Fetch, preferred_version, 7)
            .await
            .unwrap_or_else(|| {
                debug!(
                    "No mutually supported Fetch v7+ for broker {broker_id}, falling back to v4"
                );
                4
            });

        // Build the fetch request. For v7, compute an incremental session diff
        // from fetch_topics without cloning the full topic list into the base request.
        let (session_id, session_epoch, request_topics, forgotten_topics) = if fetch_version >= 7 {
            let mut sessions = self.fetch_sessions.lock();
            let session = sessions.get_or_create(broker_id);
            let session_req = session.build_request(&fetch_topics);
            if session_req.is_full_fetch {
                debug!(
                    "Fetch broker {}: full fetch (session_id={}, epoch={})",
                    broker_id, session_req.session_id, session_req.session_epoch
                );
            } else {
                debug!(
                    "Fetch broker {}: incremental (session_id={}, epoch={}, changed={}, forgotten={})",
                    broker_id,
                    session_req.session_id,
                    session_req.session_epoch,
                    session_req.topics.len(),
                    session_req.forgotten_topics.len()
                );
            }
            (
                session_req.session_id,
                session_req.session_epoch,
                session_req.topics,
                session_req.forgotten_topics,
            )
        } else {
            // v4: move fetch_topics into the request; update_from_response
            // is only called for v7+ so fetch_topics is not needed later.
            (0, -1, std::mem::take(&mut fetch_topics), Vec::new())
        };

        let request = FetchRequest {
            replica_id: -1, // Consumer
            max_wait_ms: crate::util::duration_to_millis_i32(timeout),
            min_bytes: self.config.fetch_min_bytes,
            max_bytes: self.config.fetch_max_bytes,
            isolation_level: self.config.isolation_level.to_i8(),
            session_id,
            session_epoch,
            topics: request_topics,
            forgotten_topics,
            rack_id: self.config.client_rack.clone().unwrap_or_default(),
        };

        // Send request with negotiated version.
        // For v7+ sessions, reset session on any send/decode failure so the
        // next poll re-establishes with a full fetch instead of hitting
        // InvalidFetchSessionEpoch.
        let response = match conn
            .send_request(ApiKey::Fetch, fetch_version, |buf| {
                request.encode_versioned(fetch_version, buf)
            })
            .await
        {
            Ok(r) => r,
            Err(e) => {
                if fetch_version >= 7 {
                    let mut sessions = self.fetch_sessions.lock();
                    sessions.reset_broker(broker_id);
                }
                return Err(e);
            }
        };

        // Decode response with matching version
        let mut buf = response;
        let fetch_response = match FetchResponse::decode_versioned(fetch_version, &mut buf) {
            Ok(r) => r,
            Err(e) => {
                if fetch_version >= 7 {
                    let mut sessions = self.fetch_sessions.lock();
                    sessions.reset_broker(broker_id);
                }
                return Err(e);
            }
        };

        // KIP-219: honour broker-reported throttle time.
        conn.notify_throttle(fetch_response.throttle_time_ms);

        // Handle top-level session errors (v7+)
        if fetch_version >= 7 {
            if fetch_response.error_code == crate::error::ErrorCode::FetchSessionIdNotFound
                || fetch_response.error_code == crate::error::ErrorCode::InvalidFetchSessionEpoch
            {
                // Reset session and let the next poll do a full fetch
                warn!(
                    "Fetch session error for broker {}: {:?}, resetting session",
                    broker_id, fetch_response.error_code
                );
                let mut sessions = self.fetch_sessions.lock();
                sessions.reset_broker(broker_id);
                return Ok((Vec::new(), Vec::new(), Vec::new()));
            }

            // Update session state from response
            let mut sessions = self.fetch_sessions.lock();
            let session = sessions.get_or_create(broker_id);
            session.update_from_response(fetch_response.session_id, &fetch_topics);
        }

        // Process records
        let mut records = Vec::new();
        let mut offset_updates: Vec<((String, PartitionId), Offset)> = Vec::new();
        let mut hw_updates: Vec<((String, PartitionId), Offset)> = Vec::new();
        let mut lso_updates: Vec<((String, PartitionId), Offset)> = Vec::new();

        // Preferred replica updates (KIP-392): Some(id) to set, None to clear.
        // Collected during the loop, applied in a single write lock afterwards.
        let mut pref_updates: Vec<((String, PartitionId), Option<crate::BrokerId>)> = Vec::new();

        for topic_response in fetch_response.responses {
            let topic_name = &topic_response.topic;
            for partition_response in topic_response.partitions {
                let partition = partition_response.partition;
                let key = (topic_name.clone(), partition);

                // Capture high watermark regardless of error/empty response.
                // The broker always returns a valid high_watermark even when
                // there are no records to deliver.
                if partition_response.high_watermark >= 0 {
                    hw_updates.push((key.clone(), partition_response.high_watermark));
                }

                // Cache log_start_offset (earliest available offset) when
                // present. Returned in Fetch v5+; allows `cached_beginning_offset`
                // to serve beginning offsets from cache without a network round-trip.
                if partition_response.log_start_offset >= 0 {
                    lso_updates.push((key.clone(), partition_response.log_start_offset));
                }

                // Track preferred read replica (KIP-392, v11+ only).
                // For v7-v10, preferred_read_replica is our fabricated default
                // (-1) and must not clear valid mappings from earlier v11 responses.
                if fetch_version >= 11 {
                    if partition_response.preferred_read_replica >= 0 {
                        pref_updates
                            .push((key.clone(), Some(partition_response.preferred_read_replica)));
                    } else {
                        pref_updates.push((key.clone(), None));
                    }
                }

                if !partition_response.error_code.is_ok() {
                    // When fetching from a preferred replica and the broker
                    // returns an error, clear the preferred replica so the
                    // next poll falls back to the partition leader.  We also
                    // clear when leader metadata is unavailable (None) to
                    // avoid getting stuck routing to a failing replica until
                    // expiry.  This is not gated on fetch_version >= 11
                    // because a stale preferred mapping from an earlier v11
                    // response can still route fetches to this broker even
                    // when the negotiated version is lower (e.g. rolling
                    // upgrade).
                    let is_leader = self
                        .metadata
                        .leader(topic_name, partition)
                        .is_some_and(|leader_id| leader_id == broker_id);
                    if !is_leader {
                        debug!(
                            "Error from non-leader broker {} for {}-{}: {:?}, clearing preferred replica",
                            broker_id, topic_name, partition, partition_response.error_code
                        );
                        pref_updates.push((key.clone(), None));
                    }
                    // Handle leader epoch errors by validating via OffsetForLeaderEpoch
                    if partition_response.error_code == crate::error::ErrorCode::FencedLeaderEpoch
                        || partition_response.error_code
                            == crate::error::ErrorCode::UnknownLeaderEpoch
                    {
                        warn!(
                            "Leader epoch error for {}-{}: {:?}, validating offset via OffsetForLeaderEpoch",
                            topic_name, partition, partition_response.error_code
                        );
                        // Trigger metadata refresh and reset offset if truncation detected.
                        // On validation failure (e.g. network error), fall back to
                        // auto_offset_reset so the consumer does not get stuck on a
                        // potentially truncated partition.
                        if let Err(e) = self
                            .validate_offset_for_leader_epoch(topic_name, partition)
                            .await
                        {
                            warn!(
                                "OffsetForLeaderEpoch validation failed for {}-{}: {}, \
                                 falling back to auto_offset_reset",
                                topic_name, partition, e
                            );
                            self.handle_offset_out_of_range(topic_name, partition).await;
                        }
                    } else if partition_response.error_code
                        == crate::error::ErrorCode::OffsetOutOfRange
                    {
                        warn!(
                            "OffsetOutOfRange for {}-{}, applying auto_offset_reset",
                            topic_name, partition
                        );
                        self.handle_offset_out_of_range(topic_name, partition).await;
                    } else {
                        warn!(
                            "Fetch error for {}-{}: {:?}",
                            topic_name, partition, partition_response.error_code
                        );
                    }
                    continue; // Continue with other partitions
                }

                if let Some(record_bytes) = partition_response.records {
                    let mut batch_buf = record_bytes;
                    let mut last_offset_for_partition: Option<Offset> = None;

                    // Fetch offset for this partition — used to skip records
                    // already delivered in a prior poll when Kafka returns a
                    // batch that starts before the requested offset.
                    // Read lock is acquired and dropped inline to avoid cloning
                    // the entire offsets map on every fetch pass.
                    // Missing entry means this is the first fetch; accept all
                    // records (offset 0 skips nothing since Kafka offsets ≥ 0).
                    let partition_fetch_offset = {
                        let offsets = self.offsets.read().await;
                        match offsets.get(&key).copied() {
                            Some(offset) => offset,
                            None => {
                                debug!(
                                    topic = %topic_name,
                                    partition,
                                    "No tracked offset for partition, accepting all records"
                                );
                                0
                            }
                        }
                    };

                    // Decode all fetched batches for this partition. `poll()`
                    // applies `max_poll_records` after aggregation and
                    // recomputes offsets for the returned subset, so stopping
                    // here without buffering the remaining bytes would force a
                    // re-fetch/re-decode of the dropped batches on subsequent
                    // polls.
                    while batch_buf.len() >= 12 {
                        match RecordBatch::decode_with_limit(
                            &mut batch_buf,
                            self.config.max_decompressed_size,
                        ) {
                            Ok(batch) => {
                                for record in batch.records.into_iter() {
                                    // Use offset_delta for correct offset in compacted topics
                                    // where records may have been deleted (log compaction awareness).
                                    let record_offset = batch
                                        .base_offset
                                        .saturating_add(record.offset_delta as i64);

                                    // Skip records below the fetch offset — these were
                                    // already delivered in a prior poll but are included
                                    // because Kafka returns whole batches.
                                    if record_offset < partition_fetch_offset {
                                        continue;
                                    }

                                    records.push(ConsumerRecord {
                                        topic: topic_name.clone(),
                                        partition,
                                        offset: record_offset,
                                        timestamp: batch
                                            .base_timestamp
                                            .saturating_add(record.timestamp_delta),
                                        timestamp_type: batch.attributes.timestamp_type as i8,
                                        key: record.key,
                                        value: record.value,
                                        headers: record
                                            .headers
                                            .into_iter()
                                            .map(|h| (h.key, h.value))
                                            .collect(),
                                        leader_epoch: None,
                                        delivery_count: None,
                                    });
                                    last_offset_for_partition = Some(record_offset);
                                }
                            }
                            Err(e) => {
                                debug!("Failed to decode record batch: {}", e);
                                break;
                            }
                        }
                    }

                    // Track offset update for this partition
                    if let Some(last_offset) = last_offset_for_partition {
                        offset_updates.push((key, last_offset.saturating_add(1)));
                    }
                }
            }
        }

        // NOTE: Offsets are NOT advanced here. They are advanced in poll()
        // after max_poll_records truncation to avoid silently losing records
        // whose offsets were already committed.
        // We return offset_updates and high watermarks alongside records so
        // the caller can apply them and compute lag.

        // Apply log_start_offset and preferred-replica updates in a single
        // write lock acquisition. Log-start updates reflect broker state and
        // are not affected by max_poll_records truncation. Preferred-replica
        // last-write-wins: if a partition appears multiple times (e.g. set by
        // the response then cleared by error handling), the final entry takes
        // effect.
        if !lso_updates.is_empty() || !pref_updates.is_empty() {
            let expiry = Instant::now() + self.config.metadata_max_age;
            let mut partition_state = self.partition_state.write().await;
            for (key, offset) in lso_updates {
                partition_state.entry(key).or_default().log_start_offset = Some(offset);
            }
            for (key, value) in pref_updates {
                match value {
                    // Setting a preferred replica: insert/update the entry.
                    Some(replica_id) => {
                        partition_state.entry(key).or_default().preferred_replica =
                            Some((replica_id, expiry));
                    }
                    // Clearing: only mutate an existing entry. Skipping
                    // absent entries avoids inserting empty `PartitionState`
                    // values on every fetch response that reports
                    // `preferred_read_replica = -1` (the common case), which
                    // would otherwise write-amplify this hot path.
                    None => {
                        if let Some(state) = partition_state.get_mut(&key) {
                            state.preferred_replica = None;
                        }
                    }
                }
            }
        }

        Ok((records, offset_updates, hw_updates))
    }

    /// Handle an `OffsetOutOfRange` error for a single partition by resolving
    /// a new offset via the configured `auto_offset_reset` policy.
    async fn handle_offset_out_of_range(&self, topic: &str, partition: PartitionId) {
        let Some(target) = self.config.auto_offset_reset.to_offset() else {
            return;
        };

        let key = (topic.to_string(), partition);

        let resolved = if let Some(ref gc) = self.group_coordinator {
            let mut part_map = HashMap::new();
            part_map.insert(key.0.clone(), vec![partition]);
            match gc.list_offsets(&part_map, target).await {
                Ok(offsets) => offsets.get(&key).copied(),
                Err(e) => {
                    warn!(
                        "Coordinator list_offsets failed for {}-{}: {}, falling back to direct",
                        topic, partition, e
                    );
                    None
                }
            }
        } else {
            None
        };

        // Use coordinator result, or fall back to direct ListOffsets
        let offset = match resolved {
            Some(o) => Some(o),
            None => self
                .resolve_list_offset(topic, partition, target)
                .await
                .map_err(|e| {
                    warn!("Direct list_offset failed for {topic}-{partition}: {e}");
                    e
                })
                .ok(),
        };

        if let Some(new_offset) = offset {
            self.offsets.write().await.insert(key, new_offset);
            self.recompute_lag_metrics().await;
        }
    }

    /// Validate the consumer's offset for a partition using OffsetForLeaderEpoch.
    ///
    /// When a leader epoch error occurs during fetch, this method queries the
    /// broker for the end offset of the current leader epoch. If the consumer's
    /// current offset is beyond this (indicating log truncation), the offset
    /// is reset to the truncation point.
    async fn validate_offset_for_leader_epoch(
        &self,
        topic: &str,
        partition: PartitionId,
    ) -> Result<()> {
        use crate::protocol::OffsetForLeaderEpochPartition;
        use crate::protocol::OffsetForLeaderEpochRequest;
        use crate::protocol::OffsetForLeaderEpochResponse;
        use crate::protocol::OffsetForLeaderEpochTopic;

        // Refresh metadata first to get updated leader info
        if let Err(e) = self.metadata.refresh_for_topics(Some(&[topic])).await {
            warn!(
                "Metadata refresh failed for {}: {}, using cached metadata",
                topic, e
            );
        }

        let leader_epoch = self.metadata.leader_epoch(topic, partition).unwrap_or(-1);

        if leader_epoch < 0 {
            return Ok(());
        }

        let leader_id = self.metadata.leader(topic, partition).ok_or_else(|| {
            KrafkaError::invalid_state(format!("no leader for {topic}-{partition}"))
        })?;

        let broker = self
            .metadata
            .broker(leader_id)
            .ok_or_else(|| KrafkaError::invalid_state(format!("broker {} not found", leader_id)))?;

        let conn = self
            .pool
            .get_connection_by_id(leader_id, broker.address())
            .await?;

        let request = OffsetForLeaderEpochRequest {
            replica_id: -1, // consumer
            topics: vec![OffsetForLeaderEpochTopic {
                topic: topic.to_string(),
                partitions: vec![OffsetForLeaderEpochPartition {
                    partition,
                    current_leader_epoch: leader_epoch,
                    leader_epoch,
                }],
            }],
        };

        let version = conn
            .negotiate_api_version(
                ApiKey::OffsetForLeaderEpoch,
                versions::OFFSET_FOR_LEADER_EPOCH_MAX,
                versions::OFFSET_FOR_LEADER_EPOCH_MIN,
            )
            .await
            .ok_or_else(|| {
                KrafkaError::protocol("no mutually supported OffsetForLeaderEpoch API version")
            })?;

        let response_bytes = conn
            .send_request(ApiKey::OffsetForLeaderEpoch, version, |buf| {
                request.encode_versioned(version, buf)
            })
            .await?;

        let mut buf = response_bytes;
        let response = OffsetForLeaderEpochResponse::decode_versioned(version, &mut buf)?;

        let key = (topic.to_string(), partition);
        let mut offset_changed = false;

        for topic_result in response.topics {
            for partition_result in topic_result.partitions {
                if partition_result.partition != partition {
                    continue;
                }
                if partition_result.error_code.is_ok() && partition_result.end_offset >= 0 {
                    let current_offset = {
                        let offsets = self.offsets.read().await;
                        match offsets.get(&key).copied() {
                            Some(offset) => offset,
                            None => {
                                debug!(
                                    topic = %topic,
                                    partition,
                                    "No tracked offset for partition during epoch validation"
                                );
                                0
                            }
                        }
                    };

                    if current_offset > partition_result.end_offset {
                        warn!(
                            "Log truncation detected for {}-{}: offset {} > end_offset {}, resetting",
                            topic, partition, current_offset, partition_result.end_offset
                        );
                        let mut offsets = self.offsets.write().await;
                        offsets.insert(key.clone(), partition_result.end_offset);
                        offset_changed = true;
                    }
                }
            }
        }

        if offset_changed {
            self.recompute_lag_metrics().await;
        }
        Ok(())
    }

    /// Receive the next record.
    ///
    /// This is a convenience method that returns one record at a time.
    /// Internally buffers records from `poll()` and returns them one by one,
    /// ensuring no records are lost.
    ///
    /// Returns `Ok(None)` if the consumer is closed, or `Err` on failure.
    pub async fn recv(&self) -> Result<Option<ConsumerRecord>> {
        loop {
            // Return buffered records first
            {
                let mut buffer = self.recv_buffer.lock();
                if let Some(record) = buffer.pop_front() {
                    self.metrics.buffered_records.set(buffer.len() as u64);
                    return Ok(Some(record));
                }
            }

            if self.closed.load(std::sync::atomic::Ordering::SeqCst) {
                return Ok(None);
            }

            match self.poll(Duration::from_secs(1)).await {
                Ok(records) if !records.is_empty() => {
                    let mut iter = records.into_iter();
                    // Infallible: `!records.is_empty()` guard above guarantees ≥1 element.
                    let Some(first) = iter.next() else {
                        unreachable!("non-empty ConsumerRecords yields at least one element");
                    };
                    // Buffer any remaining records for subsequent recv() calls
                    if iter.len() > 0 {
                        let mut buffer = self.recv_buffer.lock();
                        buffer.extend(iter);
                        self.metrics.buffered_records.set(buffer.len() as u64);
                    }
                    return Ok(Some(first));
                }
                Ok(_) => continue,
                Err(_) if self.closed.load(std::sync::atomic::Ordering::SeqCst) => {
                    return Ok(None);
                }
                Err(e) => {
                    return Err(e);
                }
            }
        }
    }

    /// Create an async [`Stream`](futures_core::Stream) of records.
    ///
    /// Each element is a `Result<ConsumerRecord>`. The stream terminates
    /// when the consumer is closed (returns `None`). Broker and network
    /// errors are propagated as `Some(Err(...))`.
    ///
    /// Internally delegates to [`recv()`](Self::recv), which handles
    /// polling, buffering, auto-commit, rebalancing, and shutdown.
    ///
    /// # Example
    ///
    /// ```ignore
    /// use tokio_stream::StreamExt;
    ///
    /// let mut stream = consumer.stream();
    /// while let Some(result) = stream.next().await {
    ///     let record = result?;
    ///     println!("{}: {}", record.topic, record.offset);
    /// }
    /// ```
    pub fn stream(&self) -> ConsumerStream<'_> {
        ConsumerStream::new(self)
    }

    /// Commit offsets for all consumed records.
    ///
    /// This stores the current offsets for assigned partitions only.
    /// When using a consumer group, this sends an OffsetCommit request to the group coordinator.
    /// Offsets for revoked partitions are excluded to avoid overwriting the new owner's progress.
    pub async fn commit(&self) -> Result<()> {
        let offsets = self.offsets.read().await;
        if offsets.is_empty() {
            debug!("No offsets to commit");
            return Ok(());
        }

        self.metrics.commits.inc();

        // Build the set of currently assigned partitions, so we don't commit
        // stale offsets for revoked partitions.
        let assignments = self.assignments.read().await;
        let assigned_set: HashSet<(String, PartitionId)> = assignments
            .iter()
            .flat_map(|(topic, parts)| parts.iter().map(move |&p| (topic.clone(), p)))
            .collect();

        // If we have a group coordinator, send actual OffsetCommit request
        if let Some(ref coordinator) = self.group_coordinator {
            // Convert offsets to the format expected by coordinator,
            // filtering to only currently assigned partitions.
            // Use explicit group check instead of assigned_set.is_empty()
            // to avoid committing stale offsets when assignments are temporarily empty.
            let commit_offsets: HashMap<(String, PartitionId), (i64, Option<String>)> = offsets
                .iter()
                .filter(|((topic, partition), _)| {
                    assigned_set.contains(&(topic.clone(), *partition))
                })
                .map(|((topic, partition), offset)| ((topic.clone(), *partition), (*offset, None)))
                .collect();

            if commit_offsets.is_empty() {
                debug!("No assigned partition offsets to commit");
                return Ok(());
            }

            // Only pass actually-committed offsets to interceptor
            let committed_offsets: HashMap<(String, PartitionId), Offset> = commit_offsets
                .iter()
                .map(|((topic, partition), (offset, _))| ((topic.clone(), *partition), *offset))
                .collect();

            match coordinator.commit_offsets(&commit_offsets).await {
                Ok(()) => {
                    crate::interceptor::safe_on_commit(
                        &*self.interceptor,
                        &committed_offsets,
                        None,
                    );
                }
                Err(e) if e.is_retriable() => {
                    // Retry retriable errors up to 2 more times with short backoff.
                    let mut last_err = e;
                    let backoffs = [Duration::from_millis(100), Duration::from_millis(250)];
                    for delay in &backoffs {
                        debug!(
                            "Commit failed with retriable error, retrying in {:?}: {last_err}",
                            delay
                        );
                        tokio::time::sleep(*delay).await;
                        match coordinator.commit_offsets(&commit_offsets).await {
                            Ok(()) => {
                                crate::interceptor::safe_on_commit(
                                    &*self.interceptor,
                                    &committed_offsets,
                                    None,
                                );
                                return Ok(());
                            }
                            Err(e) if e.is_retriable() => {
                                last_err = e;
                            }
                            Err(e) => {
                                crate::interceptor::safe_on_commit(
                                    &*self.interceptor,
                                    &committed_offsets,
                                    Some(&e),
                                );
                                return Err(e);
                            }
                        }
                    }
                    // All retries exhausted.
                    crate::interceptor::safe_on_commit(
                        &*self.interceptor,
                        &committed_offsets,
                        Some(&last_err),
                    );
                    return Err(last_err);
                }
                Err(e) => {
                    crate::interceptor::safe_on_commit(
                        &*self.interceptor,
                        &committed_offsets,
                        Some(&e),
                    );
                    return Err(e);
                }
            }
        } else {
            // Log offsets for non-group consumers
            for ((topic, partition), offset) in offsets.iter() {
                debug!("Committed offset for {}-{}: {}", topic, partition, offset);
            }
            info!("Committed {} partition offsets (local only)", offsets.len());
        }

        Ok(())
    }

    /// Commit offsets synchronously.
    pub async fn commit_sync(&self) -> Result<()> {
        self.commit().await
    }

    /// Commit offsets asynchronously.
    ///
    /// Spawns the offset commit as a background task. Errors are logged
    /// but not propagated to the caller. Use `commit_sync()` if you need
    /// to handle commit errors.
    pub fn commit_async(&self) {
        // Snapshot offsets without blocking (try_read avoids async in non-async fn)
        // Also filter to only assigned partitions
        let assigned_set: HashSet<(String, PartitionId)> = match self.assignments.try_read() {
            Ok(guard) => guard
                .iter()
                .flat_map(|(topic, parts)| parts.iter().map(move |&p| (topic.clone(), p)))
                .collect(),
            Err(_) => HashSet::new(), // If lock contention, include all (safe fallback)
        };

        let offsets_snapshot: HashMap<(String, PartitionId), (i64, Option<String>)> = match self
            .offsets
            .try_read()
        {
            Ok(guard) => {
                if guard.is_empty() {
                    return;
                }
                guard
                    .iter()
                    .filter(|((topic, partition), _)| {
                        // Only commit offsets for assigned partitions
                        // when using group coordination. Manual assign mode commits all.
                        self.group_coordinator.is_none()
                            || assigned_set.contains(&(topic.clone(), *partition))
                    })
                    .map(|((topic, partition), offset)| {
                        ((topic.clone(), *partition), (*offset, None))
                    })
                    .collect()
            }
            Err(_) => {
                // Log warning on contention so dropped commits are visible
                tracing::warn!("commit_async: offset lock contention, skipping this commit cycle");
                return;
            }
        };

        let coordinator = self.group_coordinator.clone();
        let group_id = self.config.group_id.clone();
        tokio::spawn(async move {
            if let Some(ref coordinator) = coordinator {
                if let Err(e) = coordinator.commit_offsets(&offsets_snapshot).await {
                    tracing::warn!(
                        group_id = ?group_id,
                        error = %e,
                        "Async offset commit failed"
                    );
                }
            } else {
                tracing::debug!("Async commit: no group coordinator, offsets stored locally");
            }
        });
    }

    /// Commit specific offsets with metadata.
    ///
    /// Allows committing offsets for specific topic-partitions with optional metadata.
    /// This is useful for checkpointing or storing application-specific context.
    ///
    /// # Example
    ///
    /// ```rust,no_run
    /// use std::collections::HashMap;
    /// use krafka::consumer::{Consumer, OffsetAndMetadata, TopicPartition};
    ///
    /// # async fn example() -> Result<(), krafka::error::KrafkaError> {
    /// # let consumer: Consumer = todo!();
    /// let mut offsets = HashMap::new();
    /// offsets.insert(
    ///     TopicPartition::new("my-topic", 0),
    ///     OffsetAndMetadata::with_metadata(100, "checkpoint-abc123"),
    /// );
    /// consumer.commit_with_metadata(offsets).await?;
    /// # Ok(())
    /// # }
    /// ```
    pub async fn commit_with_metadata(
        &self,
        offsets: HashMap<TopicPartition, OffsetAndMetadata>,
    ) -> Result<()> {
        if offsets.is_empty() {
            debug!("No offsets to commit");
            return Ok(());
        }

        // Filter to only assigned partitions
        let assignments = self.assignments.read().await;
        let filtered_offsets: HashMap<TopicPartition, OffsetAndMetadata> = if assignments.is_empty()
        {
            // No group membership — accept all
            offsets
        } else {
            offsets
                .into_iter()
                .filter(|(tp, _)| {
                    assignments
                        .get(&tp.topic)
                        .is_some_and(|ps| ps.contains(&tp.partition))
                })
                .collect()
        };
        drop(assignments);

        if filtered_offsets.is_empty() {
            debug!("No offsets to commit after filtering by assigned partitions");
            return Ok(());
        }

        // If we have a group coordinator, send actual OffsetCommit request
        if let Some(ref coordinator) = self.group_coordinator {
            // Convert offsets to the format expected by coordinator
            let commit_offsets: HashMap<(String, PartitionId), (i64, Option<String>)> =
                filtered_offsets
                    .iter()
                    .map(|(tp, offset_meta)| {
                        (
                            (tp.topic.clone(), tp.partition),
                            (offset_meta.offset, offset_meta.metadata.clone()),
                        )
                    })
                    .collect();

            let interceptor_offsets: HashMap<(String, PartitionId), i64> = filtered_offsets
                .iter()
                .map(|(tp, om)| ((tp.topic.clone(), tp.partition), om.offset))
                .collect();

            match coordinator.commit_offsets(&commit_offsets).await {
                Ok(()) => {
                    crate::interceptor::safe_on_commit(
                        &*self.interceptor,
                        &interceptor_offsets,
                        None,
                    );
                }
                Err(e) => {
                    crate::interceptor::safe_on_commit(
                        &*self.interceptor,
                        &interceptor_offsets,
                        Some(&e),
                    );
                    return Err(e);
                }
            }

            // Update internal offset store
            let mut internal_offsets = self.offsets.write().await;
            for (tp, offset_meta) in filtered_offsets {
                internal_offsets.insert((tp.topic, tp.partition), offset_meta.offset);
            }
        } else {
            // Log offsets being committed with metadata for non-group consumers
            for (tp, offset_meta) in &filtered_offsets {
                let metadata_str = offset_meta.metadata.as_deref().unwrap_or("<none>");
                debug!(
                    "Committed offset for {}-{}: {} (metadata: {})",
                    tp.topic, tp.partition, offset_meta.offset, metadata_str
                );
            }

            // Update internal offset store
            let count = {
                let mut internal_offsets = self.offsets.write().await;
                for (tp, offset_meta) in filtered_offsets {
                    internal_offsets.insert((tp.topic, tp.partition), offset_meta.offset);
                }
                internal_offsets.len()
            };

            info!(
                "Committed {} partition offsets with metadata (local only)",
                count
            );
        }

        self.recompute_lag_metrics().await;
        Ok(())
    }

    /// Get the current position for a partition.
    pub async fn position(&self, topic: &str, partition: PartitionId) -> Option<Offset> {
        let offsets = self.offsets.read().await;
        offsets.get(&(topic.to_string(), partition)).copied()
    }

    /// Returns a **snapshot** of the current partition assignments.
    ///
    /// The returned `HashMap` is a clone — modifying it has no effect on the
    /// consumer's internal state. Assignments may change asynchronously due
    /// to rebalances.
    pub async fn assignment(&self) -> HashMap<String, Vec<PartitionId>> {
        let assignments = self.assignments.read().await;
        assignments.clone()
    }

    /// Get all subscribed topics.
    pub async fn subscription(&self) -> HashSet<String> {
        let subscriptions = self.subscriptions.read().await;
        subscriptions.clone()
    }

    /// Get the current lag for a specific partition.
    ///
    /// Returns the difference between the high watermark (latest offset on the
    /// broker) and the consumer's current position. Returns `None` if the high
    /// watermark or position is not yet known (e.g., no fetch has completed for
    /// this partition).
    ///
    /// This uses cached high watermarks from the most recent fetch response —
    /// no additional network calls are made.
    pub async fn current_lag(&self, topic: &str, partition: PartitionId) -> Option<u64> {
        let key = (topic.to_string(), partition);
        // Acquire offsets before partition_state to match the documented
        // lock ordering: assignments → offsets → partition_state.
        let offsets = self.offsets.read().await;
        let position = offsets.get(&key).copied()?;
        let partition_state = self.partition_state.read().await;
        let watermark = partition_state.get(&key).and_then(|s| s.high_watermark)?;
        Some((watermark - position).max(0) as u64)
    }

    /// Get per-partition lag for all assigned partitions.
    ///
    /// Returns a map of `(topic, partition) → lag` for every partition where
    /// both the high watermark and current position are known. Partitions that
    /// haven't been fetched yet are omitted.
    pub async fn lag(&self) -> HashMap<(String, PartitionId), u64> {
        // Acquire offsets before partition_state to match the documented
        // lock ordering: assignments → offsets → partition_state.
        let offsets = self.offsets.read().await;
        let partition_state = self.partition_state.read().await;
        let mut result = HashMap::with_capacity(partition_state.len());
        for (key, state) in partition_state.iter() {
            if let (Some(watermark), Some(&position)) = (state.high_watermark, offsets.get(key)) {
                result.insert(key.clone(), (watermark - position).max(0) as u64);
            }
        }
        result
    }

    /// Get the cached beginning (log start) offset for a partition.
    ///
    /// Returns the earliest available offset on the broker, cached from
    /// fetch responses. Returns `None` if no fetch has completed for this
    /// partition yet. No network calls are made.
    pub async fn cached_beginning_offset(
        &self,
        topic: &str,
        partition: PartitionId,
    ) -> Option<Offset> {
        let key = (topic.to_string(), partition);
        self.partition_state
            .read()
            .await
            .get(&key)
            .and_then(|s| s.log_start_offset)
    }

    /// Get the cached end (high watermark) offset for a partition.
    ///
    /// Returns the latest offset on the broker, cached from fetch responses.
    /// Returns `None` if no fetch has completed for this partition yet.
    /// No network calls are made.
    pub async fn cached_end_offset(&self, topic: &str, partition: PartitionId) -> Option<Offset> {
        let key = (topic.to_string(), partition);
        self.partition_state
            .read()
            .await
            .get(&key)
            .and_then(|s| s.high_watermark)
    }

    /// Unsubscribe from all topics.
    ///
    /// properly notifies the rebalance listener, leaves the
    /// consumer group, clears offsets, paused set, and drains recv buffer.
    pub async fn unsubscribe(&self) {
        // Notify listener of revoked partitions before clearing
        let assignments = self.assignments.read().await;
        if !assignments.is_empty() {
            let revoked: Vec<TopicPartition> = assignments
                .iter()
                .flat_map(|(t, ps)| ps.iter().map(move |&p| TopicPartition::new(t, p)))
                .collect();
            self.safe_on_partitions_revoked(&revoked);
        }
        drop(assignments);

        // Leave consumer group
        if let Some(ref coordinator) = self.group_coordinator
            && let Err(e) = coordinator.leave_group().await
        {
            warn!("Error leaving consumer group during unsubscribe: {}", e);
        }

        self.subscriptions.write().await.clear();
        self.assignments.write().await.clear();
        self.clear_partition_state().await;
        self.metrics.assigned_partitions.set(0);

        debug!("Unsubscribed from all topics");
    }

    /// Pause consumption of specific partitions.
    ///
    /// Paused partitions will be skipped during poll() until resumed.
    pub async fn pause(&self, topic: &str, partitions: &[PartitionId]) {
        let mut paused = self.paused.write().await;
        let topic_owned = topic.to_string();
        for &partition in partitions {
            paused.insert((topic_owned.clone(), partition));
        }
        self.metrics.paused_partitions.set(paused.len() as u64);
        debug!("Paused partitions for {}: {:?}", topic, partitions);
    }

    /// Resume consumption of specific partitions.
    ///
    /// Resumes polling for previously paused partitions.
    pub async fn resume(&self, topic: &str, partitions: &[PartitionId]) {
        let mut paused = self.paused.write().await;
        let topic_key = topic.to_string();
        for &partition in partitions {
            paused.remove(&(topic_key.clone(), partition));
        }
        self.metrics.paused_partitions.set(paused.len() as u64);
        debug!("Resumed partitions for {}: {:?}", topic, partitions);
    }

    /// Get the set of paused partitions.
    pub async fn paused_partitions(&self) -> HashSet<(String, PartitionId)> {
        self.paused.read().await.clone()
    }

    /// Replace the bootstrap server list at runtime (KIP-899).
    ///
    /// The new addresses are used on the next metadata refresh that falls back
    /// to bootstrap servers. Does not close existing connections.
    ///
    /// # Errors
    ///
    /// Returns an error if `servers` is empty.
    pub fn update_seed_brokers(&self, servers: Vec<String>) -> Result<()> {
        self.metadata.update_seed_brokers(servers)
    }

    /// Force a rebootstrap: close all connections, clear the metadata cache,
    /// and fall back to bootstrap servers (KIP-899).
    pub async fn rebootstrap(&self) {
        self.metadata.rebootstrap().await;
    }

    /// Close the consumer.
    ///
    /// Commits offsets (if auto-commit is enabled), leaves the consumer group,
    /// and tears down connections. Calling `close()` more than once is a no-op.
    pub async fn close(&self) {
        if self.closed.swap(true, std::sync::atomic::Ordering::SeqCst) {
            return;
        }

        // Auto-commit on close (if enabled)
        if self.config.enable_auto_commit
            && let Err(e) = self.commit().await
        {
            warn!("Auto-commit on close failed: {}", e);
        }

        // Notify listener that partitions are being lost
        let assignments = self.assignments.read().await;
        if !assignments.is_empty() {
            let lost: Vec<TopicPartition> = assignments
                .iter()
                .flat_map(|(t, ps)| ps.iter().map(move |&p| TopicPartition::new(t, p)))
                .collect();
            self.safe_on_partitions_lost(&lost);
        }
        drop(assignments);

        // Leave consumer group if we have a group coordinator
        if let Some(ref coordinator) = self.group_coordinator
            && let Err(e) = coordinator.leave_group().await
        {
            warn!("Error leaving consumer group: {e}");
        }

        // Clear per-partition state so post-close recv() cannot return records
        // from partitions already signaled as lost via on_partitions_lost above.
        self.subscriptions.write().await.clear();
        self.assignments.write().await.clear();
        self.clear_partition_state().await;
        self.metrics.assigned_partitions.set(0);

        // Notify interceptor of shutdown
        crate::interceptor::safe_consumer_close(&*self.interceptor);

        self.pool.close_all().await;
        info!("Consumer closed");
    }

    /// Check if the consumer is closed.
    #[inline]
    pub fn is_closed(&self) -> bool {
        self.closed.load(std::sync::atomic::Ordering::SeqCst)
    }

    /// Get the group coordinator, if one is configured.
    #[inline]
    pub fn group_coordinator(&self) -> Option<&Arc<GroupCoordinator>> {
        self.group_coordinator.as_ref()
    }

    /// Get a snapshot of consumer metrics.
    #[inline]
    pub fn metrics(&self) -> &Arc<ConsumerMetrics> {
        &self.metrics
    }

    /// Get the shared connection metrics handle used by this consumer's broker pool.
    #[inline]
    pub fn connection_metrics(&self) -> Arc<ConnectionMetrics> {
        self.pool.metrics()
    }
}

impl Drop for Consumer {
    fn drop(&mut self) {
        // Warn when a consumer is dropped without an explicit `close()`.
        // Skipping `close()` means the broker will not see a `LeaveGroup`
        // and the partitions will only be reassigned after
        // `session.timeout.ms` expires, stalling the rest of the group.
        // Skip during panic unwinding.
        if !self.closed.load(std::sync::atomic::Ordering::SeqCst) && !std::thread::panicking() {
            warn!(
                "Consumer dropped without close(); group rebalance will be delayed \
                 until session.timeout.ms. Call `Consumer::close()` before drop."
            );
        }
    }
}

/// Builder for creating consumers.
#[derive(Default)]
#[must_use = "builders do nothing until .build() is called"]
pub struct ConsumerBuilder {
    config: ConsumerConfig,
    rebalance_listener: Option<Arc<dyn ConsumerRebalanceListener>>,
    interceptors: Vec<Arc<dyn crate::interceptor::ConsumerInterceptor>>,
}

impl ConsumerBuilder {
    /// Set the bootstrap servers.
    pub fn bootstrap_servers(mut self, servers: impl Into<String>) -> Self {
        self.config.bootstrap_servers = servers.into();
        self
    }

    /// Set the group ID.
    pub fn group_id(mut self, group_id: impl Into<String>) -> Self {
        self.config.group_id = Some(group_id.into());
        self
    }

    /// Set the client ID.
    pub fn client_id(mut self, client_id: impl Into<String>) -> Self {
        self.config.client_id = client_id.into();
        self
    }

    /// Set auto offset reset behavior.
    pub fn auto_offset_reset(mut self, reset: AutoOffsetReset) -> Self {
        self.config.auto_offset_reset = reset;
        self
    }

    /// Enable auto commit.
    pub fn enable_auto_commit(mut self, enable: bool) -> Self {
        self.config.enable_auto_commit = enable;
        self
    }

    /// Set auto commit interval.
    pub fn auto_commit_interval(mut self, interval: Duration) -> Self {
        self.config.auto_commit_interval = interval;
        self
    }

    /// Set fetch minimum bytes.
    pub fn fetch_min_bytes(mut self, bytes: i32) -> Self {
        self.config.fetch_min_bytes = bytes;
        self
    }

    /// Set fetch maximum bytes.
    pub fn fetch_max_bytes(mut self, bytes: i32) -> Self {
        self.config.fetch_max_bytes = bytes;
        self
    }

    /// Set max partition fetch bytes.
    pub fn max_partition_fetch_bytes(mut self, bytes: i32) -> Self {
        self.config.max_partition_fetch_bytes = bytes;
        self
    }

    /// Set maximum poll records per poll() call.
    pub fn max_poll_records(mut self, max: i32) -> Self {
        self.config.max_poll_records = max;
        self
    }

    /// Set maximum poll interval before consumer is considered dead.
    pub fn max_poll_interval(mut self, interval: Duration) -> Self {
        self.config.max_poll_interval = interval;
        self
    }

    /// Set request timeout.
    pub fn request_timeout(mut self, timeout: Duration) -> Self {
        self.config.request_timeout = timeout;
        self
    }

    /// Set session timeout for consumer groups.
    pub fn session_timeout(mut self, timeout: Duration) -> Self {
        self.config.session_timeout = timeout;
        self
    }

    /// Set heartbeat interval.
    pub fn heartbeat_interval(mut self, interval: Duration) -> Self {
        self.config.heartbeat_interval = interval;
        self
    }

    /// Set isolation level.
    pub fn isolation_level(mut self, level: IsolationLevel) -> Self {
        self.config.isolation_level = level;
        self
    }

    /// Set partition assignment strategy for consumer groups.
    pub fn partition_assignment_strategy(mut self, strategy: PartitionAssignmentStrategy) -> Self {
        self.config.partition_assignment_strategy = strategy;
        self
    }

    /// Set the static group membership instance ID (KIP-345).
    ///
    /// When configured, the consumer uses static group membership. The broker
    /// preserves partition assignments across restarts as long as the same
    /// instance ID is used, avoiding unnecessary rebalances.
    ///
    /// # Example
    ///
    /// ```rust,ignore
    /// let consumer = Consumer::builder()
    ///     .bootstrap_servers("localhost:9092")
    ///     .group_id("my-group")
    ///     .group_instance_id("instance-1")
    ///     .build()
    ///     .await?;
    /// ```
    pub fn group_instance_id(mut self, id: impl Into<String>) -> Self {
        self.config.group_instance_id = Some(id.into());
        self
    }

    /// Set metadata max age before forcing a refresh.
    pub fn metadata_max_age(mut self, age: Duration) -> Self {
        self.config.metadata_max_age = age;
        self
    }

    /// Set the client rack ID for closest-replica fetching (KIP-392).
    ///
    /// When configured, the consumer includes its rack in fetch requests.
    /// The broker may return a preferred read replica in the same rack,
    /// reducing cross-rack network traffic.
    ///
    /// # Example
    ///
    /// ```rust,ignore
    /// let consumer = Consumer::builder()
    ///     .bootstrap_servers("localhost:9092")
    ///     .group_id("my-group")
    ///     .client_rack("us-east-1a")
    ///     .build()
    ///     .await?;
    /// ```
    pub fn client_rack(mut self, rack: impl Into<String>) -> Self {
        self.config.client_rack = Some(rack.into());
        self
    }

    /// Set a rebalance listener to be notified of partition assignment changes.
    pub fn rebalance_listener(mut self, listener: Arc<dyn ConsumerRebalanceListener>) -> Self {
        self.rebalance_listener = Some(listener);
        self
    }

    /// Set authentication configuration.
    ///
    /// Enables TLS and/or SASL authentication for all broker connections.
    ///
    /// # Example
    ///
    /// ```rust,ignore
    /// use krafka::consumer::Consumer;
    /// use krafka::auth::AuthConfig;
    ///
    /// let consumer = Consumer::builder()
    ///     .bootstrap_servers("broker:9093")
    ///     .group_id("my-group")
    ///     .auth(AuthConfig::sasl_plain("user", "password")?)
    ///     .build()
    ///     .await?;
    /// ```
    pub fn auth(mut self, auth: AuthConfig) -> Self {
        self.config.auth = Some(auth);
        self
    }

    /// Set SOCKS5 proxy configuration.
    ///
    /// Routes all broker connections through the specified SOCKS5 proxy.
    #[cfg(feature = "socks5")]
    pub fn proxy(mut self, proxy: crate::network::ProxyConfig) -> Self {
        self.config.proxy = Some(proxy);
        self
    }

    /// Configure SASL/PLAIN authentication.
    pub fn sasl_plain(
        mut self,
        username: impl Into<String>,
        password: impl Into<String>,
    ) -> crate::Result<Self> {
        self.config.auth = Some(AuthConfig::sasl_plain(username, password)?);
        Ok(self)
    }

    /// Configure SASL/SCRAM-SHA-256 authentication.
    pub fn sasl_scram_sha256(
        mut self,
        username: impl Into<String>,
        password: impl Into<String>,
    ) -> Self {
        self.config.auth = Some(AuthConfig::sasl_scram_sha256(username, password));
        self
    }

    /// Configure SASL/SCRAM-SHA-512 authentication.
    pub fn sasl_scram_sha512(
        mut self,
        username: impl Into<String>,
        password: impl Into<String>,
    ) -> Self {
        self.config.auth = Some(AuthConfig::sasl_scram_sha512(username, password));
        self
    }

    /// Configure SASL/OAUTHBEARER authentication with a static token.
    ///
    /// For automatic token refresh, use [`sasl_oauthbearer_provider()`](Self::sasl_oauthbearer_provider).
    /// For SASL extensions, use `.auth(AuthConfig::sasl_oauthbearer_token(...))`.
    pub fn sasl_oauthbearer(mut self, token: impl Into<String>) -> Self {
        self.config.auth = Some(AuthConfig::sasl_oauthbearer(token));
        self
    }

    /// Configure SASL/OAUTHBEARER authentication with an async token provider.
    ///
    /// The provider is called on every new broker connection, ensuring
    /// tokens are always fresh.
    pub fn sasl_oauthbearer_provider(
        mut self,
        provider: impl crate::auth::OAuthBearerTokenProvider + 'static,
    ) -> Self {
        self.config.auth = Some(AuthConfig::sasl_oauthbearer_provider(provider));
        self
    }

    /// Set a consumer interceptor, replacing any previously added interceptors.
    ///
    /// The interceptor's `on_consume` method is called after records are fetched
    /// but before they are returned from `poll()`, and `on_commit` is called
    /// after offsets are committed.
    ///
    /// To register multiple interceptors as an ordered chain, use
    /// [`add_interceptor`](Self::add_interceptor) instead.
    pub fn interceptor(
        mut self,
        interceptor: Arc<dyn crate::interceptor::ConsumerInterceptor>,
    ) -> Self {
        self.interceptors = vec![interceptor];
        self
    }

    /// Append a consumer interceptor to the chain.
    ///
    /// Interceptors execute in the order they are added. Each interceptor is
    /// individually panic-isolated — a panic in one will not prevent the
    /// remaining interceptors from running.
    pub fn add_interceptor(
        mut self,
        interceptor: Arc<dyn crate::interceptor::ConsumerInterceptor>,
    ) -> Self {
        self.interceptors.push(interceptor);
        self
    }

    /// Set the topic cache TTL for partial metadata refreshes.
    ///
    /// During partial refreshes, cached topics that have not been refreshed
    /// within this duration are evicted to prevent unbounded cache growth.
    ///
    /// Default: 5 minutes (matching Java's `metadata.max.idle.ms`).
    pub fn metadata_topic_cache_ttl(mut self, ttl: Duration) -> Self {
        self.config.metadata_topic_cache_ttl = Some(ttl);
        self
    }

    /// Disable topic cache TTL eviction for partial metadata refreshes.
    ///
    /// By default, cached topics are evicted after 5 minutes to prevent
    /// unbounded growth on topic churn. Call this to opt out of TTL eviction;
    /// entries will then persist across partial refreshes indefinitely.
    pub fn disable_metadata_topic_cache_ttl(mut self) -> Self {
        self.config.metadata_topic_cache_ttl = None;
        self
    }

    /// Build the consumer.
    pub async fn build(self) -> Result<Consumer> {
        if self.config.bootstrap_servers.is_empty() {
            return Err(KrafkaError::config("bootstrap.servers is required"));
        }
        if self.config.enable_auto_commit && self.config.group_id.is_none() {
            tracing::warn!(
                "enable_auto_commit=true has no effect without group_id; \
                 offsets will not be persisted to the broker"
            );
        }
        if self.config.heartbeat_interval >= self.config.session_timeout {
            return Err(KrafkaError::config(format!(
                "heartbeat_interval ({:?}) must be less than session_timeout ({:?}) \
                 (recommended: session_timeout / 3)",
                self.config.heartbeat_interval, self.config.session_timeout,
            )));
        }
        if self.config.session_timeout > self.config.max_poll_interval {
            return Err(KrafkaError::config(format!(
                "session_timeout ({:?}) must be <= max_poll_interval ({:?})",
                self.config.session_timeout, self.config.max_poll_interval,
            )));
        }
        let mut consumer = Consumer::new(self.config).await?;
        if let Some(listener) = self.rebalance_listener {
            consumer.rebalance_listener = listener;
        }
        if !self.interceptors.is_empty() {
            consumer.interceptor = if self.interceptors.len() == 1 {
                // infallible: len == 1 guaranteed by the surrounding if
                let Some(single) = self.interceptors.into_iter().next() else {
                    unreachable!("len == 1 verified above");
                };
                single
            } else {
                Arc::new(crate::interceptor::ConsumerInterceptorChain::new(
                    self.interceptors,
                ))
            };
        }
        Ok(consumer)
    }
}

#[cfg(test)]
#[allow(clippy::unwrap_used, clippy::expect_used, clippy::panic)]
mod tests {
    use super::*;

    #[test]
    fn test_consumer_builder() {
        let builder = Consumer::builder()
            .bootstrap_servers("localhost:9092")
            .group_id("test-group")
            .client_id("test")
            .auto_offset_reset(AutoOffsetReset::Earliest)
            .enable_auto_commit(false)
            .max_poll_records(100)
            .max_poll_interval(Duration::from_secs(600));

        assert_eq!(builder.config.bootstrap_servers, "localhost:9092");
        assert_eq!(builder.config.group_id, Some("test-group".to_string()));
        assert_eq!(builder.config.client_id, "test");
        assert_eq!(builder.config.auto_offset_reset, AutoOffsetReset::Earliest);
        assert!(!builder.config.enable_auto_commit);
        assert_eq!(builder.config.max_poll_records, 100);
        assert_eq!(builder.config.max_poll_interval, Duration::from_secs(600));
        assert!(builder.config.auth.is_none());
    }

    #[test]
    fn test_consumer_builder_with_auth() {
        let builder = Consumer::builder()
            .bootstrap_servers("broker:9093")
            .group_id("secure-group")
            .auth(AuthConfig::sasl_plain("user", "pass").unwrap());

        let auth = builder.config.auth.as_ref().unwrap();
        assert!(auth.requires_sasl());
        assert!(!auth.requires_tls());
        assert_eq!(
            auth.security_protocol,
            crate::auth::SecurityProtocol::SaslPlaintext
        );
        assert_eq!(auth.sasl_mechanism, Some(crate::auth::SaslMechanism::Plain));
    }

    #[test]
    fn test_consumer_builder_aws_msk_iam() {
        let auth = AuthConfig::aws_msk_iam("AKID", "secret", "us-east-1");
        let builder = Consumer::builder()
            .bootstrap_servers("broker:9094")
            .group_id("msk-group")
            .auth(auth);

        let auth = builder.config.auth.as_ref().unwrap();
        assert!(auth.requires_tls());
        assert!(auth.requires_sasl());
        assert_eq!(
            auth.sasl_mechanism,
            Some(crate::auth::SaslMechanism::AwsMskIam)
        );
        assert!(auth.aws_msk_iam_credentials.is_some());
        assert!(auth.tls_config.is_some());
    }

    #[test]
    fn test_consumer_builder_no_auth_by_default() {
        let builder = Consumer::builder()
            .bootstrap_servers("broker:9092")
            .group_id("group");

        assert!(builder.config.auth.is_none());
    }

    #[test]
    fn test_consumer_builder_sasl_plain() {
        let builder = Consumer::builder()
            .bootstrap_servers("broker:9093")
            .sasl_plain("user", "pass")
            .unwrap();

        let auth = builder.config.auth.as_ref().unwrap();
        assert!(auth.requires_sasl());
        assert!(auth.plain_credentials.is_some());
    }

    #[test]
    fn test_consumer_builder_sasl_scram() {
        let builder = Consumer::builder()
            .bootstrap_servers("broker:9093")
            .sasl_scram_sha256("user", "pass");

        let auth = builder.config.auth.as_ref().unwrap();
        assert!(auth.requires_sasl());
        assert!(auth.scram_credentials.is_some());

        let builder = Consumer::builder()
            .bootstrap_servers("broker:9093")
            .sasl_scram_sha512("user", "pass");

        let auth = builder.config.auth.as_ref().unwrap();
        assert!(auth.requires_sasl());
        assert!(auth.scram_credentials.is_some());
    }

    #[tokio::test]
    async fn test_consumer_builder_no_servers() {
        let result = Consumer::builder().build().await;
        assert!(result.is_err());
    }

    #[test]
    fn test_paused_partitions_set() {
        let mut paused: HashSet<(String, PartitionId)> = HashSet::new();
        paused.insert(("topic1".to_string(), 0));
        paused.insert(("topic1".to_string(), 1));
        paused.insert(("topic2".to_string(), 0));

        assert!(paused.contains(&("topic1".to_string(), 0)));
        assert!(paused.contains(&("topic1".to_string(), 1)));
        assert!(paused.contains(&("topic2".to_string(), 0)));
        assert!(!paused.contains(&("topic2".to_string(), 1)));

        paused.remove(&("topic1".to_string(), 0));
        assert!(!paused.contains(&("topic1".to_string(), 0)));
    }

    #[test]
    fn test_topic_partition() {
        let tp = TopicPartition::new("my-topic", 3);
        assert_eq!(tp.topic(), "my-topic");
        assert_eq!(tp.partition(), 3);

        // Test Hash/Eq for HashMap use
        let mut map = HashMap::new();
        map.insert(TopicPartition::new("test", 0), 100i64);
        map.insert(TopicPartition::new("test", 1), 200i64);
        assert_eq!(map.get(&TopicPartition::new("test", 0)), Some(&100i64));
        assert_eq!(map.get(&TopicPartition::new("test", 1)), Some(&200i64));
    }

    #[test]
    fn test_offset_and_metadata() {
        let offset = OffsetAndMetadata::new(100);
        assert_eq!(offset.offset, 100);
        assert!(offset.metadata.is_none());

        let offset_with_meta = OffsetAndMetadata::with_metadata(200, "checkpoint-123");
        assert_eq!(offset_with_meta.offset, 200);
        assert_eq!(offset_with_meta.metadata.as_deref(), Some("checkpoint-123"));

        let offset_with_epoch = OffsetAndMetadata::with_epoch(300, 5);
        assert_eq!(offset_with_epoch.offset, 300);
        assert_eq!(offset_with_epoch.leader_epoch, Some(5));
    }

    #[test]
    fn test_consumer_builder_partition_assignment_strategy() {
        let builder = Consumer::builder()
            .bootstrap_servers("localhost:9092")
            .group_id("test-group")
            .partition_assignment_strategy(PartitionAssignmentStrategy::RoundRobin);

        assert_eq!(
            builder.config.partition_assignment_strategy,
            PartitionAssignmentStrategy::RoundRobin
        );
    }

    #[test]
    fn test_consumer_builder_with_rebalance_listener() {
        use std::sync::atomic::AtomicBool;
        use std::sync::atomic::Ordering;

        struct TestListener {
            assigned: AtomicBool,
        }
        impl ConsumerRebalanceListener for TestListener {
            fn on_partitions_assigned(&self, _: &[TopicPartition]) {
                self.assigned.store(true, Ordering::SeqCst);
            }
            fn on_partitions_revoked(&self, _: &[TopicPartition]) {}
        }

        let listener = Arc::new(TestListener {
            assigned: AtomicBool::new(false),
        });

        let builder = Consumer::builder()
            .bootstrap_servers("localhost:9092")
            .group_id("test-group")
            .rebalance_listener(listener.clone());

        assert!(builder.rebalance_listener.is_some());
    }

    #[test]
    fn test_partition_assignment_strategy_default() {
        let config = ConsumerConfig::default();
        assert_eq!(
            config.partition_assignment_strategy,
            PartitionAssignmentStrategy::Range
        );
    }

    #[test]
    fn test_partition_assignment_strategy_protocol_name() {
        assert_eq!(PartitionAssignmentStrategy::Range.protocol_name(), "range");
        assert_eq!(
            PartitionAssignmentStrategy::RoundRobin.protocol_name(),
            "roundrobin"
        );
        assert_eq!(
            PartitionAssignmentStrategy::CooperativeSticky.protocol_name(),
            "cooperative-sticky"
        );
    }

    #[test]
    fn test_consumer_config_defaults() {
        let config = ConsumerConfig::default();
        // Verify sensible defaults
        assert!(config.fetch_max_bytes > 0);
        assert!(config.fetch_min_bytes > 0);
        assert!(config.max_partition_fetch_bytes > 0);
    }

    #[tokio::test]
    async fn test_consumer_builder_rejects_bad_heartbeat() {
        // heartbeat_interval >= session_timeout should fail
        let result = Consumer::builder()
            .bootstrap_servers("localhost:9092")
            .group_id("test")
            .session_timeout(Duration::from_secs(5))
            .heartbeat_interval(Duration::from_secs(5))
            .build()
            .await;

        match result {
            Err(e) => assert!(e.to_string().contains("heartbeat_interval")),
            Ok(_) => panic!("expected error for heartbeat_interval >= session_timeout"),
        }
    }

    #[tokio::test]
    async fn test_consumer_builder_rejects_heartbeat_greater_than_session() {
        let result = Consumer::builder()
            .bootstrap_servers("localhost:9092")
            .group_id("test")
            .session_timeout(Duration::from_secs(5))
            .heartbeat_interval(Duration::from_secs(10))
            .build()
            .await;

        assert!(result.is_err());
    }

    /// Verify max_poll_records truncation recomputes offset updates
    /// to prevent data loss for undelivered records.
    #[test]
    fn test_max_poll_records_offset_recomputation() {
        // Simulate what poll() does: given 5 records but max_poll_records=3,
        // only offsets for the first 3 records should be advanced.
        let records: Vec<ConsumerRecord> = (0..5)
            .map(|i| ConsumerRecord {
                topic: "topic1".to_string(),
                partition: 0,
                offset: 100 + i,
                timestamp: 0,
                timestamp_type: 0,
                key: None,
                value: Some(bytes::Bytes::from(format!("val-{i}"))),
                headers: vec![],
                leader_epoch: None,
                delivery_count: None,
            })
            .collect();

        let original_offset_updates: Vec<((String, PartitionId), Offset)> =
            vec![(("topic1".to_string(), 0), 105)]; // offset after last record

        let max = 3usize;
        let mut truncated = records;
        truncated.truncate(max);

        // Recompute offsets from truncated records only
        let mut delivered_offsets: HashMap<(String, PartitionId), Offset> = HashMap::new();
        for r in &truncated {
            let key = (r.topic.clone(), r.partition);
            let entry = delivered_offsets.entry(key).or_insert(r.offset);
            if r.offset > *entry {
                *entry = r.offset;
            }
        }
        let new_offset_updates: Vec<_> = delivered_offsets
            .into_iter()
            .map(|(key, offset)| (key, offset + 1))
            .collect();

        // Should advance to offset 103 (100+2+1), NOT 105
        assert_eq!(new_offset_updates.len(), 1);
        let (key, offset) = &new_offset_updates[0];
        assert_eq!(key, &("topic1".to_string(), 0));
        assert_eq!(*offset, 103); // 100 + 2 (last delivered record offset) + 1

        // Not the original 105
        assert_ne!(*offset, original_offset_updates[0].1);
    }

    /// Verify max_poll_records with multiple partitions recomputes
    /// offsets correctly per partition.
    #[test]
    fn test_max_poll_records_multi_partition_offset() {
        let mut records = Vec::new();
        // 3 records from partition 0
        for i in 0..3 {
            records.push(ConsumerRecord {
                topic: "topic1".to_string(),
                partition: 0,
                offset: 50 + i,
                timestamp: 0,
                timestamp_type: 0,
                key: None,
                value: Some(bytes::Bytes::from("val")),
                headers: vec![],
                leader_epoch: None,
                delivery_count: None,
            });
        }
        // 3 records from partition 1
        for i in 0..3 {
            records.push(ConsumerRecord {
                topic: "topic1".to_string(),
                partition: 1,
                offset: 200 + i,
                timestamp: 0,
                timestamp_type: 0,
                key: None,
                value: Some(bytes::Bytes::from("val")),
                headers: vec![],
                leader_epoch: None,
                delivery_count: None,
            });
        }

        // Truncate to 4 records (all 3 from p0 + 1 from p1)
        records.truncate(4);

        let mut delivered_offsets: HashMap<(String, PartitionId), Offset> = HashMap::new();
        for r in &records {
            let key = (r.topic.clone(), r.partition);
            let entry = delivered_offsets.entry(key).or_insert(r.offset);
            if r.offset > *entry {
                *entry = r.offset;
            }
        }

        // Partition 0: last delivered = 52 → advanced to 53
        assert_eq!(
            *delivered_offsets.get(&("topic1".to_string(), 0)).unwrap(),
            52
        );
        // Partition 1: last delivered = 200 → advanced to 201
        assert_eq!(
            *delivered_offsets.get(&("topic1".to_string(), 1)).unwrap(),
            200
        );
    }

    #[test]
    fn test_consumer_builder_group_instance_id() {
        let builder = Consumer::builder()
            .bootstrap_servers("localhost:9092")
            .group_id("test-group")
            .group_instance_id("my-instance");

        assert_eq!(
            builder.config.group_instance_id,
            Some("my-instance".to_string())
        );
    }

    #[test]
    fn test_consumer_builder_interceptor() {
        use crate::interceptor::ConsumerInterceptor;
        use crate::interceptor::InterceptorResult;

        #[derive(Debug)]
        struct TestInterceptor;
        impl ConsumerInterceptor for TestInterceptor {
            fn on_consume(&self, _records: &[ConsumerRecord]) -> InterceptorResult {
                Ok(())
            }
        }

        let builder = Consumer::builder()
            .bootstrap_servers("localhost:9092")
            .group_id("test-group")
            .interceptor(Arc::new(TestInterceptor));

        assert_eq!(builder.interceptors.len(), 1);
    }

    #[test]
    fn test_consumer_builder_add_interceptor() {
        use crate::interceptor::ConsumerInterceptor;

        #[derive(Debug)]
        struct A;
        impl ConsumerInterceptor for A {}

        #[derive(Debug)]
        struct B;
        impl ConsumerInterceptor for B {}

        let builder = Consumer::builder()
            .bootstrap_servers("localhost:9092")
            .group_id("test-group")
            .add_interceptor(Arc::new(A))
            .add_interceptor(Arc::new(B));
        assert_eq!(builder.interceptors.len(), 2);
    }

    #[test]
    fn test_consumer_builder_interceptor_replaces_chain() {
        use crate::interceptor::ConsumerInterceptor;

        #[derive(Debug)]
        struct A;
        impl ConsumerInterceptor for A {}

        #[derive(Debug)]
        struct B;
        impl ConsumerInterceptor for B {}

        let builder = Consumer::builder()
            .bootstrap_servers("localhost:9092")
            .group_id("test-group")
            .add_interceptor(Arc::new(A))
            .add_interceptor(Arc::new(A))
            .interceptor(Arc::new(B));
        assert_eq!(builder.interceptors.len(), 1);
    }

    // recv() buffers remaining records so none are lost.
    #[tokio::test]
    async fn test_recv_buffer_returns_all_records() {
        use std::collections::VecDeque;

        // Simulate a consumer with pre-filled recv_buffer
        let mut buffer = VecDeque::new();
        buffer.push_back(ConsumerRecord {
            topic: "t".into(),
            partition: 0,
            offset: 1,
            timestamp: 0,
            timestamp_type: 0,
            key: None,
            value: Some(bytes::Bytes::from("r1")),
            headers: vec![],
            leader_epoch: None,
            delivery_count: None,
        });
        buffer.push_back(ConsumerRecord {
            topic: "t".into(),
            partition: 0,
            offset: 2,
            timestamp: 0,
            timestamp_type: 0,
            key: None,
            value: Some(bytes::Bytes::from("r2")),
            headers: vec![],
            leader_epoch: None,
            delivery_count: None,
        });

        assert_eq!(buffer.len(), 2);
        let first = buffer.pop_front().unwrap();
        assert_eq!(first.offset, 1);
        let second = buffer.pop_front().unwrap();
        assert_eq!(second.offset, 2);
        assert!(buffer.is_empty());
    }

    // assign() is rejected when group coordinator is active.
    #[test]
    fn test_assign_with_group_id_configured() {
        let builder = Consumer::builder()
            .bootstrap_servers("localhost:9092")
            .group_id("test-group");

        // When group_id is set, group_coordinator will be Some after new().
        // We verify the config at builder level.
        assert!(builder.config.group_id.is_some());
    }

    // subscribe() replaces rather than appending.
    #[test]
    fn test_subscribe_replaces_subscriptions() {
        let rt = tokio::runtime::Builder::new_current_thread()
            .enable_all()
            .build()
            .unwrap();

        rt.block_on(async {
            let subs: RwLock<HashSet<String>> = RwLock::new(HashSet::new());

            // First subscribe
            {
                let mut s = subs.write().await;
                s.clear(); // clear before insert
                s.insert("topic1".to_string());
            }
            assert_eq!(subs.read().await.len(), 1);
            assert!(subs.read().await.contains("topic1"));

            // Second subscribe replaces, not appends
            {
                let mut s = subs.write().await;
                s.clear(); // clear before insert
                s.insert("topic2".to_string());
            }
            assert_eq!(subs.read().await.len(), 1);
            assert!(subs.read().await.contains("topic2"));
            assert!(!subs.read().await.contains("topic1"));
        });
    }

    // unsubscribe() clears offsets, paused, and recv_buffer.
    #[test]
    fn test_unsubscribe_clears_all_state() {
        let rt = tokio::runtime::Builder::new_current_thread()
            .enable_all()
            .build()
            .unwrap();

        rt.block_on(async {
            let offsets: RwLock<HashMap<(String, PartitionId), Offset>> =
                RwLock::new(HashMap::new());
            let paused: RwLock<HashSet<(String, PartitionId)>> = RwLock::new(HashSet::new());
            let assignments: RwLock<HashMap<String, Vec<PartitionId>>> =
                RwLock::new(HashMap::new());
            let recv_buffer: RwLock<std::collections::VecDeque<ConsumerRecord>> =
                RwLock::new(std::collections::VecDeque::new());

            // Populate state
            offsets.write().await.insert(("t".into(), 0), 100);
            paused.write().await.insert(("t".into(), 0));
            assignments.write().await.insert("t".into(), vec![0]);
            recv_buffer.write().await.push_back(ConsumerRecord {
                topic: "t".into(),
                partition: 0,
                offset: 0,
                timestamp: 0,
                timestamp_type: 0,
                key: None,
                value: None,
                headers: vec![],
                leader_epoch: None,
                delivery_count: None,
            });

            // Simulate unsubscribe clearing
            offsets.write().await.clear();
            paused.write().await.clear();
            assignments.write().await.clear();
            recv_buffer.write().await.clear();

            assert!(offsets.read().await.is_empty());
            assert!(paused.read().await.is_empty());
            assert!(assignments.read().await.is_empty());
            assert!(recv_buffer.read().await.is_empty());
        });
    }

    // Fetch skips partitions with no tracked offset.
    #[test]
    fn test_fetch_skips_untracked_partitions() {
        let rt = tokio::runtime::Builder::new_current_thread()
            .enable_all()
            .build()
            .unwrap();

        rt.block_on(async {
            let offsets: RwLock<HashMap<(String, PartitionId), Offset>> =
                RwLock::new(HashMap::new());
            offsets.write().await.insert(("t".into(), 0), 42);

            let o = offsets.read().await;
            // Partition 0 has an offset
            assert_eq!(o.get(&("t".to_string(), 0)).copied(), Some(42));
            // Partition 1 has no offset — should be skipped
            assert_eq!(o.get(&("t".to_string(), 1)).copied(), None);
        });
    }

    // Commit filtering uses group_coordinator check, not assigned_set emptiness.
    #[test]
    fn test_commit_filter_does_not_leak_stale_offsets() {
        let rt = tokio::runtime::Builder::new_current_thread()
            .enable_all()
            .build()
            .unwrap();

        rt.block_on(async {
            let offsets: HashMap<(String, PartitionId), Offset> = [
                (("topic1".into(), 0), 100),
                (("topic2".into(), 0), 200), // stale: not assigned
            ]
            .into_iter()
            .collect();

            let assigned_set: HashSet<(String, PartitionId)> = HashSet::new(); // empty

            // OLD behavior: is_empty() would let ALL offsets through — BAD
            let old_filtered: Vec<_> = offsets
                .iter()
                .filter(|((t, p), _)| {
                    assigned_set.is_empty() || assigned_set.contains(&(t.clone(), *p))
                })
                .collect();
            assert_eq!(old_filtered.len(), 2); // Both pass — wrong

            // NEW behavior: group consumers never commit unassigned
            let has_group = true;
            let new_filtered: Vec<_> = offsets
                .iter()
                .filter(|((t, p), _)| !has_group || assigned_set.contains(&(t.clone(), *p)))
                .collect();
            assert_eq!(new_filtered.len(), 0); // None pass when empty — correct
        });
    }

    // group field removed — only group_coordinator accessor exists.
    #[test]
    fn test_no_legacy_group_field() {
        let builder = Consumer::builder()
            .bootstrap_servers("localhost:9092")
            .group_id("test-group");
        // The builder should have no group field; only group_coordinator is used
        assert!(builder.config.group_id.is_some());
    }

    #[test]
    fn test_max_poll_interval_used_for_rebalance() {
        // rebalance_timeout should default to max_poll_interval (not session_timeout)
        let config = ConsumerConfig::default();
        // In the Java client, rebalance_timeout defaults to max.poll.interval.ms (300s)
        // not session.timeout.ms (10s). Verify our config has both.
        assert_eq!(config.max_poll_interval, Duration::from_secs(300));
        assert_eq!(config.session_timeout, Duration::from_secs(10));
        // The rebalance_timeout passed to GroupCoordinator should be max_poll_interval
        assert!(config.max_poll_interval > config.session_timeout);
    }

    /// Test that partition grouping by leader works correctly.
    /// This mirrors the grouping logic inside resolve_list_offsets.
    #[test]
    fn test_list_offsets_partition_grouping_by_leader() {
        // Simulate the leader-based grouping that resolve_list_offsets performs.
        let leader_map: HashMap<(&str, PartitionId), crate::BrokerId> = [
            (("topic1", 0), 1),
            (("topic1", 1), 2),
            (("topic2", 0), 1), // same leader as topic1-0
            (("topic2", 1), 3),
        ]
        .into_iter()
        .collect();

        let mut partitions: HashMap<String, Vec<PartitionId>> = HashMap::new();
        partitions.insert("topic1".to_string(), vec![0, 1]);
        partitions.insert("topic2".to_string(), vec![0, 1]);

        let mut by_leader: HashMap<crate::BrokerId, Vec<(String, PartitionId)>> = HashMap::new();
        for (topic, parts) in &partitions {
            for &p in parts {
                if let Some(&leader) = leader_map.get(&(topic.as_str(), p)) {
                    by_leader
                        .entry(leader)
                        .or_default()
                        .push((topic.clone(), p));
                }
            }
        }

        // Broker 1 should have topic1-0 and topic2-0
        assert_eq!(by_leader[&1].len(), 2);
        assert!(by_leader[&1].contains(&("topic1".to_string(), 0)));
        assert!(by_leader[&1].contains(&("topic2".to_string(), 0)));
        // Broker 2 should have topic1-1
        assert_eq!(by_leader[&2].len(), 1);
        assert_eq!(by_leader[&2][0], ("topic1".to_string(), 1));
        // Broker 3 should have topic2-1
        assert_eq!(by_leader[&3].len(), 1);
        assert_eq!(by_leader[&3][0], ("topic2".to_string(), 1));
    }

    /// Test request construction from grouped partitions.
    #[test]
    fn test_list_offsets_request_construction() {
        let leader_partitions: Vec<(String, PartitionId)> = vec![
            ("topic1".to_string(), 0),
            ("topic1".to_string(), 2),
            ("topic2".to_string(), 1),
        ];
        let timestamp = -1i64; // latest

        let mut topics_map: HashMap<String, Vec<ListOffsetsRequestPartition>> = HashMap::new();
        for (topic, partition) in &leader_partitions {
            topics_map
                .entry(topic.clone())
                .or_default()
                .push(ListOffsetsRequestPartition {
                    partition_index: *partition,
                    current_leader_epoch: -1,
                    timestamp,
                });
        }

        let topics: Vec<ListOffsetsRequestTopic> = topics_map
            .into_iter()
            .map(|(name, parts)| ListOffsetsRequestTopic {
                name,
                partitions: parts,
            })
            .collect();

        let request = ListOffsetsRequest {
            replica_id: -1,
            isolation_level: 0,
            topics,
            timeout_ms: None,
        };

        assert_eq!(request.replica_id, -1);
        assert_eq!(request.topics.len(), 2);

        // Find topic1 and topic2 in the request
        let t1 = request.topics.iter().find(|t| t.name == "topic1").unwrap();
        assert_eq!(t1.partitions.len(), 2);
        assert!(t1.partitions.iter().any(|p| p.partition_index == 0));
        assert!(t1.partitions.iter().any(|p| p.partition_index == 2));
        for p in &t1.partitions {
            assert_eq!(p.timestamp, -1);
            assert_eq!(p.current_leader_epoch, -1);
        }

        let t2 = request.topics.iter().find(|t| t.name == "topic2").unwrap();
        assert_eq!(t2.partitions.len(), 1);
        assert_eq!(t2.partitions[0].partition_index, 1);
    }

    /// Test response result extraction — every partition maps to Ok(offset).
    #[test]
    fn test_list_offsets_response_result_extraction() {
        use crate::error::ErrorCode;
        use crate::protocol::ListOffsetsResponsePartition;
        use crate::protocol::ListOffsetsResponseTopic;

        let response = ListOffsetsResponse {
            topics: vec![
                ListOffsetsResponseTopic {
                    name: "topic1".to_string(),
                    partitions: vec![
                        ListOffsetsResponsePartition {
                            partition_index: 0,
                            error_code: ErrorCode::None,
                            timestamp: -1,
                            offset: 42,
                            leader_epoch: -1,
                        },
                        ListOffsetsResponsePartition {
                            partition_index: 1,
                            error_code: ErrorCode::None,
                            timestamp: -1,
                            offset: 100,
                            leader_epoch: -1,
                        },
                    ],
                },
                ListOffsetsResponseTopic {
                    name: "topic2".to_string(),
                    partitions: vec![ListOffsetsResponsePartition {
                        partition_index: 0,
                        error_code: ErrorCode::None,
                        timestamp: -1,
                        offset: 7,
                        leader_epoch: -1,
                    }],
                },
            ],
        };

        let mut result: HashMap<(String, PartitionId), Result<Offset>> = HashMap::new();
        apply_list_offsets_response(&response, &mut result);

        assert_eq!(result.len(), 3);
        assert_eq!(*result[&("topic1".to_string(), 0)].as_ref().unwrap(), 42);
        assert_eq!(*result[&("topic1".to_string(), 1)].as_ref().unwrap(), 100);
        assert_eq!(*result[&("topic2".to_string(), 0)].as_ref().unwrap(), 7);
    }

    /// Test partial failure — successful partitions map to Ok, failed partition
    /// maps to Err. No partition is dropped from the result.
    #[test]
    fn test_list_offsets_partial_failure_keeps_successes() {
        use crate::error::ErrorCode;
        use crate::protocol::ListOffsetsResponsePartition;
        use crate::protocol::ListOffsetsResponseTopic;

        let response = ListOffsetsResponse {
            topics: vec![ListOffsetsResponseTopic {
                name: "topic1".to_string(),
                partitions: vec![
                    ListOffsetsResponsePartition {
                        partition_index: 0,
                        error_code: ErrorCode::None,
                        timestamp: -1,
                        offset: 42,
                        leader_epoch: -1,
                    },
                    ListOffsetsResponsePartition {
                        partition_index: 1,
                        error_code: ErrorCode::NotLeaderForPartition,
                        timestamp: -1,
                        offset: -1,
                        leader_epoch: -1,
                    },
                    ListOffsetsResponsePartition {
                        partition_index: 2,
                        error_code: ErrorCode::None,
                        timestamp: -1,
                        offset: 99,
                        leader_epoch: -1,
                    },
                ],
            }],
        };

        let mut result: HashMap<(String, PartitionId), Result<Offset>> = HashMap::new();
        apply_list_offsets_response(&response, &mut result);

        // All three partitions are present in the map.
        assert_eq!(result.len(), 3);
        // Successful partitions carry Ok(offset).
        assert_eq!(*result[&("topic1".to_string(), 0)].as_ref().unwrap(), 42);
        assert_eq!(*result[&("topic1".to_string(), 2)].as_ref().unwrap(), 99);
        // Failed partition is present with Err, not absent.
        assert!(result[&("topic1".to_string(), 1)].is_err());
        let err_msg = result[&("topic1".to_string(), 1)]
            .as_ref()
            .unwrap_err()
            .to_string();
        assert!(
            err_msg.contains("ListOffsets error"),
            "unexpected: {err_msg}"
        );
    }

    /// Test that an all-failed response returns every partition as Err —
    /// the function itself does not return a top-level error.
    #[test]
    fn test_list_offsets_all_failed_returns_error() {
        use crate::error::ErrorCode;
        use crate::protocol::ListOffsetsResponsePartition;
        use crate::protocol::ListOffsetsResponseTopic;

        let response = ListOffsetsResponse {
            topics: vec![ListOffsetsResponseTopic {
                name: "topic1".to_string(),
                partitions: vec![ListOffsetsResponsePartition {
                    partition_index: 0,
                    error_code: ErrorCode::NotLeaderForPartition,
                    timestamp: -1,
                    offset: -1,
                    leader_epoch: -1,
                }],
            }],
        };

        let mut result: HashMap<(String, PartitionId), Result<Offset>> = HashMap::new();
        apply_list_offsets_response(&response, &mut result);

        // The partition is present in the map, mapped to Err.
        assert_eq!(result.len(), 1);
        assert!(result[&("topic1".to_string(), 0)].is_err());
        let err_msg = result[&("topic1".to_string(), 0)]
            .as_ref()
            .unwrap_err()
            .to_string();
        assert!(
            err_msg.contains("ListOffsets error"),
            "unexpected: {err_msg}"
        );
    }

    /// Test ListOffsets request encoding for v1 and v2 produces expected sizes.
    #[test]
    fn test_list_offsets_request_encode_v1_v2() {
        use bytes::BytesMut;

        let request = ListOffsetsRequest {
            replica_id: -1,
            isolation_level: 1,
            topics: vec![ListOffsetsRequestTopic {
                name: "test-topic".to_string(),
                partitions: vec![
                    ListOffsetsRequestPartition {
                        partition_index: 0,
                        current_leader_epoch: -1,
                        timestamp: -1, // latest
                    },
                    ListOffsetsRequestPartition {
                        partition_index: 1,
                        current_leader_epoch: -1,
                        timestamp: -2, // earliest
                    },
                ],
            }],
            timeout_ms: None,
        };

        // v1 encode
        let mut buf = BytesMut::new();
        request.encode_v1(&mut buf).unwrap();
        let encoded_v1_len = buf.len();
        assert!(encoded_v1_len > 0);

        // v2 encode produces additional isolation_level byte
        let mut buf_v2 = BytesMut::new();
        request.encode_v2(&mut buf_v2).unwrap();
        // v2 has one extra byte for isolation_level
        assert_eq!(buf_v2.len(), encoded_v1_len + 1);
    }

    // ── Cooperative rebalance algorithm tests ───────────────────────────

    /// Compute newly-assigned diff (new - old) as used in
    /// finalize_cooperative_assignment.
    fn cooperative_newly_assigned(
        new: &HashMap<String, Vec<PartitionId>>,
        old: &HashMap<String, Vec<PartitionId>>,
    ) -> Vec<TopicPartition> {
        let old_sets: HashMap<&String, HashSet<PartitionId>> = old
            .iter()
            .map(|(t, ps)| (t, ps.iter().copied().collect()))
            .collect();
        let mut result = Vec::new();
        for (topic, partitions) in new {
            let old_set = old_sets.get(topic);
            for &p in partitions {
                let is_new = old_set.is_none_or(|os| !os.contains(&p));
                if is_new {
                    result.push(TopicPartition::new(topic, p));
                }
            }
        }
        result
    }

    /// Compute cooperative revocations (old - new) as used in the
    /// no-revocations poll path.
    fn cooperative_revocations(
        old: &HashMap<String, Vec<PartitionId>>,
        new: &HashMap<String, Vec<PartitionId>>,
    ) -> Vec<TopicPartition> {
        let new_sets: HashMap<&String, HashSet<PartitionId>> = new
            .iter()
            .map(|(t, ps)| (t, ps.iter().copied().collect()))
            .collect();
        let mut result = Vec::new();
        for (topic, partitions) in old {
            let new_set = new_sets.get(topic);
            for &p in partitions {
                let gone = new_set.is_none_or(|ns| !ns.contains(&p));
                if gone {
                    result.push(TopicPartition::new(topic, p));
                }
            }
        }
        result
    }

    /// Simulate the apply_partition_revocations HashMap algorithm.
    fn apply_revocations_to_assignments(
        assignments: &mut HashMap<String, Vec<PartitionId>>,
        revoked: &[(String, PartitionId)],
    ) {
        let mut revoked_by_topic: HashMap<&str, HashSet<PartitionId>> = HashMap::new();
        for (topic, partition) in revoked {
            revoked_by_topic
                .entry(topic.as_str())
                .or_default()
                .insert(*partition);
        }
        for (topic, revoked_parts) in &revoked_by_topic {
            if let Some(parts) = assignments.get_mut(*topic) {
                parts.retain(|p| !revoked_parts.contains(p));
                if parts.is_empty() {
                    assignments.remove(*topic);
                }
            }
        }
    }

    #[test]
    fn test_cooperative_newly_assigned_fresh_join() {
        let old: HashMap<String, Vec<PartitionId>> = HashMap::new();
        let new: HashMap<String, Vec<PartitionId>> = [
            ("topic1".to_string(), vec![0, 1, 2]),
            ("topic2".to_string(), vec![0]),
        ]
        .into_iter()
        .collect();

        let result = cooperative_newly_assigned(&new, &old);
        assert_eq!(result.len(), 4);
        assert!(result.contains(&TopicPartition::new("topic1", 0)));
        assert!(result.contains(&TopicPartition::new("topic1", 1)));
        assert!(result.contains(&TopicPartition::new("topic1", 2)));
        assert!(result.contains(&TopicPartition::new("topic2", 0)));
    }

    #[test]
    fn test_cooperative_newly_assigned_partial_overlap() {
        let old: HashMap<String, Vec<PartitionId>> = [
            ("topic1".to_string(), vec![0, 1]),
            ("topic2".to_string(), vec![0]),
        ]
        .into_iter()
        .collect();
        let new: HashMap<String, Vec<PartitionId>> = [
            ("topic1".to_string(), vec![1, 2]),
            ("topic3".to_string(), vec![0]),
        ]
        .into_iter()
        .collect();

        let result = cooperative_newly_assigned(&new, &old);
        // topic1-1 retained, topic1-2 new, topic3-0 new
        assert_eq!(result.len(), 2);
        assert!(result.contains(&TopicPartition::new("topic1", 2)));
        assert!(result.contains(&TopicPartition::new("topic3", 0)));
        assert!(!result.contains(&TopicPartition::new("topic1", 1))); // retained
    }

    #[test]
    fn test_cooperative_newly_assigned_identical() {
        let assignment: HashMap<String, Vec<PartitionId>> =
            [("topic1".to_string(), vec![0, 1])].into_iter().collect();

        let result = cooperative_newly_assigned(&assignment, &assignment);
        assert!(result.is_empty());
    }

    #[test]
    fn test_cooperative_revocations_partial() {
        let old: HashMap<String, Vec<PartitionId>> = [
            ("topic1".to_string(), vec![0, 1, 2]),
            ("topic2".to_string(), vec![0]),
        ]
        .into_iter()
        .collect();
        let new: HashMap<String, Vec<PartitionId>> =
            [("topic1".to_string(), vec![1])].into_iter().collect();

        let result = cooperative_revocations(&old, &new);
        // topic1-0, topic1-2, topic2-0 revoked; topic1-1 retained
        assert_eq!(result.len(), 3);
        assert!(result.contains(&TopicPartition::new("topic1", 0)));
        assert!(result.contains(&TopicPartition::new("topic1", 2)));
        assert!(result.contains(&TopicPartition::new("topic2", 0)));
    }

    #[test]
    fn test_cooperative_revocations_full() {
        let old: HashMap<String, Vec<PartitionId>> =
            [("topic1".to_string(), vec![0, 1])].into_iter().collect();
        let new: HashMap<String, Vec<PartitionId>> = HashMap::new();

        let result = cooperative_revocations(&old, &new);
        assert_eq!(result.len(), 2);
    }

    #[test]
    fn test_cooperative_revocations_none() {
        let old: HashMap<String, Vec<PartitionId>> =
            [("topic1".to_string(), vec![0])].into_iter().collect();
        let new: HashMap<String, Vec<PartitionId>> =
            [("topic1".to_string(), vec![0, 1])].into_iter().collect();

        let result = cooperative_revocations(&old, &new);
        assert!(result.is_empty());
    }

    #[test]
    fn test_eager_cleanup_preserves_pause_for_retained_partitions() {
        let old: HashMap<String, Vec<PartitionId>> =
            [("topic1".to_string(), vec![0, 1])].into_iter().collect();
        let new: HashMap<String, Vec<PartitionId>> =
            [("topic1".to_string(), vec![1, 2])].into_iter().collect();

        let revoked = revoked_partitions_diff(&old, &new);
        let revoked_tuples: Vec<(String, PartitionId)> = revoked
            .into_iter()
            .map(|tp| (tp.topic, tp.partition))
            .collect();

        let mut paused: HashSet<(String, PartitionId)> =
            [("topic1".to_string(), 0), ("topic1".to_string(), 1)]
                .into_iter()
                .collect();

        for key in &revoked_tuples {
            paused.remove(key);
        }

        assert!(!paused.contains(&("topic1".to_string(), 0)));
        assert!(paused.contains(&("topic1".to_string(), 1)));
    }

    #[test]
    fn test_apply_revocations_removes_partitions() {
        let mut assignments: HashMap<String, Vec<PartitionId>> = [
            ("topic1".to_string(), vec![0, 1, 2]),
            ("topic2".to_string(), vec![0, 1]),
        ]
        .into_iter()
        .collect();

        let revoked = vec![
            ("topic1".to_string(), 0),
            ("topic1".to_string(), 2),
            ("topic2".to_string(), 1),
        ];

        apply_revocations_to_assignments(&mut assignments, &revoked);

        assert_eq!(assignments["topic1"], vec![1]);
        assert_eq!(assignments["topic2"], vec![0]);
    }

    #[test]
    fn test_apply_revocations_removes_empty_topics() {
        let mut assignments: HashMap<String, Vec<PartitionId>> = [
            ("topic1".to_string(), vec![0]),
            ("topic2".to_string(), vec![0, 1]),
        ]
        .into_iter()
        .collect();

        let revoked = vec![("topic1".to_string(), 0)];
        apply_revocations_to_assignments(&mut assignments, &revoked);

        // topic1 should be removed entirely since it became empty
        assert!(!assignments.contains_key("topic1"));
        assert_eq!(assignments.len(), 1);
        assert_eq!(assignments["topic2"], vec![0, 1]);
    }

    #[test]
    fn test_apply_revocations_nonexistent_partition() {
        let mut assignments: HashMap<String, Vec<PartitionId>> =
            [("topic1".to_string(), vec![0, 1])].into_iter().collect();

        let revoked = vec![
            ("topic1".to_string(), 5), // doesn't exist
            ("topic3".to_string(), 0), // topic doesn't exist
        ];
        apply_revocations_to_assignments(&mut assignments, &revoked);

        // Assignments unchanged
        assert_eq!(assignments["topic1"], vec![0, 1]);
    }

    /// Full cooperative two-phase scenario: verify newly-assigned and revoked
    /// diffs are consistent across the protocol flow.
    #[test]
    fn test_cooperative_two_phase_rebalance_consistency() {
        // Phase 1: existing assignment pre-rebalance
        let phase0: HashMap<String, Vec<PartitionId>> = [
            ("topic1".to_string(), vec![0, 1, 2]),
            ("topic2".to_string(), vec![0]),
        ]
        .into_iter()
        .collect();

        // Phase 1 result: broker says revoke topic1-2 and topic2-0
        let phase1_target: HashMap<String, Vec<PartitionId>> =
            [("topic1".to_string(), vec![0, 1])].into_iter().collect();

        let to_revoke = cooperative_revocations(&phase0, &phase1_target);
        assert_eq!(to_revoke.len(), 2);
        assert!(to_revoke.contains(&TopicPartition::new("topic1", 2)));
        assert!(to_revoke.contains(&TopicPartition::new("topic2", 0)));

        // Apply revocations
        let mut current = phase0.clone();
        let revoked_tuples: Vec<(String, PartitionId)> = to_revoke
            .iter()
            .map(|tp| (tp.topic.clone(), tp.partition))
            .collect();
        apply_revocations_to_assignments(&mut current, &revoked_tuples);
        assert_eq!(current["topic1"], vec![0, 1]);
        assert!(!current.contains_key("topic2"));

        // Phase 2: rejoin gives final assignment with a new partition
        let phase2_final: HashMap<String, Vec<PartitionId>> =
            [("topic1".to_string(), vec![0, 1, 3])]
                .into_iter()
                .collect();

        let newly_assigned = cooperative_newly_assigned(&phase2_final, &current);
        assert_eq!(newly_assigned.len(), 1);
        assert!(newly_assigned.contains(&TopicPartition::new("topic1", 3)));

        // No further revocations needed
        let extra_revoke = cooperative_revocations(&current, &phase2_final);
        assert!(extra_revoke.is_empty());
    }

    /// Verify that cooperative rebalance callbacks follow Java client ordering:
    /// on_partitions_revoked fires before on_partitions_assigned.
    #[test]
    fn test_cooperative_callback_ordering() {
        use std::sync::atomic::AtomicU64;
        use std::sync::atomic::Ordering;

        struct OrderTracker {
            revoke_seq: AtomicU64,
            assign_seq: AtomicU64,
            counter: AtomicU64,
        }
        impl ConsumerRebalanceListener for OrderTracker {
            fn on_partitions_assigned(&self, _: &[TopicPartition]) {
                self.assign_seq.store(
                    self.counter.fetch_add(1, Ordering::SeqCst),
                    Ordering::SeqCst,
                );
            }
            fn on_partitions_revoked(&self, _: &[TopicPartition]) {
                self.revoke_seq.store(
                    self.counter.fetch_add(1, Ordering::SeqCst),
                    Ordering::SeqCst,
                );
            }
        }

        let tracker = Arc::new(OrderTracker {
            revoke_seq: AtomicU64::new(u64::MAX),
            assign_seq: AtomicU64::new(u64::MAX),
            counter: AtomicU64::new(0),
        });

        // Simulate cooperative rebalance callback sequence:
        // 1. Revoke phase
        let revoked = vec![TopicPartition::new("topic1", 2)];
        tracker.on_partitions_revoked(&revoked);
        // 2. Assign phase
        let assigned = vec![
            TopicPartition::new("topic1", 0),
            TopicPartition::new("topic1", 1),
            TopicPartition::new("topic1", 3),
        ];
        tracker.on_partitions_assigned(&assigned);

        let revoke_order = tracker.revoke_seq.load(Ordering::SeqCst);
        let assign_order = tracker.assign_seq.load(Ordering::SeqCst);
        assert!(
            revoke_order < assign_order,
            "on_partitions_revoked (seq={revoke_order}) must fire before on_partitions_assigned (seq={assign_order})"
        );
    }

    /// Verify that on_partitions_assigned is called even with empty assignment
    /// (more consumers than partitions).
    #[test]
    fn test_cooperative_on_assigned_fires_on_empty() {
        use std::sync::atomic::AtomicBool;
        use std::sync::atomic::Ordering;

        struct EmptyTracker {
            assigned_called: AtomicBool,
        }
        impl ConsumerRebalanceListener for EmptyTracker {
            fn on_partitions_assigned(&self, parts: &[TopicPartition]) {
                assert!(parts.is_empty());
                self.assigned_called.store(true, Ordering::SeqCst);
            }
            fn on_partitions_revoked(&self, _: &[TopicPartition]) {}
        }

        let tracker = EmptyTracker {
            assigned_called: AtomicBool::new(false),
        };
        tracker.on_partitions_assigned(&[]);
        assert!(tracker.assigned_called.load(Ordering::SeqCst));
    }

    /// Build a `PartitionState` map entry with only the high watermark set.
    /// Used by the lag-computation tests below.
    fn ps_with_hw(watermark: Offset) -> PartitionState {
        PartitionState {
            high_watermark: Some(watermark),
            ..Default::default()
        }
    }

    /// Test the lag computation logic via the extracted `compute_aggregate_lag`
    /// helper — the same function used by `recompute_lag_metrics()` in
    /// production.
    #[test]
    fn test_lag_computation_logic() {
        let mut offsets: HashMap<(String, PartitionId), Offset> = HashMap::new();
        let mut partition_state: HashMap<(String, PartitionId), PartitionState> = HashMap::new();

        // No data → lag is 0
        let (total_lag, max_lag) = compute_aggregate_lag(&offsets, &partition_state);
        assert_eq!(total_lag, 0);
        assert_eq!(max_lag, 0);

        // Populate two partitions
        offsets.insert(("t".into(), 0), 50);
        offsets.insert(("t".into(), 1), 100);
        partition_state.insert(("t".into(), 0), ps_with_hw(80));
        partition_state.insert(("t".into(), 1), ps_with_hw(120));

        let (total_lag, max_lag) = compute_aggregate_lag(&offsets, &partition_state);

        assert_eq!(total_lag, 50); // (80-50) + (120-100)
        assert_eq!(max_lag, 30); // max(30, 20)
    }

    #[test]
    fn test_lag_negative_clamped_to_zero() {
        // Position ahead of high watermark (can happen briefly after a reset)
        let mut offsets: HashMap<(String, PartitionId), Offset> = HashMap::new();
        let mut partition_state: HashMap<(String, PartitionId), PartitionState> = HashMap::new();

        offsets.insert(("t".into(), 0), 100);
        partition_state.insert(("t".into(), 0), ps_with_hw(80));

        let (total_lag, _) = compute_aggregate_lag(&offsets, &partition_state);
        assert_eq!(total_lag, 0);
    }

    #[test]
    fn test_lag_partial_watermarks() {
        // High watermark known for only one of two partitions
        let mut offsets: HashMap<(String, PartitionId), Offset> = HashMap::new();
        let mut partition_state: HashMap<(String, PartitionId), PartitionState> = HashMap::new();

        offsets.insert(("t".into(), 0), 50);
        offsets.insert(("t".into(), 1), 100);
        partition_state.insert(("t".into(), 0), ps_with_hw(80));
        // Partition 1 has no high watermark

        let (total_lag, _) = compute_aggregate_lag(&offsets, &partition_state);
        assert_eq!(total_lag, 30); // Only partition 0 contributes
    }

    #[test]
    fn test_lag_after_revocation() {
        // Simulate clearing revoked partitions and recomputing lag metrics
        let mut offsets: HashMap<(String, PartitionId), Offset> = HashMap::new();
        let mut partition_state: HashMap<(String, PartitionId), PartitionState> = HashMap::new();

        offsets.insert(("t".into(), 0), 50);
        offsets.insert(("t".into(), 1), 100);
        partition_state.insert(("t".into(), 0), ps_with_hw(100)); // lag = 50
        partition_state.insert(("t".into(), 1), ps_with_hw(200)); // lag = 100

        // Revoke partition 0
        let revoked = vec![TopicPartition::new("t", 0)];
        for tp in &revoked {
            let key = (tp.topic.clone(), tp.partition);
            offsets.remove(&key);
            partition_state.remove(&key);
        }

        assert!(!partition_state.contains_key(&("t".into(), 0)));
        assert!(partition_state.contains_key(&("t".into(), 1)));

        // Recompute lag from remaining caches (same logic as apply_partition_revocations)
        let (total_lag, max_lag) = compute_aggregate_lag(&offsets, &partition_state);

        // Only partition 1 remains: lag = 200 - 100 = 100
        assert_eq!(total_lag, 100);
        assert_eq!(max_lag, 100);
    }

    #[test]
    fn test_lag_clear_resets_to_zero() {
        // After clear_partition_state, all caches are empty → lag must be 0
        let mut offsets: HashMap<(String, PartitionId), Offset> = HashMap::new();
        let mut partition_state: HashMap<(String, PartitionId), PartitionState> = HashMap::new();

        offsets.insert(("t".into(), 0), 50);
        partition_state.insert(("t".into(), 0), ps_with_hw(100));

        // Simulate clear_partition_state
        offsets.clear();
        partition_state.clear();

        let (total_lag, _) = compute_aggregate_lag(&offsets, &partition_state);
        assert_eq!(total_lag, 0);
    }

    /// Revoking a partition must clear **every** per-partition cache
    /// (high watermark, log start offset, preferred replica, offset-retry
    /// backoff) atomically. Before the `PartitionState` consolidation this
    /// was "four separate `HashMap::remove` calls under four separate locks",
    /// which was the exact bug class the refactor eliminated. This test pins
    /// the invariant: a single `HashMap::remove` of a `PartitionState` value
    /// drops all four caches together, so no future field added to
    /// `PartitionState` can be accidentally skipped by the revocation path.
    #[test]
    fn test_partition_state_revocation_is_atomic() {
        let key = ("t".to_string(), 0_i32);
        let mut partition_state: HashMap<(String, PartitionId), PartitionState> = HashMap::new();
        partition_state.insert(
            key.clone(),
            PartitionState {
                high_watermark: Some(100),
                log_start_offset: Some(0),
                preferred_replica: Some((3_i32, Instant::now() + Duration::from_secs(60))),
                offset_retry_backoff: Some((Instant::now(), Duration::from_millis(100))),
            },
        );

        // Sanity: the entry has all four facets populated.
        let state = &partition_state[&key];
        assert!(state.high_watermark.is_some());
        assert!(state.log_start_offset.is_some());
        assert!(state.preferred_replica.is_some());
        assert!(state.offset_retry_backoff.is_some());

        // Revoke — a single remove wipes all four facets at once.
        partition_state.remove(&key);

        assert!(!partition_state.contains_key(&key));
    }

    // --- Fetch routing plan tests (KIP-392) ---

    /// Build a `PartitionState` map entry with only the preferred replica set.
    /// Used by the routing-plan tests below.
    fn ps_with_preferred(replica_id: crate::BrokerId, expiry: Instant) -> PartitionState {
        PartitionState {
            preferred_replica: Some((replica_id, expiry)),
            ..Default::default()
        }
    }

    #[test]
    fn test_routing_plan_uses_leader_when_no_preferred() {
        let keys = vec![("t".into(), 0), ("t".into(), 1)];

        let leaders = HashMap::from([(("t".into(), 0), 1), (("t".into(), 1), 2)]);

        let plan = build_fetch_routing_plan(keys, &HashMap::new(), &leaders, Instant::now());

        assert!(plan.expired_preferred.is_empty());
        assert_eq!(plan.partitions_by_broker[&1], vec![("t".into(), 0)]);
        assert_eq!(plan.partitions_by_broker[&2], vec![("t".into(), 1)]);
    }

    #[test]
    fn test_routing_plan_routes_to_preferred_replica() {
        let keys = vec![("t".into(), 0)];

        let leaders = HashMap::from([(("t".into(), 0), 1)]);
        let partition_state = HashMap::from([(
            ("t".into(), 0),
            ps_with_preferred(3_i32, Instant::now() + Duration::from_secs(60)),
        )]);

        let plan = build_fetch_routing_plan(keys, &partition_state, &leaders, Instant::now());

        assert!(plan.expired_preferred.is_empty());
        // Should route to preferred replica (broker 3), not leader (broker 1)
        assert_eq!(plan.partitions_by_broker.len(), 1);
        assert_eq!(plan.partitions_by_broker[&3], vec![("t".into(), 0)]);
    }

    #[test]
    fn test_routing_plan_falls_back_on_expired_preferred() {
        let keys = vec![("t".into(), 0)];

        let leaders = HashMap::from([(("t".into(), 0), 1)]);
        // Preferred replica that expired 10 seconds ago
        let partition_state = HashMap::from([(
            ("t".into(), 0),
            ps_with_preferred(3_i32, Instant::now() - Duration::from_secs(10)),
        )]);

        let plan = build_fetch_routing_plan(keys, &partition_state, &leaders, Instant::now());

        // Should fall back to leader (broker 1)
        assert_eq!(plan.partitions_by_broker[&1], vec![("t".into(), 0)]);
        // Should report the expired entry for cleanup
        assert_eq!(plan.expired_preferred, vec![("t".into(), 0)]);
    }

    #[test]
    fn test_routing_plan_skips_partitions_without_leader() {
        // Only partition 0 has a leader; partition 1 has neither leader nor
        // preferred replica and should be skipped.
        let keys = vec![("t".into(), 0), ("t".into(), 1)];

        // Only partition 0 has a leader
        let leaders = HashMap::from([(("t".into(), 0), 1)]);

        let plan = build_fetch_routing_plan(keys, &HashMap::new(), &leaders, Instant::now());

        let all: Vec<_> = plan.partitions_by_broker.values().flatten().collect();
        assert_eq!(all.len(), 1);
        assert_eq!(*all[0], ("t".into(), 0));
        assert_eq!(plan.skipped, vec![("t".into(), 1)]);
    }

    #[test]
    fn test_routing_plan_all_partitions_skipped() {
        // No leaders and no preferred replicas → every partition is skipped,
        // plan is empty.
        let keys = vec![("t".into(), 0), ("t".into(), 1)];

        let plan = build_fetch_routing_plan(keys, &HashMap::new(), &HashMap::new(), Instant::now());

        assert!(plan.partitions_by_broker.is_empty());
        assert!(plan.expired_preferred.is_empty());
        assert_eq!(plan.skipped.len(), 2);
    }

    #[test]
    fn test_routing_plan_mixed_preferred_and_leader() {
        let keys = vec![("t".into(), 0), ("t".into(), 1), ("t".into(), 2)];

        let leaders = HashMap::from([
            (("t".into(), 0), 1),
            (("t".into(), 1), 1),
            (("t".into(), 2), 2),
        ]);
        let future = Instant::now() + Duration::from_secs(300);
        let partition_state = HashMap::from([
            // p0 has a valid preferred replica
            (("t".into(), 0), ps_with_preferred(3_i32, future)),
            // p1 has an expired preferred replica
            (
                ("t".into(), 1),
                ps_with_preferred(3_i32, Instant::now() - Duration::from_secs(1)),
            ),
            // p2 has no preferred replica
        ]);

        let plan = build_fetch_routing_plan(keys, &partition_state, &leaders, Instant::now());

        // p0 → broker 3 (preferred), p1 → broker 1 (leader, expired), p2 → broker 2 (leader)
        assert!(plan.partitions_by_broker[&3].contains(&("t".into(), 0)));
        assert!(plan.partitions_by_broker[&1].contains(&("t".into(), 1)));
        assert!(plan.partitions_by_broker[&2].contains(&("t".into(), 2)));
        assert_eq!(plan.expired_preferred, vec![("t".into(), 1)]);
    }

    #[test]
    fn test_consumer_is_send_sync() {
        fn assert_send_sync<T: Send + Sync>() {}
        assert_send_sync::<Consumer>();
    }

    #[test]
    fn test_consumer_stream_is_send() {
        fn assert_send<T: Send>() {}
        // ConsumerStream must be Send so it can be used across .await in
        // spawned tasks (e.g., tokio::spawn with an Arc<Consumer>).
        assert_send::<ConsumerStream<'_>>();
    }

    /// The flat `&[(&str, PartitionId)]` input to `offsets_for_times` is
    /// grouped by topic via `group_topic_partitions`. Verify that the grouping
    /// preserves all pairs, deduplicates topic keys, and keeps partitions in
    /// insertion order.
    #[test]
    fn test_offsets_for_times_grouping() {
        let partitions: &[(&str, PartitionId)] =
            &[("topic1", 0), ("topic1", 2), ("topic2", 1), ("topic1", 5)];

        let grouped = group_topic_partitions(partitions);

        assert_eq!(grouped.len(), 2);
        assert_eq!(grouped["topic1"], vec![0, 2, 5]);
        assert_eq!(grouped["topic2"], vec![1]);
    }
}