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
//! Broker connection implementation.
//!
//! This module provides connection handling with support for:
//! - **Request priority**: High-priority requests (heartbeats, metadata) are processed
//!   before normal-priority requests to prevent consumer group ejection during backpressure.
//! - **Multi-connection bundles**: Multiple connections per broker for extreme high-throughput.
//! - **TLS/SSL encryption**: Automatic TLS upgrade when configured.
//! - **SASL authentication**: PLAIN, SCRAM-SHA-256/512, AWS MSK IAM handshake on connect.

use std::collections::HashMap;
use std::sync::Arc;
use std::sync::atomic::{AtomicI64, AtomicU64, Ordering};
use std::time::{Duration, Instant, SystemTime};

use arc_swap::ArcSwap;
use bytes::{Bytes, BytesMut};
use tokio::io::{AsyncRead, AsyncReadExt, AsyncWrite, AsyncWriteExt};
#[cfg(feature = "socks5")]
use tokio::net::TcpSocket;
use tokio::sync::{mpsc, oneshot};
use tokio::time::timeout;
#[cfg(feature = "socks5")]
use tokio::time::timeout_at;
use tokio_rustls::TlsConnector;
use tokio_util::time::{DelayQueue, delay_queue};
use tracing::{debug, error, info, trace, warn};

use crate::CorrelationId;
use crate::auth::msk_iam::MAX_SIGV4_CLOCK_SKEW_SECS;
use crate::auth::tls::build_tls_connector;
use crate::auth::{
    AuthConfig, ChannelBinding, SaslMechanism, SecurityProtocol, connect_tls,
    extract_tls_server_end_point,
};
use crate::error::{KrafkaError, Result};
use crate::metrics::ConnectionMetrics;

/// Maximum number of consecutive high-priority commands the event loop will
/// process before forcing one normal-priority drain attempt.
const MAX_HIGH_PRIORITY_BYPASSES_PER_ROUND: usize = 4;

/// SOCKS5 proxy configuration for connecting to brokers through a proxy.
///
/// When set on a [`ConnectionConfig`], all TCP connections to Kafka brokers
/// are tunneled through the specified SOCKS5 proxy. The proxy performs DNS
/// resolution of the broker address, which is essential for VPN/bastion
/// setups where broker hostnames are not resolvable from the client network.
///
/// TLS and SASL authentication are layered on top of the proxied connection
/// transparently — no additional configuration is needed.
///
/// # Example
///
/// ```rust,ignore
/// use krafka::network::{ConnectionConfig, ProxyConfig};
///
/// let proxy = ProxyConfig::new("socks5-proxy:1080");
/// let config = ConnectionConfig::builder()
///     .proxy(proxy)
///     .build();
/// ```
#[cfg(feature = "socks5")]
#[derive(Clone)]
pub struct ProxyConfig {
    /// SOCKS5 proxy address (`host:port`).
    address: String,
    /// Optional proxy authentication credentials.
    credentials: Option<ProxyCredentials>,
}

#[cfg(feature = "socks5")]
impl ProxyConfig {
    /// Create a new SOCKS5 proxy configuration.
    pub fn new(address: impl Into<String>) -> Self {
        Self {
            address: address.into(),
            credentials: None,
        }
    }

    /// Create a SOCKS5 proxy configuration with username/password authentication.
    pub fn with_credentials(
        address: impl Into<String>,
        username: impl Into<String>,
        password: impl Into<String>,
    ) -> Self {
        Self {
            address: address.into(),
            credentials: Some(ProxyCredentials {
                username: username.into(),
                password: password.into(),
            }),
        }
    }

    /// Returns the proxy address.
    #[inline]
    pub fn address(&self) -> &str {
        &self.address
    }

    /// Returns the proxy credentials, if set.
    #[inline]
    pub fn credentials(&self) -> Option<&ProxyCredentials> {
        self.credentials.as_ref()
    }
}

#[cfg(feature = "socks5")]
impl std::fmt::Debug for ProxyConfig {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("ProxyConfig")
            .field("address", &self.address)
            .field(
                "credentials",
                if self.credentials.is_some() {
                    &"[REDACTED]"
                } else {
                    &"None"
                },
            )
            .finish()
    }
}

/// Credentials for SOCKS5 proxy authentication.
///
/// Implements [`Zeroize`](zeroize::Zeroize) to ensure the password is scrubbed
/// from memory on drop.
#[cfg(feature = "socks5")]
#[derive(Clone, zeroize::Zeroize, zeroize::ZeroizeOnDrop)]
pub struct ProxyCredentials {
    /// Proxy username.
    username: String,
    /// Proxy password.
    password: String,
}

#[cfg(feature = "socks5")]
impl ProxyCredentials {
    /// Returns the proxy username.
    #[inline]
    pub fn username(&self) -> &str {
        &self.username
    }

    /// Returns the proxy password.
    #[inline]
    pub fn password(&self) -> &str {
        &self.password
    }
}

#[cfg(feature = "socks5")]
impl std::fmt::Debug for ProxyCredentials {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("ProxyCredentials")
            .field("username", &self.username)
            .field("password", &"[REDACTED]")
            .finish()
    }
}
use crate::protocol::{
    ApiKey, ApiVersionRange, ApiVersionsRequest, ApiVersionsResponse, Decoder, Encoder,
    RequestHeader, ResponseHeader, SaslAuthenticateRequest, SaslAuthenticateResponse,
    SaslHandshakeRequest, SaslHandshakeResponse,
};
use crate::util::{CorrelationIdGenerator, NO_RESPONSE_CORRELATION_ID, extract_sni_hostname};

use super::secure::{ChallengeResponse, SaslAuthenticator};

/// Request priority level.
///
/// High-priority requests are processed before normal-priority requests,
/// which is critical for preventing consumer group ejection during backpressure.
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
pub enum RequestPriority {
    /// High priority for time-critical requests like heartbeats and metadata.
    ///
    /// These requests are processed first to prevent consumer group ejection
    /// during periods of high throughput or backpressure.
    High,
    /// Normal priority for data requests like produce and fetch.
    Normal,
}

impl RequestPriority {
    /// Determine the priority for an API key.
    ///
    /// Time-sensitive coordination requests get high priority.
    #[inline]
    pub fn for_api_key(api_key: ApiKey) -> Self {
        match api_key {
            // Group coordination - must not be delayed
            ApiKey::Heartbeat | ApiKey::ConsumerGroupHeartbeat => Self::High,
            // Metadata refresh - critical for proper routing
            ApiKey::Metadata => Self::High,
            // Coordinator discovery - needed for heartbeats
            ApiKey::FindCoordinator => Self::High,
            // Leader discovery
            ApiKey::LeaderAndIsr => Self::High,
            // API version negotiation
            ApiKey::ApiVersions => Self::High,
            // Everything else is normal priority
            _ => Self::Normal,
        }
    }
}

/// Configuration for broker connections.
///
/// Use [`ConnectionConfig::builder()`] or [`Default::default()`] to construct.
/// Call [`init_tls()`](ConnectionConfig::init_tls) after building when TLS is
/// configured to pre-build and cache the TLS connector, avoiding repeated disk
/// I/O for certificates on every reconnection.
#[derive(Clone)]
pub struct ConnectionConfig {
    /// Connection timeout.
    pub(crate) connect_timeout: Duration,
    /// Request timeout.
    pub(crate) request_timeout: Duration,
    /// Socket send buffer size.
    pub(crate) send_buffer_size: Option<usize>,
    /// Socket receive buffer size.
    pub(crate) recv_buffer_size: Option<usize>,
    /// TCP nodelay.
    pub(crate) nodelay: bool,
    /// Client ID.
    pub(crate) client_id: String,
    /// Number of connections per broker for high-throughput scenarios.
    ///
    /// Default is 1. For extreme high-throughput (>100k msg/s per broker),
    /// consider 2-4 connections to parallelize I/O operations.
    pub(crate) connections_per_broker: usize,
    /// High-priority channel capacity for heartbeats and metadata requests.
    ///
    /// This should be small since high-priority requests should be rare.
    pub(crate) high_priority_channel_capacity: usize,
    /// Normal-priority channel capacity for produce and fetch requests.
    pub(crate) normal_priority_channel_capacity: usize,
    /// Maximum response size in bytes.
    ///
    /// Responses larger than this are rejected to prevent excessive memory allocation.
    /// Default: 100 MB (matching `MAX_MESSAGE_SIZE`).
    pub(crate) max_response_size: usize,
    /// Maximum number of in-flight requests per connection.
    ///
    /// When this limit is reached, new requests are rejected with an error
    /// until existing requests complete or time out. This prevents unbounded
    /// memory growth from a stalled broker or runaway producer.
    ///
    /// Default: 256. Use 5 for idempotent producers to match Kafka's
    /// `max.in.flight.requests.per.connection` guarantee.
    pub(crate) max_in_flight_requests: usize,
    /// Authentication configuration (optional).
    ///
    /// When set, the connection will perform TLS upgrade and/or SASL
    /// authentication handshake during establishment.
    pub(crate) auth: Option<AuthConfig>,
    /// Cached TLS connector built from [`AuthConfig::tls_config`].
    ///
    /// Populated by [`init_tls()`](ConnectionConfig::init_tls). When present,
    /// connections reuse this connector instead of reading certificate files
    /// from disk on every connection attempt.
    ///
    /// Wrapped in `Arc<ArcSwap<…>>` so that all clones of this config share
    /// the same connector and [`refresh_tls()`](ConnectionConfig::refresh_tls)
    /// atomically updates it for every future connection.
    pub(crate) tls_connector: Arc<ArcSwap<Option<TlsConnector>>>,
    /// TCP keepalive interval.
    ///
    /// When set, enables TCP keepalive on all broker connections with the
    /// given interval. This prevents idle connections from being silently
    /// dropped by firewalls and load balancers.
    pub(crate) tcp_keepalive: Option<Duration>,
    /// Happy Eyeballs connection attempt delay (RFC 8305 §5).
    ///
    /// The delay between staggered connection attempts when racing multiple
    /// addresses. Clamped to 100 ms – 2 s at connect time (RFC 8305 §5).
    /// Default: 250 ms.
    pub(crate) connection_attempt_delay: Duration,
    /// Shared clock offset for MSK IAM signing (seconds).
    ///
    /// When SASL/MSK_IAM authentication fails with a signature-mismatch
    /// that looks like clock skew, the connection layer stores the estimated
    /// offset here.  Subsequent reconnection attempts apply this offset to
    /// `SystemTime::now()` so the SigV4 timestamp matches the broker's
    /// clock.  Default: 0 (no adjustment).
    pub(crate) msk_iam_clock_offset_secs: Arc<AtomicI64>,
    /// Shared connection metrics recorded by broker connections created from this config.
    pub(crate) connection_metrics: Arc<ConnectionMetrics>,
    /// SOCKS5 proxy configuration (optional).
    ///
    /// When set, all connections are tunneled through the proxy.
    /// Requires the `socks5` feature.
    #[cfg(feature = "socks5")]
    pub(crate) proxy: Option<ProxyConfig>,
}

impl std::fmt::Debug for ConnectionConfig {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        let mut s = f.debug_struct("ConnectionConfig");
        s.field("connect_timeout", &self.connect_timeout)
            .field("request_timeout", &self.request_timeout)
            .field("send_buffer_size", &self.send_buffer_size)
            .field("recv_buffer_size", &self.recv_buffer_size)
            .field("nodelay", &self.nodelay)
            .field("client_id", &self.client_id)
            .field("connections_per_broker", &self.connections_per_broker)
            .field(
                "high_priority_channel_capacity",
                &self.high_priority_channel_capacity,
            )
            .field(
                "normal_priority_channel_capacity",
                &self.normal_priority_channel_capacity,
            )
            .field("max_response_size", &self.max_response_size)
            .field("max_in_flight_requests", &self.max_in_flight_requests)
            .field("auth", &self.auth)
            .field("tls_connector", &self.tls_connector.load().is_some())
            .field("tcp_keepalive", &self.tcp_keepalive)
            .field("connection_attempt_delay", &self.connection_attempt_delay)
            .field(
                "msk_iam_clock_offset_secs",
                &self.msk_iam_clock_offset_secs.load(Ordering::Relaxed),
            );
        #[cfg(feature = "socks5")]
        s.field("proxy", &self.proxy);
        s.finish()
    }
}

impl Default for ConnectionConfig {
    fn default() -> Self {
        Self {
            connect_timeout: Duration::from_secs(10),
            request_timeout: Duration::from_secs(30),
            send_buffer_size: None,
            recv_buffer_size: None,
            nodelay: true,
            client_id: "krafka".to_string(),
            connections_per_broker: 1,
            high_priority_channel_capacity: 64,
            normal_priority_channel_capacity: 256,
            max_response_size: crate::protocol::MAX_MESSAGE_SIZE,
            max_in_flight_requests: 256,
            auth: None,
            tls_connector: Arc::new(ArcSwap::new(Arc::new(None))),
            tcp_keepalive: Some(Duration::from_secs(60)),
            connection_attempt_delay: Duration::from_millis(250),
            msk_iam_clock_offset_secs: Arc::new(AtomicI64::new(0)),
            connection_metrics: Arc::new(ConnectionMetrics::default()),
            #[cfg(feature = "socks5")]
            proxy: None,
        }
    }
}

impl ConnectionConfig {
    /// Create a new connection config builder.
    pub fn builder() -> ConnectionConfigBuilder {
        ConnectionConfigBuilder::default()
    }

    /// Pre-build and cache the TLS connector from the configured certificates.
    ///
    /// When TLS is configured, this reads the certificate and key files once
    /// (via `spawn_blocking`) and stores the resulting [`TlsConnector`] for
    /// reuse across all connections and reconnections. Without this call,
    /// every connection attempt re-reads the files from disk.
    ///
    /// This is a no-op when no TLS configuration is present.
    ///
    /// # Errors
    ///
    /// Returns an error if certificate or key files cannot be read or parsed.
    pub async fn init_tls(&mut self) -> Result<()> {
        if let Some(ref auth) = self.auth
            && let Some(ref tls_config) = auth.tls_config
        {
            let connector = build_tls_connector(tls_config).await?;
            self.tls_connector.store(Arc::new(Some(connector)));
        }
        Ok(())
    }

    /// Re-read certificate files from disk and atomically replace the cached
    /// TLS connector.
    ///
    /// All future connections (including reconnections from the pool) will use
    /// the new certificates. Existing TLS sessions are unaffected — they
    /// continue using the connector that was active at handshake time.
    ///
    /// Call this after rotating certificates on disk, or on a periodic timer
    /// (e.g. once per hour) to pick up renewed certificates without a client
    /// restart.
    ///
    /// This is a no-op when no TLS configuration is present.
    ///
    /// # Errors
    ///
    /// Returns an error if the new certificate or key files cannot be read or
    /// parsed. The existing (old) connector remains active on failure.
    pub async fn refresh_tls(&self) -> Result<()> {
        if let Some(ref auth) = self.auth
            && let Some(ref tls_config) = auth.tls_config
        {
            let connector = build_tls_connector(tls_config).await?;
            self.tls_connector.store(Arc::new(Some(connector)));
            info!("TLS connector refreshed from disk");
        }
        Ok(())
    }

    /// Returns the connection timeout.
    #[inline]
    pub fn connect_timeout(&self) -> Duration {
        self.connect_timeout
    }

    /// Returns the request timeout.
    #[inline]
    pub fn request_timeout(&self) -> Duration {
        self.request_timeout
    }

    /// Returns the socket send buffer size, if set.
    #[inline]
    pub fn send_buffer_size(&self) -> Option<usize> {
        self.send_buffer_size
    }

    /// Returns the socket receive buffer size, if set.
    #[inline]
    pub fn recv_buffer_size(&self) -> Option<usize> {
        self.recv_buffer_size
    }

    /// Returns whether TCP nodelay is enabled.
    #[inline]
    pub fn nodelay(&self) -> bool {
        self.nodelay
    }

    /// Returns the client ID.
    #[inline]
    pub fn client_id(&self) -> &str {
        &self.client_id
    }

    /// Returns the number of connections per broker.
    #[inline]
    pub fn connections_per_broker(&self) -> usize {
        self.connections_per_broker
    }

    /// Returns the high-priority channel capacity.
    #[inline]
    pub fn high_priority_channel_capacity(&self) -> usize {
        self.high_priority_channel_capacity
    }

    /// Returns the normal-priority channel capacity.
    #[inline]
    pub fn normal_priority_channel_capacity(&self) -> usize {
        self.normal_priority_channel_capacity
    }

    /// Returns the maximum response size in bytes.
    #[inline]
    pub fn max_response_size(&self) -> usize {
        self.max_response_size
    }

    /// Returns the maximum number of in-flight requests per connection.
    #[inline]
    pub fn max_in_flight_requests(&self) -> usize {
        self.max_in_flight_requests
    }

    /// Returns the authentication configuration, if set.
    #[inline]
    pub fn auth(&self) -> Option<&AuthConfig> {
        self.auth.as_ref()
    }

    /// Returns the Happy Eyeballs connection attempt delay.
    #[inline]
    pub fn connection_attempt_delay(&self) -> Duration {
        self.connection_attempt_delay
    }

    /// Returns the shared connection metrics handle.
    #[inline]
    pub fn connection_metrics(&self) -> Arc<ConnectionMetrics> {
        self.connection_metrics.clone()
    }

    /// Returns the SOCKS5 proxy configuration, if set.
    ///
    /// Requires the `socks5` feature.
    #[cfg(feature = "socks5")]
    #[inline]
    pub fn proxy(&self) -> Option<&ProxyConfig> {
        self.proxy.as_ref()
    }
}

/// Builder for ConnectionConfig.
#[must_use = "builders do nothing until .build() is called"]
#[derive(Debug, Default)]
pub struct ConnectionConfigBuilder(ConnectionConfig);

impl ConnectionConfigBuilder {
    /// Set the connect timeout.
    pub fn connect_timeout(mut self, timeout: Duration) -> Self {
        self.0.connect_timeout = timeout;
        self
    }

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

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

    /// Set TCP nodelay.
    pub fn nodelay(mut self, nodelay: bool) -> Self {
        self.0.nodelay = nodelay;
        self
    }

    /// Set the number of connections per broker.
    ///
    /// For extreme high-throughput (>100k msg/s per broker), use 2-4 connections.
    /// Default is 1.
    pub fn connections_per_broker(mut self, count: usize) -> Self {
        self.0.connections_per_broker = count.max(1);
        self
    }

    /// Set the high-priority channel capacity.
    ///
    /// This channel is used for heartbeats and metadata requests.
    pub fn high_priority_channel_capacity(mut self, capacity: usize) -> Self {
        self.0.high_priority_channel_capacity = capacity.max(16);
        self
    }

    /// Set the normal-priority channel capacity.
    ///
    /// This channel is used for produce and fetch requests.
    pub fn normal_priority_channel_capacity(mut self, capacity: usize) -> Self {
        self.0.normal_priority_channel_capacity = capacity.max(64);
        self
    }

    /// Set the maximum response size in bytes.
    ///
    /// Responses exceeding this limit are rejected. Default: 100 MB.
    pub fn max_response_size(mut self, size: usize) -> Self {
        self.0.max_response_size = size.max(1024); // at least 1 KB
        self
    }

    /// Set the maximum number of in-flight requests per connection.
    ///
    /// Limits the number of requests waiting for a response on a single
    /// connection. Default: 256. Use 5 for idempotent producers.
    pub fn max_in_flight_requests(mut self, max: usize) -> Self {
        self.0.max_in_flight_requests = max.max(1);
        self
    }

    /// Set authentication configuration.
    ///
    /// When set, the connection will perform TLS upgrade and/or SASL
    /// authentication handshake during establishment.
    pub fn auth(mut self, auth: AuthConfig) -> Self {
        self.0.auth = Some(auth);
        self
    }

    /// Set the TCP keepalive interval.
    ///
    /// When set, enables TCP keepalive on all broker connections.
    /// Pass `None` to disable keepalive. Default: 60 seconds.
    pub fn tcp_keepalive(mut self, interval: Option<Duration>) -> Self {
        self.0.tcp_keepalive = interval;
        self
    }

    /// Set the Happy Eyeballs connection attempt delay (RFC 8305 §5).
    ///
    /// This controls the stagger interval between parallel connection
    /// attempts. Clamped to 100 ms – 2 s at connect time.
    /// Default: 250 ms.
    pub fn connection_attempt_delay(mut self, delay: Duration) -> Self {
        self.0.connection_attempt_delay = delay;
        self
    }

    /// Set the shared connection metrics handle.
    pub fn connection_metrics(mut self, metrics: Arc<ConnectionMetrics>) -> Self {
        self.0.connection_metrics = metrics;
        self
    }

    /// Set SOCKS5 proxy configuration.
    ///
    /// When set, all connections are tunneled through the specified SOCKS5
    /// proxy. The proxy performs DNS resolution, which is essential for
    /// VPN/bastion setups where broker hostnames are not directly resolvable.
    ///
    /// Requires the `socks5` feature.
    #[cfg(feature = "socks5")]
    pub fn proxy(mut self, proxy: ProxyConfig) -> Self {
        self.0.proxy = Some(proxy);
        self
    }

    /// Build the config.
    pub fn build(self) -> ConnectionConfig {
        self.0
    }
}

/// A pending request waiting for a response.
struct PendingRequest {
    response_tx: oneshot::Sender<Result<Bytes>>,
    api_key: ApiKey,
    api_version: i16,
}

/// Command sent to the connection task.
enum ConnectionCommand {
    /// Send a request and wait for response.
    Request {
        data: Bytes,
        correlation_id: CorrelationId,
        api_key: ApiKey,
        api_version: i16,
        response_tx: oneshot::Sender<Result<Bytes>>,
    },
    /// Send data without registering a pending response (fire-and-forget).
    ///
    /// Used for `acks=0` produce requests where the broker sends no response.
    /// The data is written to the wire without inserting into the pending map.
    FireAndForget { data: Bytes },
    /// Close the connection.
    Close,
}

/// A connection to a Kafka broker.
///
/// This connection supports priority-based request handling:
/// - High-priority requests (heartbeats, metadata) are processed first
/// - Normal-priority requests (produce, fetch) are processed when no high-priority pending
pub struct BrokerConnection {
    /// Broker address.
    address: String,
    /// Connection config.
    config: ConnectionConfig,
    /// Correlation ID generator.
    correlation_id_gen: Arc<CorrelationIdGenerator>,
    /// High-priority command sender (heartbeats, metadata).
    high_priority_tx: mpsc::Sender<ConnectionCommand>,
    /// Normal-priority command sender (produce, fetch).
    normal_priority_tx: mpsc::Sender<ConnectionCommand>,
    /// API versions supported by the broker.
    api_versions: Arc<parking_lot::Mutex<HashMap<ApiKey, ApiVersionRange>>>,
    /// Whether the connection is alive.
    alive: Arc<std::sync::atomic::AtomicBool>,
    /// When the SASL session expires (KIP-368).
    ///
    /// `None` when authentication is not used or the broker reported a
    /// session lifetime of zero (no expiry).
    session_expiry: Option<Instant>,
    /// Statistics for monitoring.
    stats: Arc<ConnectionStats>,
    /// KIP-219: deadline until which normal-priority requests should be
    /// delayed because the broker signalled quota throttling.
    throttle_until: Arc<parking_lot::Mutex<Instant>>,
    /// Instant anchor used with `last_used_nanos` to compute idle duration
    /// without locking. Set once at connect time; never mutated.
    created_at: Instant,
    /// Monotonic-nanoseconds since `created_at` of the last submitted
    /// request. Updated on every `send_request_with_priority` and
    /// `send_fire_and_forget` entry. Read by `ConnectionPool::evict_idle`
    /// to decide whether a connection has been idle past
    /// `connections.max.idle.ms`. An `AtomicU64` rather than a lock keeps
    /// the network hot path free of contention; the only race is two
    /// concurrent senders both storing a "recent" value, which is fine
    /// because either observer still reads "recently used".
    last_used_nanos: AtomicU64,
}

/// Connection statistics for monitoring.
#[derive(Debug, Default)]
#[non_exhaustive]
pub struct ConnectionStats {
    /// Total high-priority requests sent.
    pub high_priority_requests: AtomicU64,
    /// Total normal-priority requests sent.
    pub normal_priority_requests: AtomicU64,
    /// High-priority requests that bypassed the queue (processed immediately).
    pub high_priority_bypasses: AtomicU64,
    /// Number of times the loop yielded to normal-priority work after hitting
    /// the high-priority bypass budget.
    pub high_priority_bypass_yields: AtomicU64,
}

impl ConnectionStats {
    /// Get the total high-priority requests sent.
    #[inline]
    pub fn high_priority_count(&self) -> u64 {
        self.high_priority_requests.load(Ordering::Relaxed)
    }

    /// Get the total normal-priority requests sent.
    #[inline]
    pub fn normal_priority_count(&self) -> u64 {
        self.normal_priority_requests.load(Ordering::Relaxed)
    }

    /// Get the number of high-priority bypasses.
    #[inline]
    pub fn bypass_count(&self) -> u64 {
        self.high_priority_bypasses.load(Ordering::Relaxed)
    }

    /// Get the number of fairness yields after exhausting the bypass budget.
    #[inline]
    pub fn bypass_yield_count(&self) -> u64 {
        self.high_priority_bypass_yields.load(Ordering::Relaxed)
    }
}

impl BrokerConnection {
    /// Connect to a broker.
    ///
    /// When `config.auth` is set, the connection will:
    /// 1. Establish a TCP connection
    /// 2. Upgrade to TLS if required by the security protocol
    /// 3. Perform SASL authentication handshake if required
    /// 4. Fetch API versions
    pub async fn connect(address: &str, config: ConnectionConfig) -> Result<Self> {
        // Establish TCP stream — either direct or through a SOCKS5 proxy.
        let stream = Self::establish_tcp(address, &config).await?;

        stream.set_nodelay(config.nodelay)?;

        debug!("Connected to broker at {address}");

        // Create priority channels
        let (high_priority_tx, high_priority_rx) =
            mpsc::channel(config.high_priority_channel_capacity);
        let (normal_priority_tx, normal_priority_rx) =
            mpsc::channel(config.normal_priority_channel_capacity);

        let alive = Arc::new(std::sync::atomic::AtomicBool::new(true));
        let alive_clone = alive.clone();
        let stats = Arc::new(ConnectionStats::default());
        let stats_clone = stats.clone();

        let mut connection = Self {
            address: address.to_string(),
            config: config.clone(),
            correlation_id_gen: Arc::new(CorrelationIdGenerator::new()),
            high_priority_tx,
            normal_priority_tx,
            api_versions: Arc::new(parking_lot::Mutex::new(HashMap::new())),
            alive,
            session_expiry: None,
            stats,
            throttle_until: Arc::new(parking_lot::Mutex::new(Instant::now())),
            created_at: Instant::now(),
            last_used_nanos: AtomicU64::new(0),
        };

        let request_timeout = config.request_timeout;

        // Determine auth requirements and dispatch to the appropriate path.
        // Using `filter` means `auth` is already in scope — no secondary
        // unreachable guard needed to re-establish the invariant.
        if let Some(auth) = config.auth.as_ref().filter(|a| a.requires_tls()) {
            // TLS path: upgrade stream then optionally do SASL
            let tls_config = auth
                .tls_config
                .as_ref()
                .ok_or_else(|| KrafkaError::config("TLS required but no TLS config provided"))?;

            // Use cached TLS connector or build one from config. Calling
            // `init_tls()` before first use avoids this fallback and the
            // repeated disk I/O it entails.
            let connector = match &**config.tls_connector.load() {
                Some(c) => c.clone(),
                None => build_tls_connector(tls_config).await?,
            };

            // Extract hostname (without port) for TLS SNI.
            // Handle IPv6 bracket notation like [::1]:9092.
            let hostname = extract_sni_hostname(address)?;
            let tls_stream = connect_tls(
                stream,
                hostname,
                tls_config.sni_hostname.as_deref(),
                &connector,
            )
            .await?;

            info!("TLS handshake completed for {address}");

            if auth.requires_sasl() {
                // TLS + SASL: authenticate on the TLS stream, then run event loop
                let mut tls_stream = tls_stream;

                // Extract tls-server-end-point channel binding data (RFC 5929 §4.1)
                // before the stream is consumed. This binds the SCRAM exchange to
                // this specific TLS session.
                let channel_binding = extract_tls_server_end_point(&tls_stream)
                    .map(ChannelBinding::TlsServerEndPoint)
                    .unwrap_or(ChannelBinding::None);

                let session_lifetime_ms = Self::perform_sasl_handshake(
                    &mut tls_stream,
                    auth,
                    address,
                    &config.client_id,
                    config.max_response_size,
                    request_timeout,
                    channel_binding,
                    &config.msk_iam_clock_offset_secs,
                )
                .await?;

                connection.session_expiry =
                    Self::effective_session_expiry(session_lifetime_ms, auth);

                // Spawn the connection task with TLS stream
                let (reader, writer) = tokio::io::split(tls_stream);
                let loop_address = address.to_string();
                let loop_metrics = config.connection_metrics.clone();
                let close_metrics = loop_metrics.clone();
                let max_response_size = config.max_response_size;
                let max_in_flight_requests = config.max_in_flight_requests;
                config.connection_metrics.record_connect();
                tokio::spawn(async move {
                    if let Err(e) = Self::run_connection_loop(
                        loop_address,
                        reader,
                        writer,
                        high_priority_rx,
                        normal_priority_rx,
                        request_timeout,
                        stats_clone,
                        loop_metrics,
                        max_response_size,
                        max_in_flight_requests,
                    )
                    .await
                    {
                        close_metrics.record_error();
                        error!("Connection error: {e}");
                    }
                    close_metrics.record_close();
                    alive_clone.store(false, std::sync::atomic::Ordering::SeqCst);
                });
            } else {
                // TLS only, no SASL
                let (reader, writer) = tokio::io::split(tls_stream);
                let loop_address = address.to_string();
                let loop_metrics = config.connection_metrics.clone();
                let close_metrics = loop_metrics.clone();
                let max_response_size = config.max_response_size;
                let max_in_flight_requests = config.max_in_flight_requests;
                config.connection_metrics.record_connect();
                tokio::spawn(async move {
                    if let Err(e) = Self::run_connection_loop(
                        loop_address,
                        reader,
                        writer,
                        high_priority_rx,
                        normal_priority_rx,
                        request_timeout,
                        stats_clone,
                        loop_metrics,
                        max_response_size,
                        max_in_flight_requests,
                    )
                    .await
                    {
                        close_metrics.record_error();
                        error!("Connection error: {e}");
                    }
                    close_metrics.record_close();
                    alive_clone.store(false, std::sync::atomic::Ordering::SeqCst);
                });
            }
        } else if let Some(auth) = config.auth.as_ref().filter(|a| a.requires_sasl()) {
            // SASL without TLS
            let mut stream = stream;
            let session_lifetime_ms = Self::perform_sasl_handshake(
                &mut stream,
                auth,
                address,
                &config.client_id,
                config.max_response_size,
                request_timeout,
                ChannelBinding::None,
                &config.msk_iam_clock_offset_secs,
            )
            .await?;

            connection.session_expiry = Self::effective_session_expiry(session_lifetime_ms, auth);

            let (reader, writer) = stream.into_split();
            let loop_address = address.to_string();
            let loop_metrics = config.connection_metrics.clone();
            let close_metrics = loop_metrics.clone();
            let max_response_size = config.max_response_size;
            let max_in_flight_requests = config.max_in_flight_requests;
            config.connection_metrics.record_connect();
            tokio::spawn(async move {
                if let Err(e) = Self::run_connection_loop(
                    loop_address,
                    reader,
                    writer,
                    high_priority_rx,
                    normal_priority_rx,
                    request_timeout,
                    stats_clone,
                    loop_metrics,
                    max_response_size,
                    max_in_flight_requests,
                )
                .await
                {
                    close_metrics.record_error();
                    error!("Connection error: {e}");
                }
                close_metrics.record_close();
                alive_clone.store(false, std::sync::atomic::Ordering::SeqCst);
            });
        } else {
            // Plain TCP — fast path (most common for local dev)
            let (reader, writer) = stream.into_split();
            let loop_address = address.to_string();
            let loop_metrics = config.connection_metrics.clone();
            let close_metrics = loop_metrics.clone();
            let max_response_size = config.max_response_size;
            let max_in_flight_requests = config.max_in_flight_requests;
            config.connection_metrics.record_connect();
            tokio::spawn(async move {
                if let Err(e) = Self::run_connection_loop(
                    loop_address,
                    reader,
                    writer,
                    high_priority_rx,
                    normal_priority_rx,
                    request_timeout,
                    stats_clone,
                    loop_metrics,
                    max_response_size,
                    max_in_flight_requests,
                )
                .await
                {
                    close_metrics.record_error();
                    error!("Connection error: {e}");
                }
                close_metrics.record_close();
                alive_clone.store(false, std::sync::atomic::Ordering::SeqCst);
            });
        }

        // Fetch API versions
        connection.fetch_api_versions().await?;

        Ok(connection)
    }

    /// Establish a TCP connection — direct or through a SOCKS5 proxy.
    async fn establish_tcp(
        address: &str,
        config: &ConnectionConfig,
    ) -> Result<tokio::net::TcpStream> {
        #[cfg(feature = "socks5")]
        if let Some(ref proxy) = config.proxy {
            return Self::connect_via_proxy(address, proxy, config).await;
        }

        Self::connect_direct(address, config).await
    }

    /// Direct TCP connection using Happy Eyeballs v2 (RFC 8305).
    ///
    /// Resolves DNS, interleaves IPv6/IPv4 addresses, and races staggered
    /// connection attempts — returning the first successful socket.
    async fn connect_direct(
        address: &str,
        config: &ConnectionConfig,
    ) -> Result<tokio::net::TcpStream> {
        super::happy_eyeballs::connect_happy_eyeballs(address, config).await
    }

    /// Connect through a SOCKS5 proxy.
    ///
    /// The proxy performs DNS resolution of the broker address, which is
    /// essential for VPN/bastion setups where broker hostnames are not
    /// resolvable from the client network.
    #[cfg(feature = "socks5")]
    async fn connect_via_proxy(
        address: &str,
        proxy: &ProxyConfig,
        config: &ConnectionConfig,
    ) -> Result<tokio::net::TcpStream> {
        use tokio_socks::tcp::Socks5Stream;

        debug!("Connecting to {address} via SOCKS5 proxy {}", proxy.address);

        // Use a single deadline for the entire proxy connect path (DNS + TCP + SOCKS5)
        // so the overall wall-clock time never exceeds connect_timeout.
        let deadline = tokio::time::Instant::now() + config.connect_timeout;

        // Resolve proxy address and create a socket with buffer sizes applied.
        let addrs: Vec<std::net::SocketAddr> =
            timeout_at(deadline, tokio::net::lookup_host(&proxy.address))
                .await
                .map_err(|_| KrafkaError::timeout("SOCKS5 proxy DNS resolution"))?
                .map_err(KrafkaError::network)?
                .collect();

        if addrs.is_empty() {
            return Err(KrafkaError::invalid_state(format!(
                "no addresses resolved for SOCKS5 proxy '{}'",
                proxy.address
            )));
        }

        // Try proxy addresses in resolver order.
        let proxy_addr = addrs[0];

        let socket = Self::create_socket(proxy_addr, config)?;

        // Connect to the proxy and perform the SOCKS5 handshake, bounded by
        // the remaining budget from the same deadline.
        let proxy_stream = timeout_at(deadline, async {
            let tcp = socket
                .connect(proxy_addr)
                .await
                .map_err(KrafkaError::network)?;

            // SOCKS5 handshake — pass the broker address as a string so the
            // proxy performs DNS resolution (remote resolution).
            let socks = if let Some(ref creds) = proxy.credentials {
                Socks5Stream::connect_with_password_and_socket(
                    tcp,
                    address,
                    creds.username(),
                    creds.password(),
                )
                .await
            } else {
                Socks5Stream::connect_with_socket(tcp, address).await
            }
            .map_err(|e| {
                KrafkaError::network(std::io::Error::other(format!("SOCKS5 proxy error: {e}")))
            })?;

            Ok::<_, KrafkaError>(socks.into_inner())
        })
        .await
        .map_err(|_| KrafkaError::timeout("SOCKS5 proxy connection"))??;

        info!(
            "SOCKS5 tunnel established to {address} via {}",
            proxy.address
        );

        Ok(proxy_stream)
    }

    /// Create a TCP socket for the given address with buffer sizes and keepalive applied.
    #[cfg(feature = "socks5")]
    fn create_socket(addr: std::net::SocketAddr, config: &ConnectionConfig) -> Result<TcpSocket> {
        super::happy_eyeballs::create_socket(addr, config)
    }

    /// Perform the SASL handshake and authentication on a stream.
    ///
    /// This sends:
    /// 1. SaslHandshake request to negotiate the mechanism
    /// 2. SaslAuthenticate request(s) for the actual authentication
    ///
    /// For multi-step mechanisms (SCRAM-SHA-*), the challenge-response
    /// loop is handled automatically.
    ///
    /// The `channel_binding` parameter is forwarded to the SCRAM client when
    /// the mechanism is SCRAM-SHA-*. Pass [`ChannelBinding::TlsServerEndPoint`]
    /// when the underlying transport is TLS, or [`ChannelBinding::None`] for
    /// plaintext SASL.
    ///
    /// Returns the session lifetime in milliseconds reported by the broker
    /// (KIP-368). A value of `0` means the broker does not enforce
    /// session expiry.
    #[allow(clippy::too_many_arguments)]
    async fn perform_sasl_handshake<S>(
        stream: &mut S,
        auth: &AuthConfig,
        address: &str,
        client_id: &str,
        max_response_size: usize,
        request_timeout: Duration,
        channel_binding: ChannelBinding,
        msk_iam_clock_offset_secs: &Arc<AtomicI64>,
    ) -> Result<i64>
    where
        S: AsyncRead + AsyncWrite + Unpin,
    {
        // For MSK IAM with a credential provider, resolve fresh credentials
        // before creating the authenticator.
        let resolved_msk_iam;
        let auth = if let Some(resolved) = timeout(request_timeout, auth.resolve_msk_iam_provider())
            .await
            .map_err(|_| KrafkaError::timeout("MSK IAM credential provider"))??
        {
            debug!("Resolved MSK IAM credentials from provider for {address}");
            resolved_msk_iam = resolved;
            &resolved_msk_iam
        } else {
            auth
        };

        // For OAUTHBEARER with a provider, resolve a fresh token before
        // creating the authenticator (which is synchronous).
        // Apply the request timeout so a hung provider cannot stall reconnect loops.
        let resolved_auth;
        let auth = if let Some(resolved) =
            timeout(request_timeout, auth.resolve_provider_to_token())
                .await
                .map_err(|_| KrafkaError::timeout("OAUTHBEARER token provider"))??
        {
            debug!("Resolved OAUTHBEARER token from provider for {address}");
            resolved_auth = resolved;
            &resolved_auth
        } else {
            auth
        };

        let mut authenticator = SaslAuthenticator::new(auth, channel_binding)
            .ok_or_else(|| KrafkaError::auth("Failed to create SASL authenticator"))?;

        // Warn about SASL PLAIN over cleartext — credentials sent unencrypted
        if auth.security_protocol == SecurityProtocol::SaslPlaintext
            && auth.sasl_mechanism == Some(SaslMechanism::Plain)
        {
            warn!(
                "SASL PLAIN credentials will be sent in cleartext to {}. \
                 Use SASL_SSL (sasl_plain_ssl) for production environments.",
                address
            );
        }

        // For MSK IAM, set the broker host (handles IPv6 brackets like [::1]:9092)
        let hostname = extract_sni_hostname(address)?;
        let clock_offset = msk_iam_clock_offset_secs.load(Ordering::Relaxed);
        authenticator.set_msk_host(auth, hostname, clock_offset)?;

        let mechanism_name = authenticator.mechanism_name().to_string();

        debug!("Starting SASL handshake with mechanism {mechanism_name} for {address}");

        // Step 1: SaslHandshake request
        let handshake_request = SaslHandshakeRequest::new(&mechanism_name);
        let mut encoder = Encoder::new();
        let pos = encoder.start_message();
        let header = RequestHeader::new(ApiKey::SaslHandshake, 1, 0).with_client_id(client_id);
        header.encode_v1(encoder.buffer_mut())?;
        handshake_request.encode_v1(encoder.buffer_mut())?;
        encoder.finish_message(pos)?;

        stream
            .write_all(&encoder.take())
            .await
            .map_err(KrafkaError::network)?;
        stream.flush().await.map_err(KrafkaError::network)?;

        // Read handshake response
        let mut response_buf = Self::read_framed_response(stream, max_response_size).await?;
        let _header = ResponseHeader::decode(&mut response_buf, ApiKey::SaslHandshake, 1)?;

        let handshake_response = SaslHandshakeResponse::decode_v0(&mut response_buf)?;
        if !handshake_response.is_ok() {
            return Err(KrafkaError::auth(format!(
                "SASL handshake failed: {:?}. Broker supports: {:?}",
                handshake_response.error_code, handshake_response.enabled_mechanisms
            )));
        }

        debug!(
            "SASL handshake accepted mechanism {mechanism_name}, broker supports: {:?}",
            handshake_response.enabled_mechanisms
        );

        // Step 2: SaslAuthenticate - initial response
        let initial_bytes = authenticator.initial_response()?;
        Self::send_sasl_authenticate(stream, &initial_bytes, client_id).await?;

        let auth_response =
            Self::read_sasl_authenticate_response(stream, max_response_size).await?;
        if !auth_response.error_code.is_ok() {
            let err_msg = auth_response.error_message.unwrap_or_default();
            // Best-effort clock skew detection for MSK IAM (C4).
            // AWS SigV4 errors for clock skew typically contain
            // phrases like "Signature expired" or "request time
            // too skewed".  When detected, apply a ±5 min offset
            // so the next reconnection attempt uses a corrected
            // timestamp.  This is a single-shot heuristic; more
            // sophisticated NTP-style correction is out of scope.
            if mechanism_name == "AWS_MSK_IAM" {
                let lower = err_msg.to_ascii_lowercase();
                if lower.contains("signature expired")
                    || lower.contains("signature not yet current")
                    || lower.contains("request time too")
                    || lower.contains("clock")
                    || lower.contains("time skew")
                {
                    // Try to extract an ISO-8601 timestamp from the error to
                    // compute the exact offset; fall back to a ±5 min nudge.
                    let skew = Self::extract_clock_skew_secs(&err_msg);
                    let prev = msk_iam_clock_offset_secs.load(Ordering::Relaxed);
                    let nudge = if skew != 0 {
                        skew
                    } else if lower.contains("expired") || lower.contains("past") {
                        // Signature expired → local clock is behind broker.
                        300
                    } else {
                        // Not yet current → local clock is ahead of broker.
                        -300
                    };
                    let adjusted =
                        Self::clamp_msk_iam_clock_offset_secs(prev.saturating_add(nudge));
                    msk_iam_clock_offset_secs.store(adjusted, Ordering::Relaxed);
                    warn!(
                        "MSK IAM auth failed with possible clock skew ({}); \
                         adjusted clock offset to {}s for next attempt",
                        err_msg, adjusted,
                    );
                }
            }
            return Err(KrafkaError::auth(format!(
                "SASL authentication failed: {:?} - {}",
                auth_response.error_code, err_msg
            )));
        }

        let mut session_lifetime_ms = auth_response.session_lifetime_ms;

        // Step 3: Challenge-response loop (for SCRAM-SHA-*)
        // Capped at MAX_SASL_ROUNDS to guard against malicious brokers.
        const MAX_SASL_ROUNDS: usize = 10;

        if !authenticator.is_complete() {
            let mut challenge = auth_response.auth_bytes;
            let mut rounds = 0;

            loop {
                match authenticator.process_challenge(&challenge)? {
                    ChallengeResponse::Done => break,
                    ChallengeResponse::AckThenFail { ack, error } => {
                        // Send the protocol-required ack (e.g., OAuthBearer \x01)
                        // then surface the auth error without reading a response —
                        // the server may close the connection immediately.
                        let _ = Self::send_sasl_authenticate(stream, &ack, client_id).await;
                        return Err(error);
                    }
                    ChallengeResponse::Continue(response_bytes) => {
                        rounds += 1;
                        if rounds > MAX_SASL_ROUNDS {
                            return Err(KrafkaError::auth(format!(
                                "SASL challenge-response exceeded {MAX_SASL_ROUNDS} rounds"
                            )));
                        }

                        Self::send_sasl_authenticate(stream, &response_bytes, client_id).await?;

                        let resp = Self::read_sasl_authenticate_response(stream, max_response_size)
                            .await?;
                        if !resp.error_code.is_ok() {
                            return Err(KrafkaError::auth(format!(
                                "SASL authentication step failed: {:?} - {}",
                                resp.error_code,
                                resp.error_message.unwrap_or_default()
                            )));
                        }

                        // The last successful response carries the session lifetime.
                        session_lifetime_ms = resp.session_lifetime_ms;
                        challenge = resp.auth_bytes;

                        if authenticator.is_complete() {
                            break;
                        }
                    }
                }
            }
        }

        info!("SASL authentication completed ({mechanism_name}) for {address}");

        if session_lifetime_ms > 0 {
            debug!("Broker reported session lifetime of {session_lifetime_ms}ms for {address}");
        }

        Ok(session_lifetime_ms)
    }

    /// Send a SaslAuthenticate v1 request on a raw stream.
    ///
    /// Uses API version 1 so the broker returns `session_lifetime_ms`
    /// in the response (KIP-368).
    async fn send_sasl_authenticate<S>(
        stream: &mut S,
        auth_bytes: &[u8],
        client_id: &str,
    ) -> Result<()>
    where
        S: AsyncWrite + Unpin,
    {
        let request = SaslAuthenticateRequest::new(auth_bytes.to_vec());
        let mut encoder = Encoder::new();
        let pos = encoder.start_message();
        let header = RequestHeader::new(ApiKey::SaslAuthenticate, 1, 1).with_client_id(client_id);
        header.encode(encoder.buffer_mut())?;
        request.encode_v1(encoder.buffer_mut())?;
        encoder.finish_message(pos)?;

        stream
            .write_all(&encoder.take())
            .await
            .map_err(KrafkaError::network)?;
        stream.flush().await.map_err(KrafkaError::network)?;
        Ok(())
    }

    /// Read a SaslAuthenticate v1 response from a raw stream.
    ///
    /// Decodes using v1 to obtain the `session_lifetime_ms` field (KIP-368).
    async fn read_sasl_authenticate_response<S>(
        stream: &mut S,
        max_response_size: usize,
    ) -> Result<SaslAuthenticateResponse>
    where
        S: AsyncRead + Unpin,
    {
        let mut buf = Self::read_framed_response(stream, max_response_size).await?;
        let _header = ResponseHeader::decode(&mut buf, ApiKey::SaslAuthenticate, 1)?;
        SaslAuthenticateResponse::decode_v1(&mut buf)
    }

    /// Read a length-prefixed Kafka response from a stream.
    async fn read_framed_response<S>(stream: &mut S, max_response_size: usize) -> Result<Bytes>
    where
        S: AsyncRead + Unpin,
    {
        // Read 4-byte length prefix
        let mut len_buf = [0u8; 4];
        stream
            .read_exact(&mut len_buf)
            .await
            .map_err(KrafkaError::network)?;
        let len_i32 = i32::from_be_bytes(len_buf);

        if len_i32 <= 0 || (len_i32 as usize) > max_response_size {
            return Err(KrafkaError::protocol(format!(
                "Invalid response length: {len_i32} (max: {max_response_size})"
            )));
        }

        let len = len_i32 as usize;

        // Read the response body
        let mut body = vec![0u8; len];
        stream
            .read_exact(&mut body)
            .await
            .map_err(KrafkaError::network)?;

        Ok(Bytes::from(body))
    }

    /// Try to extract a clock skew (in seconds) from an AWS SigV4 error message.
    ///
    /// AWS error messages for clock skew embed an ISO-8601 basic-format timestamp
    /// (e.g. `20250413T120000Z`) that represents the server's view of "now".
    /// If such a timestamp is found, returns `server_unix - local_unix` in seconds.
    /// Returns `0` if no parseable timestamp is present.
    fn extract_clock_skew_secs(error_msg: &str) -> i64 {
        use time::PrimitiveDateTime;
        use time::format_description::BorrowedFormatItem;
        use time::macros::format_description;

        // AWS SigV4 basic-format: YYYYMMDDTHHMMSSZ (16 bytes, ASCII-only).
        const AWS_TS_FMT: &[BorrowedFormatItem<'_>] =
            format_description!("[year][month][day]T[hour][minute][second]Z");
        const AWS_TS_LEN: usize = 16;

        let bytes = error_msg.as_bytes();
        if bytes.len() < AWS_TS_LEN {
            return 0;
        }
        // Scan every byte-aligned window of AWS_TS_LEN bytes. The grammar is
        // ASCII-only, so indexing into `error_msg` at these offsets is safe
        // (no UTF-8 split risk — a successful parse guarantees ASCII content).
        for i in 0..=bytes.len() - AWS_TS_LEN {
            // Cheap pre-filter: byte 8 must be 'T', byte 15 must be 'Z'.
            if bytes[i + 8] != b'T' || bytes[i + 15] != b'Z' {
                continue;
            }
            let Ok(candidate) = std::str::from_utf8(&bytes[i..i + AWS_TS_LEN]) else {
                continue;
            };
            let Ok(dt) = PrimitiveDateTime::parse(candidate, AWS_TS_FMT) else {
                continue;
            };
            let server_unix = dt.assume_utc().unix_timestamp();
            let local_unix = SystemTime::now()
                .duration_since(std::time::UNIX_EPOCH)
                .map(|d| d.as_secs() as i64)
                .unwrap_or(0);
            return server_unix - local_unix;
        }
        0
    }

    fn clamp_msk_iam_clock_offset_secs(offset: i64) -> i64 {
        offset.clamp(-MAX_SIGV4_CLOCK_SKEW_SECS, MAX_SIGV4_CLOCK_SKEW_SECS)
    }

    /// Run the connection event loop with priority handling.
    ///
    /// This is generic over the stream type, supporting both plain TCP and TLS.
    /// High-priority requests are always checked first using try_recv,
    /// ensuring heartbeats are never starved by backpressure on data requests.
    #[allow(clippy::too_many_arguments)]
    async fn run_connection_loop<R, W>(
        broker_address: String,
        mut reader: R,
        mut writer: W,
        mut high_priority_rx: mpsc::Receiver<ConnectionCommand>,
        mut normal_priority_rx: mpsc::Receiver<ConnectionCommand>,
        request_timeout: Duration,
        stats: Arc<ConnectionStats>,
        metrics: Arc<ConnectionMetrics>,
        max_response_size: usize,
        max_in_flight_requests: usize,
    ) -> Result<()>
    where
        R: AsyncRead + Unpin + Send + 'static,
        W: AsyncWrite + Unpin + Send + 'static,
    {
        // All pending request state is owned exclusively by this task.
        // No Arc<Mutex> needed — all access is single-threaded on this event loop.
        let mut pending: HashMap<CorrelationId, PendingRequest> = HashMap::new();

        // Per-request timeout via timer-wheel (tokio_util::time::DelayQueue).
        // Cost: O(log n) per insertion/expiration vs O(n × connections) for the
        // old 1-second polling task.  Each entry fires exactly once at
        // `enqueue_time + request_timeout`.
        let mut delay_queue: DelayQueue<CorrelationId> = DelayQueue::new();
        // Maps correlation_id → queue key for O(1) cancellation on response receipt.
        let mut delay_keys: HashMap<CorrelationId, delay_queue::Key> = HashMap::new();

        // Reader task sends decoded response frames to this loop via a bounded
        // channel.  The capacity matches max_in_flight_requests: the broker
        // can only send responses for outstanding requests, so this cap is
        // an exact fit.  It also provides back-pressure — if the main loop
        // is momentarily stalled (e.g., on a write), the reader suspends
        // instead of buffering unboundedly.
        let (frame_tx, mut frame_rx) =
            mpsc::channel::<Result<Bytes>>(max_in_flight_requests.max(1));

        let reader_handle = tokio::spawn(async move {
            let mut decoder = Decoder::with_max_size(max_response_size);
            let mut buf = vec![0u8; 65536];
            loop {
                match reader.read(&mut buf).await {
                    Ok(0) => {
                        debug!("Connection closed by peer");
                        break;
                    }
                    Ok(n) => {
                        decoder.extend(&buf[..n]);
                        loop {
                            match decoder.decode() {
                                Ok(Some(frame)) => {
                                    // Exit silently when the main loop has already gone away.
                                    if frame_tx.send(Ok(frame)).await.is_err() {
                                        return Ok::<_, KrafkaError>(());
                                    }
                                }
                                Ok(None) => break,
                                Err(e) => {
                                    let _ = frame_tx.send(Err(e)).await;
                                    return Ok(());
                                }
                            }
                        }
                    }
                    Err(e) => {
                        let _ = frame_tx.send(Err(KrafkaError::network(e))).await;
                        return Ok(());
                    }
                }
            }
            Ok(())
        });

        let mut terminal_error: Option<KrafkaError> = None;
        let mut consecutive_high_priority_commands = 0usize;
        let mut deferred_high_priority_cmd: Option<ConnectionCommand> = None;

        // Main event loop — lock-free on the hot path.
        loop {
            if consecutive_high_priority_commands >= MAX_HIGH_PRIORITY_BYPASSES_PER_ROUND {
                if deferred_high_priority_cmd.is_none() {
                    match high_priority_rx.try_recv() {
                        Ok(ConnectionCommand::Close) => {
                            consecutive_high_priority_commands = 0;
                            match Self::process_loop_command(
                                &mut writer,
                                &mut pending,
                                &mut delay_queue,
                                &mut delay_keys,
                                ConnectionCommand::Close,
                                max_in_flight_requests,
                                request_timeout,
                            )
                            .await
                            {
                                Ok(true) => break,
                                Ok(false) => {}
                                Err(err) => {
                                    terminal_error = Some(err);
                                    break;
                                }
                            }
                            continue;
                        }
                        Ok(cmd) => {
                            deferred_high_priority_cmd = Some(cmd);
                        }
                        Err(mpsc::error::TryRecvError::Empty)
                        | Err(mpsc::error::TryRecvError::Disconnected) => {}
                    }
                }

                match normal_priority_rx.try_recv() {
                    Ok(cmd) => {
                        stats
                            .high_priority_bypass_yields
                            .fetch_add(1, Ordering::Relaxed);
                        metrics.record_high_priority_bypass_yield();
                        consecutive_high_priority_commands = 0;
                        match Self::process_loop_command(
                            &mut writer,
                            &mut pending,
                            &mut delay_queue,
                            &mut delay_keys,
                            cmd,
                            max_in_flight_requests,
                            request_timeout,
                        )
                        .await
                        {
                            Ok(true) => break,
                            Ok(false) => {}
                            Err(err) => {
                                terminal_error = Some(err);
                                break;
                            }
                        }
                        continue;
                    }
                    Err(mpsc::error::TryRecvError::Empty)
                    | Err(mpsc::error::TryRecvError::Disconnected) => {
                        consecutive_high_priority_commands = 0;
                    }
                }
            }

            if let Some(cmd) = deferred_high_priority_cmd.take() {
                consecutive_high_priority_commands += 1;
                match Self::process_loop_command(
                    &mut writer,
                    &mut pending,
                    &mut delay_queue,
                    &mut delay_keys,
                    cmd,
                    max_in_flight_requests,
                    request_timeout,
                )
                .await
                {
                    Ok(true) => break,
                    Ok(false) => {}
                    Err(err) => {
                        terminal_error = Some(err);
                        break;
                    }
                }
                continue;
            }

            // Fast path: drain the high-priority channel without yielding to the
            // scheduler.  Heartbeats are the most latency-sensitive request type.
            if let Ok(cmd) = high_priority_rx.try_recv() {
                stats.high_priority_bypasses.fetch_add(1, Ordering::Relaxed);
                metrics.record_high_priority_bypass();
                consecutive_high_priority_commands += 1;
                match Self::process_loop_command(
                    &mut writer,
                    &mut pending,
                    &mut delay_queue,
                    &mut delay_keys,
                    cmd,
                    max_in_flight_requests,
                    request_timeout,
                )
                .await
                {
                    Ok(true) => break,
                    Ok(false) => {}
                    Err(err) => {
                        terminal_error = Some(err);
                        break;
                    }
                }
                continue;
            }

            tokio::select! {
                biased;

                // Response frames from the reader task — dispatch immediately so
                // callers receive results as soon as the bytes arrive.
                frame_result = frame_rx.recv() => {
                    consecutive_high_priority_commands = 0;
                    match frame_result {
                        Some(Ok(frame)) => {
                            if let Err(e) = Self::dispatch_response(
                                &mut pending,
                                &mut delay_queue,
                                &mut delay_keys,
                                frame,
                                &broker_address,
                            ) {
                                // Protocol desynchronisation — close the connection.
                                terminal_error = Some(e);
                                break;
                            }
                        }
                        Some(Err(e)) => {
                            terminal_error = Some(e);
                            break;
                        }
                        None => {
                            // Reader task exited (peer closed the connection).
                            break;
                        }
                    }
                }

                // Timer-wheel: fires exactly when a per-request deadline expires.
                // O(log n) cost vs O(n × connections) for the old 1-second sweep.
                Some(expired) = std::future::poll_fn(|cx| {
                    use futures_core::Stream;
                    std::pin::Pin::new(&mut delay_queue).poll_next(cx)
                }) => {
                    consecutive_high_priority_commands = 0;
                    let id = expired.into_inner();
                    if let Some(req) = pending.remove(&id) {
                        delay_keys.remove(&id);
                        warn!(
                            correlation_id = id,
                            "Request timed out after {:?}", request_timeout
                        );
                        let _ = req.response_tx.send(Err(KrafkaError::timeout(format!(
                            "request {id} timed out after {request_timeout:?}"
                        ))));
                    }
                }

                // High-priority commands (heartbeats, metadata, coordinator lookups).
                cmd = high_priority_rx.recv() => {
                    match cmd {
                        Some(cmd) => {
                            consecutive_high_priority_commands += 1;
                            match Self::process_loop_command(
                                &mut writer,
                                &mut pending,
                                &mut delay_queue,
                                &mut delay_keys,
                                cmd,
                                max_in_flight_requests,
                                request_timeout,
                            )
                            .await {
                                Ok(true) => break,
                                Ok(false) => {}
                                Err(err) => {
                                    terminal_error = Some(err);
                                    break;
                                }
                            }
                        }
                        None => break,
                    }
                }

                // Normal-priority commands (produce, fetch, and all others).
                cmd = normal_priority_rx.recv() => {
                    match cmd {
                        Some(cmd) => {
                            consecutive_high_priority_commands = 0;
                            match Self::process_loop_command(
                                &mut writer,
                                &mut pending,
                                &mut delay_queue,
                                &mut delay_keys,
                                cmd,
                                max_in_flight_requests,
                                request_timeout,
                            )
                            .await {
                                Ok(true) => break,
                                Ok(false) => {}
                                Err(err) => {
                                    terminal_error = Some(err);
                                    break;
                                }
                            }
                        }
                        None => break,
                    }
                }
            }
        }

        // Drop the writer half to signal EOF to the broker, then abort the
        // reader task — we no longer need its output.
        drop(writer);
        reader_handle.abort();

        // Drain all in-flight requests and notify callers that the connection
        // is gone.
        let pending_error = terminal_error
            .clone()
            .unwrap_or_else(|| KrafkaError::invalid_state("connection closed"));
        for (_, req) in pending.drain() {
            let _ = req.response_tx.send(Err(pending_error.clone()));
        }

        if let Some(err) = terminal_error {
            return Err(err);
        }

        Ok(())
    }

    async fn process_loop_command<W: AsyncWrite + Unpin>(
        writer: &mut W,
        pending: &mut HashMap<CorrelationId, PendingRequest>,
        delay_queue: &mut DelayQueue<CorrelationId>,
        delay_keys: &mut HashMap<CorrelationId, delay_queue::Key>,
        cmd: ConnectionCommand,
        max_in_flight_requests: usize,
        request_timeout: Duration,
    ) -> Result<bool> {
        Self::handle_command_direct(
            writer,
            pending,
            delay_queue,
            delay_keys,
            cmd,
            max_in_flight_requests,
            request_timeout,
        )
        .await
    }

    /// Handle a single connection command.
    ///
    /// Returns `true` if the connection should close.
    ///
    /// # Lock-free hot path
    ///
    /// The pending map is owned by the single event-loop task — all insertions
    /// and removals are O(1) HashMap operations with no synchronization overhead.
    async fn handle_command_direct<W: AsyncWrite + Unpin>(
        writer: &mut W,
        pending: &mut HashMap<CorrelationId, PendingRequest>,
        delay_queue: &mut DelayQueue<CorrelationId>,
        delay_keys: &mut HashMap<CorrelationId, delay_queue::Key>,
        cmd: ConnectionCommand,
        max_in_flight_requests: usize,
        request_timeout: Duration,
    ) -> Result<bool> {
        match cmd {
            ConnectionCommand::Request {
                data,
                correlation_id,
                api_key,
                api_version,
                response_tx,
            } => {
                if pending.contains_key(&correlation_id) {
                    let error = KrafkaError::invalid_state(format!(
                        "correlation ID collision on broker connection: correlation_id={correlation_id}, pending_requests={}; closing connection",
                        pending.len()
                    ));
                    error!(
                        correlation_id,
                        pending_requests = pending.len(),
                        "Detected correlation ID collision; closing connection"
                    );
                    let _ = response_tx.send(Err(error.clone()));
                    return Err(error);
                }

                // Reject when at capacity to prevent unbounded memory growth.
                if pending.len() >= max_in_flight_requests {
                    warn!(
                        pending = pending.len(),
                        max = max_in_flight_requests,
                        "Rejecting request: max in-flight requests reached"
                    );
                    let _ = response_tx.send(Err(KrafkaError::invalid_state(format!(
                        "max in-flight requests ({max_in_flight_requests}) reached"
                    ))));
                    return Ok(false);
                }

                // Snapshot the deadline before touching the wire so that the
                // end-to-end budget (write + network round-trip) is exactly
                // request_timeout, not up to 2× request_timeout.
                let deadline = tokio::time::Instant::now() + request_timeout;

                // Write to the wire.  Register in pending only after a successful
                // write so we never create a leaked entry for an undelivered request.
                //
                // Uses the same absolute deadline as the DelayQueue entry below so
                // write + response wait together consume exactly one request_timeout
                // budget.  A stalled TCP write cannot freeze the event loop (and
                // therefore block all in-flight timeout processing) for longer than
                // the remaining budget.
                let write_result = tokio::time::timeout_at(deadline, async {
                    writer.write_all(&data).await?;
                    writer.flush().await
                })
                .await;
                match write_result {
                    Ok(Ok(())) => {}
                    Ok(Err(e)) => {
                        error!("Write error: {}", e);
                        let _ = response_tx.send(Err(KrafkaError::network(e)));
                        return Ok(false);
                    }
                    Err(_) => {
                        let msg = format!("write timed out after {request_timeout:?}");
                        error!("{msg}");
                        let _ = response_tx.send(Err(KrafkaError::timeout(msg.clone())));
                        // The stream is in an indeterminate state — close the connection.
                        return Err(KrafkaError::timeout(msg));
                    }
                }

                // Register pending entry and arm the per-request timeout at the
                // same absolute deadline used for the write, so the whole
                // request (write + response wait) is bounded by request_timeout.
                let key = delay_queue.insert_at(correlation_id, deadline);
                delay_keys.insert(correlation_id, key);
                pending.insert(
                    correlation_id,
                    PendingRequest {
                        response_tx,
                        api_key,
                        api_version,
                    },
                );
                Ok(false)
            }
            ConnectionCommand::Close => {
                debug!("Closing connection");
                Ok(true)
            }
            ConnectionCommand::FireAndForget { data } => {
                // No response is expected, so a relative timeout is sufficient —
                // there is no second phase to share a deadline with.
                let write_result = tokio::time::timeout(request_timeout, async {
                    writer.write_all(&data).await?;
                    writer.flush().await
                })
                .await;
                match write_result {
                    Ok(Ok(())) => {}
                    Ok(Err(e)) => error!("Fire-and-forget write error: {}", e),
                    Err(_) => {
                        error!(
                            "Fire-and-forget write timed out after {:?}",
                            request_timeout
                        );
                        return Err(KrafkaError::timeout(format!(
                            "fire-and-forget write timed out after {request_timeout:?}"
                        )));
                    }
                }
                Ok(false)
            }
        }
    }

    /// Dispatch an incoming response frame to the waiting caller.
    ///
    /// Looks up the correlation ID in the pending map, cancels the associated
    /// timeout, decodes the response header, and delivers the body.
    ///
    /// Returns `Err` only on protocol-level desynchronisation (unknown
    /// correlation ID or undecodable response header) — both indicate a corrupt
    /// stream and require the connection to be closed.
    fn dispatch_response(
        pending: &mut HashMap<CorrelationId, PendingRequest>,
        delay_queue: &mut DelayQueue<CorrelationId>,
        delay_keys: &mut HashMap<CorrelationId, delay_queue::Key>,
        response: Bytes,
        broker_address: &str,
    ) -> Result<()> {
        if response.len() < 4 {
            return Err(KrafkaError::protocol(format!(
                "response too short from broker {broker_address}: frame_bytes={}",
                response.len()
            )));
        }

        let correlation_id =
            i32::from_be_bytes([response[0], response[1], response[2], response[3]]);

        let pending_before_remove = pending.len();
        if let Some(req) = pending.remove(&correlation_id) {
            // Cancel the timeout — the response arrived before the deadline.
            if let Some(key) = delay_keys.remove(&correlation_id) {
                delay_queue.remove(&key);
            }

            trace!("Received response for correlation_id={}", correlation_id);

            let mut response_buf = response.slice(..);
            match ResponseHeader::decode(&mut response_buf, req.api_key, req.api_version) {
                Ok(_header) => {
                    let header_size = response.len() - response_buf.len();
                    let body = response.slice(header_size..);
                    let _ = req.response_tx.send(Ok(body));
                }
                Err(e) => {
                    // Header decode failure means the stream is desynchronised
                    // — notify the caller and tear down the connection.
                    let response_header_version =
                        ResponseHeader::header_version(req.api_key, req.api_version);
                    let context = format!(
                        "response header decode failed: broker={broker_address}, api_key={:?}, api_version={}, response_header_version={}, correlation_id={correlation_id}, frame_bytes={}, pending_before_remove={pending_before_remove}, error={e}",
                        req.api_key,
                        req.api_version,
                        response_header_version,
                        response.len(),
                    );
                    warn!(
                        broker = broker_address,
                        api_key = ?req.api_key,
                        api_version = req.api_version,
                        response_header_version,
                        correlation_id,
                        frame_bytes = response.len(),
                        pending_before_remove,
                        error = %e,
                        "Failed to decode response header; closing connection"
                    );
                    let _ = req
                        .response_tx
                        .send(Err(KrafkaError::protocol(context.clone())));
                    return Err(KrafkaError::protocol(format!(
                        "{context}; stream desynchronized"
                    )));
                }
            }
        } else {
            // Unknown correlation ID indicates a protocol desync.
            return Err(KrafkaError::protocol(format!(
                "Received response for unknown correlation_id={correlation_id} from broker {broker_address}; frame_bytes={}, pending_requests={pending_before_remove}; closing connection",
                response.len()
            )));
        }

        Ok(())
    }

    /// Fetch API versions from the broker.
    async fn fetch_api_versions(&self) -> Result<()> {
        let request =
            ApiVersionsRequest::new().with_client_software("krafka", env!("CARGO_PKG_VERSION"));

        let correlation_id = self.correlation_id_gen.next();
        let mut encoder = Encoder::new();

        // Build request
        let pos = encoder.start_message();
        let header = RequestHeader::new(ApiKey::ApiVersions, 0, correlation_id)
            .with_client_id(&self.config.client_id);
        header.encode_v1(encoder.buffer_mut())?;
        request.encode_v0(encoder.buffer_mut())?;
        encoder.finish_message(pos)?;

        // Send request (use high priority for API versions)
        let (response_tx, response_rx) = oneshot::channel();
        self.high_priority_tx
            .send(ConnectionCommand::Request {
                data: encoder.take(),
                correlation_id,
                api_key: ApiKey::ApiVersions,
                api_version: 0,
                response_tx,
            })
            .await
            .map_err(|_| KrafkaError::invalid_state("connection closed"))?;

        self.stats
            .high_priority_requests
            .fetch_add(1, Ordering::Relaxed);
        self.config
            .connection_metrics
            .record_high_priority_request();

        // Wait for response
        let response = timeout(self.config.request_timeout, response_rx)
            .await
            .map_err(|_| KrafkaError::timeout("api versions request"))?
            .map_err(|_| KrafkaError::invalid_state("response channel closed"))??;

        // Decode response
        let mut buf = response;
        let api_versions_response = ApiVersionsResponse::decode_v0(&mut buf)?;

        if api_versions_response.error_code != 0 {
            return Err(KrafkaError::protocol(format!(
                "ApiVersions error: {}",
                api_versions_response.error_code
            )));
        }

        // Store API versions
        let mut versions = self.api_versions.lock();
        for range in api_versions_response.api_keys {
            versions.insert(range.api_key, range);
        }

        debug!("Fetched {} API versions", versions.len());
        Ok(())
    }

    /// Choose the appropriate channel based on request priority.
    #[inline]
    fn channel_for_priority(&self, priority: RequestPriority) -> &mpsc::Sender<ConnectionCommand> {
        match priority {
            RequestPriority::High => &self.high_priority_tx,
            RequestPriority::Normal => &self.normal_priority_tx,
        }
    }

    /// Record a broker-reported throttle time (KIP-219).
    ///
    /// When the broker returns `throttle_time_ms > 0` in a response, the
    /// client should voluntarily delay subsequent normal-priority requests
    /// by that amount. High-priority requests (heartbeats, metadata) are
    /// never delayed.
    pub fn notify_throttle(&self, throttle_time_ms: i32) {
        if throttle_time_ms > 0 {
            let new_deadline = Instant::now() + Duration::from_millis(throttle_time_ms as u64);
            let mut deadline = self.throttle_until.lock();
            if new_deadline > *deadline {
                debug!(
                    throttle_ms = throttle_time_ms,
                    broker = %self.address,
                    "Broker throttle applied (KIP-219)"
                );
                *deadline = new_deadline;
            }
        }
    }

    /// Send a request with automatic priority based on API key.
    ///
    /// Priority is determined automatically:
    /// - High: Heartbeat, Metadata, FindCoordinator, ApiVersions
    /// - Normal: Produce, Fetch, and all other requests
    pub async fn send_request(
        &self,
        api_key: ApiKey,
        api_version: i16,
        request_body: impl FnOnce(&mut BytesMut) -> Result<()>,
    ) -> Result<Bytes> {
        let priority = RequestPriority::for_api_key(api_key);
        self.send_request_with_priority(api_key, api_version, priority, request_body)
            .await
    }

    /// Send a request with explicit priority.
    ///
    /// Use this when you need to override the automatic priority selection.
    /// Normal-priority requests are delayed when the broker has signalled
    /// quota throttling (KIP-219).
    pub async fn send_request_with_priority(
        &self,
        api_key: ApiKey,
        api_version: i16,
        priority: RequestPriority,
        request_body: impl FnOnce(&mut BytesMut) -> Result<()>,
    ) -> Result<Bytes> {
        // M1: refresh the idle timestamp on every submission so the pool's
        // idle-evictor does not close an actively used connection.
        self.mark_used();

        // KIP-219: honour broker throttle for normal-priority requests.
        if priority == RequestPriority::Normal {
            let remaining = {
                let deadline = self.throttle_until.lock();
                deadline.checked_duration_since(Instant::now())
            };
            if let Some(delay) = remaining {
                debug!(
                    delay_ms = delay.as_millis() as u64,
                    broker = %self.address,
                    "Delaying request due to broker throttle (KIP-219)"
                );
                self.config.connection_metrics.record_throttle_delay(delay);
                tokio::time::sleep(delay).await;
            }
        }

        let correlation_id = self.correlation_id_gen.next();
        let mut encoder = Encoder::new();

        // Build request
        let pos = encoder.start_message();
        let header = RequestHeader::new(api_key, api_version, correlation_id)
            .with_client_id(&self.config.client_id);
        header.encode(encoder.buffer_mut())?;
        request_body(encoder.buffer_mut())?;
        encoder.finish_message(pos)?;

        // Send request to appropriate channel
        let (response_tx, response_rx) = oneshot::channel();
        let channel = self.channel_for_priority(priority);
        channel
            .send(ConnectionCommand::Request {
                data: encoder.take(),
                correlation_id,
                api_key,
                api_version,
                response_tx,
            })
            .await
            .map_err(|_| KrafkaError::invalid_state("connection closed"))?;

        // Update stats
        match priority {
            RequestPriority::High => {
                self.stats
                    .high_priority_requests
                    .fetch_add(1, Ordering::Relaxed);
                self.config
                    .connection_metrics
                    .record_high_priority_request();
            }
            RequestPriority::Normal => {
                self.stats
                    .normal_priority_requests
                    .fetch_add(1, Ordering::Relaxed);
                self.config
                    .connection_metrics
                    .record_normal_priority_request();
            }
        }

        // Wait for response
        let response = timeout(self.config.request_timeout, response_rx)
            .await
            .map_err(|_| KrafkaError::timeout("request"))?
            .map_err(|_| KrafkaError::invalid_state("response channel closed"))??;

        Ok(response)
    }

    /// Send a request without waiting for a response (fire-and-forget).
    ///
    /// Used for `acks=0` produce requests where the Kafka broker does not
    /// send a response. The request is written to the wire but no response
    /// channel is registered in the pending map, avoiding resource leaks and
    /// preserving the normal correlation-ID space for requests that expect
    /// responses.
    pub async fn send_fire_and_forget(
        &self,
        api_key: ApiKey,
        api_version: i16,
        request_body: impl FnOnce(&mut BytesMut) -> Result<()>,
    ) -> Result<()> {
        // M1: refresh the idle timestamp on every submission.
        self.mark_used();

        let mut encoder = Encoder::new();

        // Build request
        let pos = encoder.start_message();
        let header = RequestHeader::new(api_key, api_version, NO_RESPONSE_CORRELATION_ID)
            .with_client_id(&self.config.client_id);
        header.encode(encoder.buffer_mut())?;
        request_body(encoder.buffer_mut())?;
        encoder.finish_message(pos)?;

        // Send as fire-and-forget — no pending entry is created
        let channel = self.channel_for_priority(RequestPriority::Normal);
        channel
            .send(ConnectionCommand::FireAndForget {
                data: encoder.take(),
            })
            .await
            .map_err(|_| KrafkaError::invalid_state("connection closed"))?;

        self.stats
            .normal_priority_requests
            .fetch_add(1, Ordering::Relaxed);
        self.config
            .connection_metrics
            .record_normal_priority_request();

        Ok(())
    }

    /// Get the supported API version for a specific API.
    pub async fn get_api_version(&self, api_key: ApiKey) -> Option<ApiVersionRange> {
        let versions = self.api_versions.lock();
        versions.get(&api_key).copied()
    }

    /// Negotiate the best API version for a given API key.
    ///
    /// Returns the highest mutually supported version between the client and broker.
    ///
    /// # Arguments
    ///
    /// * `api_key` - The API key to negotiate
    /// * `client_max` - Maximum version the client supports
    /// * `client_min` - Minimum version the client supports (default 0)
    ///
    /// # Returns
    ///
    /// The negotiated version, or None if:
    /// - The broker doesn't support this API
    /// - There's no overlap between client and broker versions
    ///
    /// # Example
    ///
    /// ```rust,ignore
    /// // Client supports Fetch v4-v12
    /// let version = conn.negotiate_api_version(ApiKey::Fetch, 12, 4).await;
    /// ```
    pub async fn negotiate_api_version(
        &self,
        api_key: ApiKey,
        client_max: i16,
        client_min: i16,
    ) -> Option<i16> {
        let versions = self.api_versions.lock();
        versions
            .get(&api_key)
            .and_then(|range| range.negotiate(client_max, client_min))
    }

    /// Negotiate the best API version with minimum version defaulting to 0.
    pub async fn negotiate_api_version_max(&self, api_key: ApiKey, client_max: i16) -> Option<i16> {
        self.negotiate_api_version(api_key, client_max, 0).await
    }

    /// Compute the session expiry instant from a broker-reported lifetime.
    ///
    /// Returns `None` when `session_lifetime_ms` is zero or negative (no expiry).
    /// Otherwise picks a random reauthentication point between 85 % and 95 %
    /// of the session lifetime.  The jitter avoids a thundering-herd where
    /// many connections to the same broker all need replacement at the same
    /// instant.  This matches the approach taken by the Java Kafka client.
    fn compute_session_expiry(session_lifetime_ms: i64) -> Option<Instant> {
        if session_lifetime_ms <= 0 {
            return None;
        }
        // Randomised window: 85 % base + up to 10 % jitter = 85-95 % of lifetime.
        // Mirrors Java client's pctWindowFactor (0.85) + jitter (0.10).
        const MIN_REAUTH_MS: u64 = 100;
        let base_factor: f64 = 0.85;
        let jitter_range: f64 = 0.10;
        let jitter: f64 = rand::random::<f64>() * jitter_range;
        let factor = base_factor + jitter;
        let computed_reauth_ms = (session_lifetime_ms as f64 * factor) as u64;
        let reauth_ms = computed_reauth_ms.max(MIN_REAUTH_MS);
        if computed_reauth_ms < MIN_REAUTH_MS {
            warn!(
                session_lifetime_ms,
                computed_reauth_ms,
                reauth_ms,
                "broker reported unusually small SASL session lifetime; clamping reauthentication delay"
            );
        }
        Some(Instant::now() + Duration::from_millis(reauth_ms))
    }

    /// Compute session expiry, falling back to an OAuthBearer token lifetime
    /// when the broker does not report `session_lifetime_ms` (KIP-368).
    ///
    /// The token's `lifetime_ms` is an epoch-millisecond timestamp. We convert
    /// it to a remaining duration before passing it through the standard
    /// jittered-window logic.
    fn effective_session_expiry(session_lifetime_ms: i64, auth: &AuthConfig) -> Option<Instant> {
        if session_lifetime_ms > 0 {
            return Self::compute_session_expiry(session_lifetime_ms);
        }

        // Fall back to OAuthBearer token lifetime if available.
        if let Some(token) = auth.oauthbearer_token.as_ref()
            && let Some(expiry_epoch_ms) = token.lifetime_ms()
        {
            let now_epoch_ms = std::time::SystemTime::now()
                .duration_since(std::time::UNIX_EPOCH)
                .unwrap_or_default()
                .as_millis() as i64;
            let remaining_ms = expiry_epoch_ms.saturating_sub(now_epoch_ms);
            if remaining_ms > 0 {
                return Self::compute_session_expiry(remaining_ms);
            }
        }

        None
    }

    /// Whether the SASL session is approaching expiry and the connection
    /// should be replaced (KIP-368).
    ///
    /// Returns `false` when no session lifetime was reported by the broker.
    #[inline]
    pub fn needs_reauthentication(&self) -> bool {
        self.session_expiry
            .is_some_and(|expiry| Instant::now() >= expiry)
    }

    /// The instant at which the client should start reauthentication, if any.
    #[inline]
    pub fn session_expiry(&self) -> Option<Instant> {
        self.session_expiry
    }

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

    /// Whether the connection is alive and its SASL session has not expired.
    ///
    /// This is the primary check used by the connection pool to decide if an
    /// existing connection can be reused or must be replaced.
    #[inline]
    pub fn is_usable(&self) -> bool {
        self.is_alive() && !self.needs_reauthentication()
    }

    /// Record that the connection was just used for a request.
    ///
    /// Called from the submission paths (`send_request_with_priority`,
    /// `send_fire_and_forget`). Stores monotonic nanos since `created_at`
    /// into `last_used_nanos`; reads happen from [`idle_duration`].
    #[inline]
    fn mark_used(&self) {
        let elapsed = self.created_at.elapsed().as_nanos();
        // Saturate on the (astronomical) overflow boundary rather than panic.
        let nanos = u64::try_from(elapsed).unwrap_or(u64::MAX);
        self.last_used_nanos.store(nanos, Ordering::Relaxed);
    }

    /// Duration since the last submitted request on this connection.
    ///
    /// A freshly connected socket that has sent no requests reports its
    /// full age (since `created_at`) as idle — identical to Java's
    /// `connections.max.idle.ms` accounting.
    #[inline]
    pub fn idle_duration(&self) -> Duration {
        let last = self.last_used_nanos.load(Ordering::Relaxed);
        let now = self.created_at.elapsed();
        now.saturating_sub(Duration::from_nanos(last))
    }

    /// Test-only: construct a minimal, non-I/O-capable `BrokerConnection`
    /// with `created_at` backdated by `idle_for`, so `idle_duration()`
    /// reports at least `idle_for`. Used by pool eviction tests that need
    /// to exercise `evict_idle` without standing up a real broker.
    ///
    /// The returned connection:
    /// - has dropped receivers for both priority channels (sending on it
    ///   will fail; this is intentional — the stub is only consumed by
    ///   the eviction scan, which never sends);
    /// - is marked `alive = true` so `is_alive()` reports consistently;
    /// - has `last_used_nanos = 0` so idle time equals full age.
    #[cfg(test)]
    #[allow(clippy::expect_used)]
    pub(crate) fn test_stub_idle_for(address: &str, idle_for: Duration) -> Self {
        let (high_priority_tx, _) = mpsc::channel(1);
        let (normal_priority_tx, _) = mpsc::channel(1);
        Self {
            address: address.to_string(),
            config: ConnectionConfig::default(),
            correlation_id_gen: Arc::new(CorrelationIdGenerator::new()),
            high_priority_tx,
            normal_priority_tx,
            api_versions: Arc::new(parking_lot::Mutex::new(HashMap::new())),
            alive: Arc::new(std::sync::atomic::AtomicBool::new(true)),
            session_expiry: None,
            stats: Arc::new(ConnectionStats::default()),
            throttle_until: Arc::new(parking_lot::Mutex::new(Instant::now())),
            created_at: Instant::now()
                .checked_sub(idle_for)
                // `unwrap`: test idle_for values are always small (≤ 10s) and
                // any system uptime on which tests run exceeds that easily;
                // failing loudly here is better than silently yielding a fresh
                // timestamp that makes eviction tests vacuously pass.
                .expect("idle_for exceeds system uptime; cannot backdate Instant"),
            last_used_nanos: AtomicU64::new(0),
        }
    }

    /// Test-only: refresh `last_used_nanos` to "now" without going through
    /// a send path. Used to verify the evictor's race re-check rescues a
    /// connection that was refreshed between the snapshot and the write.
    #[cfg(test)]
    pub(crate) fn test_mark_fresh(&self) {
        self.mark_used();
    }

    /// Get the broker address.
    #[inline]
    pub fn address(&self) -> &str {
        &self.address
    }

    /// Get connection statistics.
    #[inline]
    pub fn stats(&self) -> &ConnectionStats {
        &self.stats
    }

    /// Close the connection.
    pub async fn close(&self) {
        // Use high-priority channel for close command
        let _ = self.high_priority_tx.send(ConnectionCommand::Close).await;
    }
}

impl Drop for BrokerConnection {
    fn drop(&mut self) {
        // Only attempt close if a Tokio runtime is active — avoids panic
        // when dropped outside a runtime (e.g., during process exit or in tests).
        if let Ok(_handle) = tokio::runtime::Handle::try_current() {
            let tx = self.high_priority_tx.clone();
            tokio::spawn(async move {
                let _ = tx.send(ConnectionCommand::Close).await;
            });
        }
    }
}

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

    #[test]
    fn test_connection_config_builder() {
        let config = ConnectionConfig::builder()
            .connect_timeout(Duration::from_secs(5))
            .request_timeout(Duration::from_secs(15))
            .client_id("test-client")
            .nodelay(false)
            .build();

        assert_eq!(config.connect_timeout, Duration::from_secs(5));
        assert_eq!(config.request_timeout, Duration::from_secs(15));
        assert_eq!(config.client_id, "test-client");
        assert!(!config.nodelay);
    }

    #[test]
    fn test_connection_config_default() {
        let config = ConnectionConfig::default();
        assert_eq!(config.connect_timeout, Duration::from_secs(10));
        assert_eq!(config.request_timeout, Duration::from_secs(30));
        assert_eq!(config.client_id, "krafka");
        assert!(config.nodelay);
        assert_eq!(config.connections_per_broker, 1);
        assert_eq!(config.high_priority_channel_capacity, 64);
        assert_eq!(config.normal_priority_channel_capacity, 256);
        assert!(config.auth.is_none());
    }

    #[test]
    fn test_connection_config_uses_shared_metrics_handle() {
        let metrics = Arc::new(ConnectionMetrics::default());
        let config = ConnectionConfig::builder()
            .connection_metrics(metrics.clone())
            .build();

        config.connection_metrics.record_high_priority_request();
        assert_eq!(metrics.high_priority_requests.get(), 1);
        assert!(Arc::ptr_eq(&metrics, &config.connection_metrics()));
    }

    #[test]
    fn test_connection_config_with_auth() {
        use crate::auth::AuthConfig;
        let config = ConnectionConfig::builder()
            .client_id("test")
            .auth(AuthConfig::sasl_plain("user", "pass").unwrap())
            .build();

        assert_eq!(config.client_id, "test");
        let auth = config.auth.as_ref().unwrap();
        assert!(auth.requires_sasl());
        assert!(!auth.requires_tls());
    }

    #[test]
    fn test_connection_config_builder_with_priority() {
        let config = ConnectionConfig::builder()
            .connections_per_broker(4)
            .high_priority_channel_capacity(32)
            .normal_priority_channel_capacity(512)
            .build();

        assert_eq!(config.connections_per_broker, 4);
        assert_eq!(config.high_priority_channel_capacity, 32);
        assert_eq!(config.normal_priority_channel_capacity, 512);
    }

    #[test]
    fn test_connection_config_min_values() {
        // Ensure minimums are enforced
        let config = ConnectionConfig::builder()
            .connections_per_broker(0) // Should become 1
            .high_priority_channel_capacity(0) // Should become 16
            .normal_priority_channel_capacity(0) // Should become 64
            .build();

        assert_eq!(config.connections_per_broker, 1);
        assert_eq!(config.high_priority_channel_capacity, 16);
        assert_eq!(config.normal_priority_channel_capacity, 64);
    }

    #[test]
    fn test_request_priority_for_api_key() {
        // High priority APIs
        assert_eq!(
            RequestPriority::for_api_key(ApiKey::Heartbeat),
            RequestPriority::High
        );
        assert_eq!(
            RequestPriority::for_api_key(ApiKey::Metadata),
            RequestPriority::High
        );
        assert_eq!(
            RequestPriority::for_api_key(ApiKey::FindCoordinator),
            RequestPriority::High
        );
        assert_eq!(
            RequestPriority::for_api_key(ApiKey::ApiVersions),
            RequestPriority::High
        );

        // Normal priority APIs
        assert_eq!(
            RequestPriority::for_api_key(ApiKey::Produce),
            RequestPriority::Normal
        );
        assert_eq!(
            RequestPriority::for_api_key(ApiKey::Fetch),
            RequestPriority::Normal
        );
        assert_eq!(
            RequestPriority::for_api_key(ApiKey::OffsetCommit),
            RequestPriority::Normal
        );
        assert_eq!(
            RequestPriority::for_api_key(ApiKey::OffsetFetch),
            RequestPriority::Normal
        );
    }

    #[test]
    fn test_connection_stats_default() {
        let stats = ConnectionStats::default();
        assert_eq!(stats.high_priority_count(), 0);
        assert_eq!(stats.normal_priority_count(), 0);
        assert_eq!(stats.bypass_count(), 0);
        assert_eq!(stats.bypass_yield_count(), 0);
    }

    #[test]
    fn test_connection_stats_increment() {
        let stats = ConnectionStats::default();
        stats.high_priority_requests.fetch_add(5, Ordering::Relaxed);
        stats
            .normal_priority_requests
            .fetch_add(10, Ordering::Relaxed);
        stats.high_priority_bypasses.fetch_add(2, Ordering::Relaxed);
        stats
            .high_priority_bypass_yields
            .fetch_add(1, Ordering::Relaxed);

        assert_eq!(stats.high_priority_count(), 5);
        assert_eq!(stats.normal_priority_count(), 10);
        assert_eq!(stats.bypass_count(), 2);
        assert_eq!(stats.bypass_yield_count(), 1);
    }

    #[test]
    fn test_dispatch_response_header_decode_error_includes_context() {
        let correlation_id = 7;
        let (response_tx, mut response_rx) = oneshot::channel();
        let mut pending = HashMap::new();
        pending.insert(
            correlation_id,
            PendingRequest {
                response_tx,
                api_key: ApiKey::Metadata,
                api_version: 9,
            },
        );
        let mut delay_queue = DelayQueue::new();
        let mut delay_keys = HashMap::new();

        let err = BrokerConnection::dispatch_response(
            &mut pending,
            &mut delay_queue,
            &mut delay_keys,
            Bytes::copy_from_slice(&correlation_id.to_be_bytes()),
            "broker-1:9092",
        )
        .unwrap_err();
        let caller_err = response_rx.try_recv().unwrap().unwrap_err();
        let err_text = caller_err.to_string();

        assert!(err.to_string().contains("stream desynchronized"));
        assert!(err_text.contains("broker=broker-1:9092"));
        assert!(err_text.contains("api_key=Metadata"));
        assert!(err_text.contains("api_version=9"));
        assert!(err_text.contains("response_header_version=1"));
        assert!(err_text.contains("correlation_id=7"));
        assert!(err_text.contains("frame_bytes=4"));
    }

    /// Mock Kafka broker that handles the SASL handshake protocol.
    ///
    /// Accepts a connection, reads SaslHandshakeRequest, SaslAuthenticateRequest,
    /// and ApiVersionsRequest, responding to each with valid responses.
    /// The `session_lifetime_ms` value is included in the SaslAuthenticate v1
    /// response (KIP-368). The broker stays open until the test signals
    /// shutdown so the connection remains usable after the initial handshake.
    /// Returns the captured auth bytes from SaslAuthenticate for verification.
    async fn run_mock_sasl_broker(
        listener: tokio::net::TcpListener,
        shutdown_rx: oneshot::Receiver<()>,
    ) -> (String, Vec<u8>) {
        run_mock_sasl_broker_with_lifetime(listener, 0, shutdown_rx).await
    }

    /// Like [`run_mock_sasl_broker`] but lets the caller set the session
    /// lifetime reported in the SaslAuthenticateResponse (KIP-368).
    async fn run_mock_sasl_broker_with_lifetime(
        listener: tokio::net::TcpListener,
        session_lifetime_ms: i64,
        shutdown_rx: oneshot::Receiver<()>,
    ) -> (String, Vec<u8>) {
        use bytes::BufMut;
        use tokio::io::{AsyncReadExt, AsyncWriteExt};

        let (mut stream, _) = listener.accept().await.unwrap();

        // Helper: read a length-prefixed Kafka frame
        async fn read_frame(stream: &mut tokio::net::TcpStream) -> Vec<u8> {
            let mut len_buf = [0u8; 4];
            stream.read_exact(&mut len_buf).await.unwrap();
            let len = i32::from_be_bytes(len_buf) as usize;
            let mut body = vec![0u8; len];
            stream.read_exact(&mut body).await.unwrap();
            body
        }

        // Helper: write a length-prefixed Kafka frame
        async fn write_frame(stream: &mut tokio::net::TcpStream, data: &[u8]) {
            let len = data.len() as i32;
            stream.write_all(&len.to_be_bytes()).await.unwrap();
            stream.write_all(data).await.unwrap();
            stream.flush().await.unwrap();
        }

        // 1. Read SaslHandshakeRequest
        let req = read_frame(&mut stream).await;
        // Parse: api_key(2) + api_version(2) + correlation_id(4) = bytes[4..8]
        let correlation_id = i32::from_be_bytes(req[4..8].try_into().unwrap());
        // Parse mechanism name: skip header (api_key + version + corr_id + client_id)
        // client_id is a KafkaString: i16 len + bytes
        let client_id_len = i16::from_be_bytes(req[8..10].try_into().unwrap());
        let mech_offset = if client_id_len < 0 {
            10 // null client_id
        } else {
            10 + client_id_len as usize
        };
        let mech_len =
            i16::from_be_bytes(req[mech_offset..mech_offset + 2].try_into().unwrap()) as usize;
        let mechanism =
            String::from_utf8(req[mech_offset + 2..mech_offset + 2 + mech_len].to_vec()).unwrap();

        // Send SaslHandshakeResponse: correlation_id + error_code(0) + 1 mechanism
        let mut resp = BytesMut::new();
        resp.put_i32(correlation_id);
        resp.put_i16(0); // error_code = NONE
        resp.put_i32(1); // 1 enabled mechanism
        let mech_bytes = mechanism.as_bytes();
        resp.put_i16(mech_bytes.len() as i16);
        resp.put_slice(mech_bytes);
        write_frame(&mut stream, &resp).await;

        // 2. Read SaslAuthenticateRequest
        let req = read_frame(&mut stream).await;
        let correlation_id = i32::from_be_bytes(req[4..8].try_into().unwrap());
        // Parse auth_bytes: skip header, find KafkaBytes (i32 len + bytes)
        let client_id_len = i16::from_be_bytes(req[8..10].try_into().unwrap());
        let auth_offset = if client_id_len < 0 {
            10
        } else {
            10 + client_id_len as usize
        };
        let auth_bytes_len =
            i32::from_be_bytes(req[auth_offset..auth_offset + 4].try_into().unwrap()) as usize;
        let auth_bytes = req[auth_offset + 4..auth_offset + 4 + auth_bytes_len].to_vec();

        // Send SaslAuthenticateResponse v1: correlation_id + error_code(0) + null message + empty bytes + session_lifetime_ms
        let mut resp = BytesMut::new();
        resp.put_i32(correlation_id);
        resp.put_i16(0); // error_code = NONE
        resp.put_i16(-1_i16); // error_message = null (KafkaString)
        resp.put_i32(0); // auth_bytes = empty (KafkaBytes, 0 length)
        resp.put_i64(session_lifetime_ms); // session_lifetime_ms (v1)
        write_frame(&mut stream, &resp).await;

        // 3. Read ApiVersionsRequest
        let req = read_frame(&mut stream).await;
        let correlation_id = i32::from_be_bytes(req[4..8].try_into().unwrap());

        // Send ApiVersionsResponse: correlation_id + error_code(0) + 0 api keys
        let mut resp = BytesMut::new();
        resp.put_i32(correlation_id);
        resp.put_i16(0); // error_code
        resp.put_i32(0); // 0 api keys
        write_frame(&mut stream, &resp).await;

        let _ = shutdown_rx.await;

        (mechanism, auth_bytes)
    }

    #[tokio::test]
    async fn test_sasl_plain_handshake_with_mock_broker() {
        // Start a mock broker
        let listener = tokio::net::TcpListener::bind("127.0.0.1:0").await.unwrap();
        let addr = listener.local_addr().unwrap();
        let addr_str = addr.to_string();

        // Run mock broker in background
        let (shutdown_tx, shutdown_rx) = oneshot::channel();
        let mock_handle = tokio::spawn(run_mock_sasl_broker(listener, shutdown_rx));

        // Connect with SASL/PLAIN auth
        let config = ConnectionConfig::builder()
            .client_id("test-client")
            .auth(crate::auth::AuthConfig::sasl_plain("testuser", "testpassword").unwrap())
            .build();

        let conn = BrokerConnection::connect(&addr_str, config).await;
        assert!(
            conn.is_ok(),
            "Connection with SASL/PLAIN should succeed: {:?}",
            conn.err()
        );

        let conn = conn.unwrap();
        assert!(conn.is_alive());

        conn.close().await;
        let _ = shutdown_tx.send(());

        // Verify the mock received the correct handshake
        let (mechanism, auth_bytes) = mock_handle.await.unwrap();
        assert_eq!(mechanism, "PLAIN");

        // SASL PLAIN format: \0username\0password
        assert_eq!(auth_bytes, b"\0testuser\0testpassword");
    }

    #[tokio::test]
    async fn test_sasl_oauthbearer_provider_handshake_with_mock_broker() {
        use crate::auth::OAuthBearerToken;

        // Start a mock broker
        let listener = tokio::net::TcpListener::bind("127.0.0.1:0").await.unwrap();
        let addr = listener.local_addr().unwrap();
        let addr_str = addr.to_string();

        // Run mock broker in background
        let (shutdown_tx, shutdown_rx) = oneshot::channel();
        let mock_handle = tokio::spawn(run_mock_sasl_broker(listener, shutdown_rx));

        // Connect with OAUTHBEARER provider (not a static token)
        let config = ConnectionConfig::builder()
            .client_id("test-client")
            .auth(crate::auth::AuthConfig::sasl_oauthbearer_provider(
                || async { Ok(OAuthBearerToken::new("provider-jwt-token")) },
            ))
            .build();

        let conn = BrokerConnection::connect(&addr_str, config).await;
        assert!(
            conn.is_ok(),
            "Connection with OAUTHBEARER provider should succeed: {:?}",
            conn.err()
        );

        let conn = conn.unwrap();
        assert!(conn.is_alive());

        conn.close().await;
        let _ = shutdown_tx.send(());

        // Verify the mock received the correct OAUTHBEARER handshake
        let (mechanism, auth_bytes) = mock_handle.await.unwrap();
        assert_eq!(mechanism, "OAUTHBEARER");

        // GS2 format: n,,\x01auth=Bearer <token>\x01\x01
        let expected = OAuthBearerToken::new("provider-jwt-token").to_gs2_initial_response();
        assert_eq!(auth_bytes, expected);
    }

    #[tokio::test]
    async fn test_sasl_oauthbearer_provider_timeout() {
        // Provider that hangs forever
        let config = ConnectionConfig::builder()
            .client_id("test-client")
            .request_timeout(Duration::from_millis(100))
            .auth(crate::auth::AuthConfig::sasl_oauthbearer_provider(
                || async {
                    // Simulate a hung OAuth server
                    tokio::time::sleep(Duration::from_secs(60)).await;
                    Ok(crate::auth::OAuthBearerToken::new("never"))
                },
            ))
            .build();

        // We need a listening socket so TCP connect succeeds before the handshake
        let listener = tokio::net::TcpListener::bind("127.0.0.1:0").await.unwrap();
        let addr = listener.local_addr().unwrap();
        let addr_str = addr.to_string();

        // Accept in background so the connect() doesn't hang
        tokio::spawn(async move {
            let (_stream, _) = listener.accept().await.unwrap();
            // Keep the stream alive so the client side doesn't get a connection reset
            tokio::time::sleep(Duration::from_secs(5)).await;
        });

        let result = BrokerConnection::connect(&addr_str, config).await;
        assert!(
            result.is_err(),
            "Connection should fail when provider times out"
        );
        let err = match result {
            Err(e) => e.to_string(),
            Ok(_) => panic!("Expected error"),
        };
        assert!(
            err.contains("timed out") || err.contains("timeout"),
            "Error should mention timeout: {err}"
        );
    }

    #[tokio::test]
    async fn test_no_sasl_handshake_without_auth() {
        // Start a mock broker that only handles ApiVersionsRequest (no SASL)
        let listener = tokio::net::TcpListener::bind("127.0.0.1:0").await.unwrap();
        let addr = listener.local_addr().unwrap();
        let addr_str = addr.to_string();

        let mock_handle = tokio::spawn(async move {
            use bytes::BufMut;
            use tokio::io::{AsyncReadExt, AsyncWriteExt};

            let (mut stream, _) = listener.accept().await.unwrap();

            // Should receive ApiVersionsRequest directly (no SASL handshake)
            let mut len_buf = [0u8; 4];
            stream.read_exact(&mut len_buf).await.unwrap();
            let len = i32::from_be_bytes(len_buf) as usize;
            let mut body = vec![0u8; len];
            stream.read_exact(&mut body).await.unwrap();

            // Verify it's ApiVersions (api_key = 18), not SaslHandshake (api_key = 17)
            let api_key = i16::from_be_bytes(body[0..2].try_into().unwrap());
            let correlation_id = i32::from_be_bytes(body[4..8].try_into().unwrap());

            // Send ApiVersionsResponse
            let mut resp = BytesMut::new();
            resp.put_i32(correlation_id);
            resp.put_i16(0); // error_code
            resp.put_i32(0); // 0 api keys
            let len = resp.len() as i32;
            stream.write_all(&len.to_be_bytes()).await.unwrap();
            stream.write_all(&resp).await.unwrap();
            stream.flush().await.unwrap();

            api_key
        });

        // Connect without auth
        let config = ConnectionConfig::builder().client_id("test-client").build();

        let conn = BrokerConnection::connect(&addr_str, config).await;
        assert!(conn.is_ok());

        let api_key = mock_handle.await.unwrap();
        assert_eq!(
            api_key, 18,
            "First request without auth should be ApiVersions (18), not SaslHandshake (17)"
        );

        conn.unwrap().close().await;
    }

    #[tokio::test]
    async fn test_sasl_handshake_failure_rejects_connection() {
        // Mock broker that rejects the SASL handshake
        let listener = tokio::net::TcpListener::bind("127.0.0.1:0").await.unwrap();
        let addr = listener.local_addr().unwrap();
        let addr_str = addr.to_string();

        tokio::spawn(async move {
            use bytes::BufMut;
            use tokio::io::{AsyncReadExt, AsyncWriteExt};

            let (mut stream, _) = listener.accept().await.unwrap();

            // Read SaslHandshakeRequest
            let mut len_buf = [0u8; 4];
            stream.read_exact(&mut len_buf).await.unwrap();
            let len = i32::from_be_bytes(len_buf) as usize;
            let mut body = vec![0u8; len];
            stream.read_exact(&mut body).await.unwrap();
            let correlation_id = i32::from_be_bytes(body[4..8].try_into().unwrap());

            // Send error response (unsupported mechanism, error_code = 33)
            let mut resp = BytesMut::new();
            resp.put_i32(correlation_id);
            resp.put_i16(33); // error_code = UNSUPPORTED_SASL_MECHANISM
            resp.put_i32(1); // 1 supported mechanism
            let mech = b"GSSAPI";
            resp.put_i16(mech.len() as i16);
            resp.put_slice(mech);
            let len = resp.len() as i32;
            stream.write_all(&len.to_be_bytes()).await.unwrap();
            stream.write_all(&resp).await.unwrap();
            stream.flush().await.unwrap();
        });

        let config = ConnectionConfig::builder()
            .client_id("test-client")
            .auth(crate::auth::AuthConfig::sasl_plain("user", "pass").unwrap())
            .build();

        let result = BrokerConnection::connect(&addr_str, config).await;
        assert!(
            result.is_err(),
            "Connection should fail when SASL handshake is rejected"
        );
        let err = match result {
            Err(e) => e,
            Ok(_) => panic!("Expected error"),
        };
        assert!(
            err.to_string().contains("SASL handshake failed"),
            "Error should mention SASL handshake failure: {err}"
        );
    }

    #[tokio::test]
    async fn test_sasl_auth_failure_rejects_connection() {
        // Mock broker that accepts handshake but rejects authentication
        let listener = tokio::net::TcpListener::bind("127.0.0.1:0").await.unwrap();
        let addr = listener.local_addr().unwrap();
        let addr_str = addr.to_string();

        tokio::spawn(async move {
            use bytes::BufMut;
            use tokio::io::{AsyncReadExt, AsyncWriteExt};

            let (mut stream, _) = listener.accept().await.unwrap();

            // Helper
            async fn read_frame(stream: &mut tokio::net::TcpStream) -> Vec<u8> {
                let mut len_buf = [0u8; 4];
                stream.read_exact(&mut len_buf).await.unwrap();
                let len = i32::from_be_bytes(len_buf) as usize;
                let mut body = vec![0u8; len];
                stream.read_exact(&mut body).await.unwrap();
                body
            }

            // 1. SaslHandshake — accept
            let req = read_frame(&mut stream).await;
            let correlation_id = i32::from_be_bytes(req[4..8].try_into().unwrap());
            let mut resp = BytesMut::new();
            resp.put_i32(correlation_id);
            resp.put_i16(0); // OK
            resp.put_i32(1);
            let mech = b"PLAIN";
            resp.put_i16(mech.len() as i16);
            resp.put_slice(mech);
            let len = resp.len() as i32;
            stream.write_all(&len.to_be_bytes()).await.unwrap();
            stream.write_all(&resp).await.unwrap();
            stream.flush().await.unwrap();

            // 2. SaslAuthenticate — reject with auth error (v1 format)
            let req = read_frame(&mut stream).await;
            let correlation_id = i32::from_be_bytes(req[4..8].try_into().unwrap());
            let mut resp = BytesMut::new();
            resp.put_i32(correlation_id);
            resp.put_i16(58); // error_code = SASL_AUTHENTICATION_FAILED
            // error_message: "Authentication failed"
            let msg = b"Authentication failed";
            resp.put_i16(msg.len() as i16);
            resp.put_slice(msg);
            resp.put_i32(0); // empty auth_bytes
            resp.put_i64(0); // session_lifetime_ms (v1)
            let len = resp.len() as i32;
            stream.write_all(&len.to_be_bytes()).await.unwrap();
            stream.write_all(&resp).await.unwrap();
            stream.flush().await.unwrap();
        });

        let config = ConnectionConfig::builder()
            .client_id("test-client")
            .auth(crate::auth::AuthConfig::sasl_plain("user", "wrongpass").unwrap())
            .build();

        let result = BrokerConnection::connect(&addr_str, config).await;
        assert!(
            result.is_err(),
            "Connection should fail when authentication is rejected"
        );
        let err = match result {
            Err(e) => e,
            Ok(_) => panic!("Expected error"),
        };
        assert!(
            err.to_string().contains("authentication failed")
                || err.to_string().contains("Authentication failed"),
            "Error should mention auth failure: {err}"
        );
    }

    #[test]
    fn test_connection_config_socket_buffer_sizes() {
        let mut config = ConnectionConfig::default();
        assert!(config.send_buffer_size.is_none());
        assert!(config.recv_buffer_size.is_none());

        config.send_buffer_size = Some(1024 * 1024);
        config.recv_buffer_size = Some(512 * 1024);
        assert_eq!(config.send_buffer_size, Some(1024 * 1024));
        assert_eq!(config.recv_buffer_size, Some(512 * 1024));
    }

    #[tokio::test]
    async fn test_connection_invalid_address_format() {
        let config = ConnectionConfig::default();
        // Invalid address (not a valid SocketAddr) should return an error
        let result = BrokerConnection::connect("not-a-valid-address", config).await;
        assert!(result.is_err());
    }

    #[tokio::test]
    async fn test_read_framed_response_rejects_negative_length() {
        // Simulate a stream that sends a negative i32 as the length prefix
        let data: [u8; 4] = (-1i32).to_be_bytes();
        let mut cursor = std::io::Cursor::new(data);
        let result =
            BrokerConnection::read_framed_response(&mut cursor, crate::protocol::MAX_MESSAGE_SIZE)
                .await;
        assert!(result.is_err(), "negative frame length should be rejected");
        let err_msg = format!("{}", result.unwrap_err());
        assert!(
            err_msg.contains("Invalid response length: -1"),
            "error should show negative value: {err_msg}"
        );
    }

    #[tokio::test]
    async fn test_read_framed_response_rejects_zero_length() {
        let data: [u8; 4] = 0i32.to_be_bytes();
        let mut cursor = std::io::Cursor::new(data);
        let result =
            BrokerConnection::read_framed_response(&mut cursor, crate::protocol::MAX_MESSAGE_SIZE)
                .await;
        assert!(result.is_err(), "zero frame length should be rejected");
    }

    #[tokio::test]
    async fn test_connection_loop_enforces_configured_max_response_size() {
        let (client, mut server) = tokio::io::duplex(256);
        let (reader, writer) = tokio::io::split(client);
        let (_high_tx, high_rx) = mpsc::channel(4);
        let (normal_tx, normal_rx) = mpsc::channel(4);
        let stats = Arc::new(ConnectionStats::default());
        let metrics = Arc::new(ConnectionMetrics::default());

        let loop_task = tokio::spawn(BrokerConnection::run_connection_loop(
            "test-broker".to_string(),
            reader,
            writer,
            high_rx,
            normal_rx,
            Duration::from_secs(30),
            stats,
            metrics,
            16,
            256,
        ));

        let (response_tx, response_rx) = oneshot::channel();
        normal_tx
            .send(ConnectionCommand::Request {
                data: Bytes::from_static(b"ping"),
                correlation_id: 7,
                api_key: ApiKey::Metadata,
                api_version: 0,
                response_tx,
            })
            .await
            .unwrap();

        let mut request = [0u8; 4];
        server.read_exact(&mut request).await.unwrap();
        assert_eq!(&request, b"ping");

        server.write_all(&(32i32).to_be_bytes()).await.unwrap();
        server.write_all(&[0u8; 32]).await.unwrap();
        server.flush().await.unwrap();

        let err = response_rx.await.unwrap().unwrap_err();
        assert!(
            err.to_string()
                .contains("message size 32 exceeds maximum 16"),
            "pending request should receive the configured frame-limit error: {err}"
        );

        let loop_err = loop_task.await.unwrap().unwrap_err();
        assert!(
            loop_err
                .to_string()
                .contains("message size 32 exceeds maximum 16"),
            "connection loop should stop on oversized steady-state frames: {loop_err}"
        );
    }

    #[test]
    fn test_connection_config_default_max_response_size() {
        let config = ConnectionConfig::default();
        assert_eq!(
            config.max_response_size,
            100 * 1024 * 1024,
            "default max_response_size should be MAX_MESSAGE_SIZE (100 MB)"
        );
        assert_eq!(
            config.max_response_size,
            crate::protocol::MAX_MESSAGE_SIZE,
            "default max_response_size should equal protocol::MAX_MESSAGE_SIZE"
        );
    }

    #[tokio::test]
    async fn test_connection_loop_yields_to_normal_priority_after_bypass_budget() {
        use tokio::io::AsyncReadExt;

        let (client, mut server) = tokio::io::duplex(4096);
        let (reader, writer) = tokio::io::split(client);
        let (high_tx, high_rx) = mpsc::channel(16);
        let (normal_tx, normal_rx) = mpsc::channel(16);
        let stats = Arc::new(ConnectionStats::default());
        let metrics = Arc::new(ConnectionMetrics::default());

        for index in 0..8 {
            let (response_tx, _response_rx) = oneshot::channel();
            high_tx
                .try_send(ConnectionCommand::Request {
                    data: Bytes::copy_from_slice(format!("H{index:03}").as_bytes()),
                    correlation_id: index + 1,
                    api_key: ApiKey::Heartbeat,
                    api_version: 0,
                    response_tx,
                })
                .unwrap();
        }

        let (normal_response_tx, _normal_response_rx) = oneshot::channel();
        normal_tx
            .try_send(ConnectionCommand::Request {
                data: Bytes::from_static(b"N000"),
                correlation_id: 100,
                api_key: ApiKey::Produce,
                api_version: 0,
                response_tx: normal_response_tx,
            })
            .unwrap();

        let loop_task = tokio::spawn(BrokerConnection::run_connection_loop(
            "test-broker".to_string(),
            reader,
            writer,
            high_rx,
            normal_rx,
            Duration::from_secs(30),
            stats.clone(),
            metrics.clone(),
            crate::protocol::MAX_MESSAGE_SIZE,
            32,
        ));

        let mut writes = Vec::new();
        for _ in 0..5 {
            let mut frame = [0u8; 4];
            server.read_exact(&mut frame).await.unwrap();
            writes.push(String::from_utf8(frame.to_vec()).unwrap());
        }

        assert_eq!(writes[0], "H000");
        assert_eq!(writes[1], "H001");
        assert_eq!(writes[2], "H002");
        assert_eq!(writes[3], "H003");
        assert_eq!(writes[4], "N000");
        assert_eq!(stats.bypass_yield_count(), 1);
        assert_eq!(metrics.snapshot().high_priority_bypass_yields, 1);

        loop_task.abort();
    }

    #[tokio::test]
    async fn test_connection_loop_rejects_correlation_id_collision() {
        use tokio::io::AsyncReadExt;

        let (client, mut server) = tokio::io::duplex(4096);
        let (reader, writer) = tokio::io::split(client);
        let (_high_tx, high_rx) = mpsc::channel(4);
        let (normal_tx, normal_rx) = mpsc::channel(4);
        let stats = Arc::new(ConnectionStats::default());
        let metrics = Arc::new(ConnectionMetrics::default());

        let loop_task = tokio::spawn(BrokerConnection::run_connection_loop(
            "test-broker".to_string(),
            reader,
            writer,
            high_rx,
            normal_rx,
            Duration::from_secs(30),
            stats,
            metrics,
            crate::protocol::MAX_MESSAGE_SIZE,
            32,
        ));

        let (first_response_tx, first_response_rx) = oneshot::channel();
        normal_tx
            .send(ConnectionCommand::Request {
                data: Bytes::from_static(b"req1"),
                correlation_id: 77,
                api_key: ApiKey::Metadata,
                api_version: 0,
                response_tx: first_response_tx,
            })
            .await
            .unwrap();

        let mut first_write = [0u8; 4];
        server.read_exact(&mut first_write).await.unwrap();
        assert_eq!(&first_write, b"req1");

        let (second_response_tx, second_response_rx) = oneshot::channel();
        normal_tx
            .send(ConnectionCommand::Request {
                data: Bytes::from_static(b"req2"),
                correlation_id: 77,
                api_key: ApiKey::Metadata,
                api_version: 0,
                response_tx: second_response_tx,
            })
            .await
            .unwrap();

        let second_err = second_response_rx.await.unwrap().unwrap_err();
        assert!(second_err.to_string().contains("correlation ID collision"));

        let first_err = first_response_rx.await.unwrap().unwrap_err();
        assert!(first_err.to_string().contains("correlation ID collision"));

        let loop_err = loop_task.await.unwrap().unwrap_err();
        assert!(loop_err.to_string().contains("correlation ID collision"));
    }

    #[test]
    fn test_connection_config_builder_max_response_size() {
        let config = ConnectionConfig::builder()
            .max_response_size(50 * 1024 * 1024)
            .build();
        assert_eq!(
            config.max_response_size,
            50 * 1024 * 1024,
            "max_response_size should be settable via builder"
        );
    }

    #[test]
    fn test_connection_config_builder_max_response_size_minimum() {
        // Setting a value below 1024 should be clamped to 1024
        let config = ConnectionConfig::builder().max_response_size(100).build();
        assert_eq!(
            config.max_response_size, 1024,
            "max_response_size should be clamped to minimum of 1024 bytes"
        );

        let config_zero = ConnectionConfig::builder().max_response_size(0).build();
        assert_eq!(
            config_zero.max_response_size, 1024,
            "max_response_size(0) should clamp to 1024"
        );
    }

    #[tokio::test]
    async fn test_connect_resolves_hostname() {
        // Bind a TCP listener so we have a real port to connect to.
        let listener = tokio::net::TcpListener::bind("127.0.0.1:0").await.unwrap();
        let port = listener.local_addr().unwrap().port();

        // Use "localhost" (a hostname, not an IP) to verify DNS resolution works.
        let hostname_addr = format!("localhost:{port}");
        let config = ConnectionConfig::builder()
            .connect_timeout(Duration::from_secs(2))
            .request_timeout(Duration::from_secs(2))
            .build();

        // The connect will resolve "localhost" via lookup_host, establish TCP,
        // then fail on the ApiVersions handshake because our listener doesn't
        // speak the Kafka protocol — but it must NOT fail with an address
        // parsing error.
        let result = BrokerConnection::connect(&hostname_addr, config).await;
        match result {
            Ok(_) => {} // Unlikely but acceptable — means the mock spoke enough Kafka
            Err(err) => {
                let err_msg = format!("{err}");
                assert!(
                    !err_msg.contains("invalid address"),
                    "should not fail on address resolution, got: {err_msg}"
                );
            }
        }
    }

    #[tokio::test]
    async fn test_connect_dns_failure_is_retriable() {
        let config = ConnectionConfig::builder()
            .connect_timeout(Duration::from_secs(5))
            .build();
        let result =
            BrokerConnection::connect("this-host-does-not-exist.invalid:9092", config).await;
        match result {
            Ok(_) => panic!("connect to non-existent host should fail"),
            Err(err) => {
                assert!(
                    err.is_retriable(),
                    "DNS resolution failure should be retriable (Network), got: {err}"
                );
            }
        }
    }

    #[cfg(feature = "socks5")]
    #[test]
    fn test_proxy_config_new() {
        let proxy = ProxyConfig::new("proxy.example.com:1080");
        assert_eq!(proxy.address(), "proxy.example.com:1080");
        assert!(proxy.credentials().is_none());
    }

    #[cfg(feature = "socks5")]
    #[test]
    fn test_proxy_config_with_credentials() {
        let proxy = ProxyConfig::with_credentials("proxy.example.com:1080", "user", "s3cret");
        assert_eq!(proxy.address(), "proxy.example.com:1080");
        let creds = proxy.credentials().expect("should have credentials");
        assert_eq!(creds.username(), "user");
        assert_eq!(creds.password(), "s3cret");
    }

    #[cfg(feature = "socks5")]
    #[test]
    fn test_proxy_config_debug_redacts_credentials() {
        let proxy = ProxyConfig::with_credentials("proxy.example.com:1080", "admin", "hunter2");
        let debug_str = format!("{proxy:?}");
        assert!(
            debug_str.contains("proxy.example.com:1080"),
            "Debug should contain the address"
        );
        assert!(
            !debug_str.contains("hunter2"),
            "Debug must NOT contain the password"
        );
        assert!(
            debug_str.contains("[REDACTED]"),
            "Debug should show [REDACTED] for credentials"
        );
    }

    #[cfg(feature = "socks5")]
    #[test]
    fn test_proxy_credentials_debug_redacts() {
        let proxy = ProxyConfig::with_credentials("proxy.example.com:1080", "user", "password123");
        let creds = proxy.credentials().expect("should have credentials");
        let debug_str = format!("{creds:?}");
        assert!(
            !debug_str.contains("password123"),
            "Debug must NOT contain the password"
        );
        assert!(
            debug_str.contains("[REDACTED]"),
            "Debug should show [REDACTED]"
        );
    }

    #[cfg(feature = "socks5")]
    #[test]
    fn test_connection_config_builder_with_proxy() {
        let proxy = ProxyConfig::new("socks5.internal:1080");
        let config = ConnectionConfig::builder()
            .client_id("proxy-test")
            .proxy(proxy)
            .build();

        assert!(config.proxy.is_some());
        assert_eq!(
            config.proxy.as_ref().unwrap().address(),
            "socks5.internal:1080"
        );
    }

    #[cfg(feature = "socks5")]
    #[tokio::test]
    async fn test_connect_via_proxy_dns_failure_is_retriable() {
        let proxy = ProxyConfig::new("this-proxy-does-not-exist.invalid:1080");
        let config = ConnectionConfig::builder()
            .connect_timeout(Duration::from_secs(5))
            .proxy(proxy)
            .build();
        let result = BrokerConnection::connect("broker:9092", config).await;
        match result {
            Ok(_) => panic!("connect through non-existent proxy should fail"),
            Err(err) => {
                assert!(
                    err.is_retriable(),
                    "proxy DNS failure should be retriable (Network or Timeout), got: {err}"
                );
            }
        }
    }

    #[cfg(feature = "socks5")]
    #[tokio::test]
    async fn test_connect_via_proxy_stalled_handshake_times_out() {
        let listener = tokio::net::TcpListener::bind("127.0.0.1:0").await.unwrap();
        let proxy = ProxyConfig::new(listener.local_addr().unwrap().to_string());
        let config = ConnectionConfig::builder()
            .connect_timeout(Duration::from_millis(75))
            .proxy(proxy.clone())
            .build();

        let (shutdown_tx, shutdown_rx) = oneshot::channel();
        let mock_proxy = tokio::spawn(async move {
            let (_stream, _) = listener.accept().await.unwrap();
            let _ = shutdown_rx.await;
        });

        let started_at = Instant::now();
        let err = BrokerConnection::connect_via_proxy("broker.internal:9092", &proxy, &config)
            .await
            .unwrap_err();

        assert!(matches!(err, KrafkaError::Timeout { .. }));
        assert!(
            err.to_string().contains("SOCKS5 proxy connection"),
            "timeout should identify the proxy connect path: {err}"
        );
        assert!(
            started_at.elapsed() < Duration::from_secs(1),
            "proxy handshake timeout should respect the configured deadline"
        );

        let _ = shutdown_tx.send(());
        mock_proxy.await.unwrap();
    }

    #[tokio::test]
    async fn test_send_fire_and_forget_uses_reserved_correlation_id() {
        let (high_priority_tx, _high_priority_rx) = mpsc::channel(1);
        let (normal_priority_tx, mut normal_priority_rx) = mpsc::channel(1);
        let conn = BrokerConnection {
            address: "test-broker".to_string(),
            config: ConnectionConfig::default(),
            correlation_id_gen: Arc::new(CorrelationIdGenerator::new()),
            high_priority_tx,
            normal_priority_tx,
            api_versions: Arc::new(parking_lot::Mutex::new(HashMap::new())),
            alive: Arc::new(std::sync::atomic::AtomicBool::new(true)),
            session_expiry: None,
            stats: Arc::new(ConnectionStats::default()),
            throttle_until: Arc::new(parking_lot::Mutex::new(Instant::now())),
            created_at: Instant::now(),
            last_used_nanos: AtomicU64::new(0),
        };

        conn.send_fire_and_forget(ApiKey::Produce, 0, |_| Ok(()))
            .await
            .unwrap();

        let Some(ConnectionCommand::FireAndForget { data }) = normal_priority_rx.recv().await
        else {
            panic!("expected fire-and-forget command");
        };

        let frame_len = i32::from_be_bytes(data[..4].try_into().unwrap()) as usize;
        assert_eq!(frame_len, data.len() - 4);
        let correlation_id = i32::from_be_bytes(data[8..12].try_into().unwrap());
        assert_eq!(correlation_id, NO_RESPONSE_CORRELATION_ID);
        assert_eq!(conn.correlation_id_gen.next(), 1);
    }

    // ========================================================================
    // KIP-368: Session lifetime / reauthentication
    // ========================================================================

    #[test]
    fn test_compute_session_expiry_zero_means_no_expiry() {
        assert!(
            BrokerConnection::compute_session_expiry(0).is_none(),
            "session_lifetime_ms = 0 should mean no expiry"
        );
    }

    #[test]
    fn test_compute_session_expiry_negative_means_no_expiry() {
        assert!(
            BrokerConnection::compute_session_expiry(-1).is_none(),
            "negative session_lifetime_ms should mean no expiry"
        );
    }

    #[test]
    fn test_compute_session_expiry_applies_jittered_margin() {
        let before = Instant::now();
        let expiry = BrokerConnection::compute_session_expiry(10_000).unwrap();
        let after = Instant::now();

        // Randomised window: 85-95% of 10_000ms = 8_500-9_500ms
        let expected_low = before + Duration::from_millis(8_500);
        let expected_high = after + Duration::from_millis(9_500);

        assert!(
            expiry >= expected_low && expiry <= expected_high,
            "expiry should be between 8.5s and 9.5s from now (85-95% of 10s)"
        );
    }

    #[test]
    fn test_compute_session_expiry_jitter_varies() {
        // Call multiple times and verify we don't always get the exact same value.
        // With 10% jitter on a 100s lifetime, outcomes should vary.
        let results: Vec<Instant> = (0..20)
            .map(|_| BrokerConnection::compute_session_expiry(100_000).unwrap())
            .collect();
        let first = results[0];
        let any_different = results.iter().any(|r| *r != first);
        assert!(
            any_different,
            "20 calls should produce at least one different expiry (randomised jitter)"
        );
    }

    #[test]
    fn test_compute_session_expiry_small_lifetime() {
        // Even very short lifetimes should produce a valid expiry
        let expiry = BrokerConnection::compute_session_expiry(100);
        assert!(expiry.is_some(), "100ms lifetime should produce an expiry");
    }

    #[tokio::test]
    async fn test_session_lifetime_tracked_from_broker() {
        // Mock broker that reports a 60-second session lifetime
        let listener = tokio::net::TcpListener::bind("127.0.0.1:0").await.unwrap();
        let addr = listener.local_addr().unwrap().to_string();

        let (shutdown_tx, shutdown_rx) = oneshot::channel();
        let mock_handle = tokio::spawn(run_mock_sasl_broker_with_lifetime(
            listener,
            60_000,
            shutdown_rx,
        ));

        let config = ConnectionConfig::builder()
            .client_id("test-client")
            .auth(crate::auth::AuthConfig::sasl_plain("user", "pass").unwrap())
            .build();

        let conn = BrokerConnection::connect(&addr, config).await.unwrap();

        // The connection should have a session expiry set
        assert!(
            conn.session_expiry().is_some(),
            "session_expiry should be set when broker reports a lifetime"
        );

        // The expiry should be roughly 51-57s from now (85-95% of 60s, randomised)
        let remaining = conn.session_expiry().unwrap() - Instant::now();
        assert!(
            remaining > Duration::from_secs(49) && remaining < Duration::from_secs(58),
            "session expiry should be ~51-57s from now (85-95% of 60s), got {:?}",
            remaining
        );

        // Should not need reauthentication immediately
        assert!(
            !conn.needs_reauthentication(),
            "fresh connection should not need reauthentication"
        );

        // is_usable should be true
        assert!(conn.is_usable(), "fresh connection should be usable");

        conn.close().await;
        let _ = shutdown_tx.send(());
        mock_handle.await.unwrap();
    }

    #[tokio::test]
    async fn test_no_session_expiry_when_lifetime_zero() {
        // Mock broker that reports 0 session lifetime (no expiry)
        let listener = tokio::net::TcpListener::bind("127.0.0.1:0").await.unwrap();
        let addr = listener.local_addr().unwrap().to_string();

        let (shutdown_tx, shutdown_rx) = oneshot::channel();
        let mock_handle =
            tokio::spawn(run_mock_sasl_broker_with_lifetime(listener, 0, shutdown_rx));

        let config = ConnectionConfig::builder()
            .client_id("test-client")
            .auth(crate::auth::AuthConfig::sasl_plain("user", "pass").unwrap())
            .build();

        let conn = BrokerConnection::connect(&addr, config).await.unwrap();

        assert!(
            conn.session_expiry().is_none(),
            "session_expiry should be None when broker reports 0"
        );
        assert!(
            !conn.needs_reauthentication(),
            "should never need reauth with no session lifetime"
        );
        assert!(conn.is_usable());

        conn.close().await;
        let _ = shutdown_tx.send(());
        mock_handle.await.unwrap();
    }

    // ========================================================================
    // KIP-219: Broker throttle compliance
    // ========================================================================

    #[test]
    fn test_throttle_initial_state_is_past() {
        // The throttle deadline starts at `Instant::now()`, so any remaining
        // delay must be effectively zero. Consecutive `Instant::now()` calls
        // can observe the same instant on fast machines, which yields `Some(0)`.
        let deadline = Instant::now();
        let throttle = parking_lot::Mutex::new(deadline);
        let guard = throttle.lock();
        let remaining = guard.checked_duration_since(Instant::now());
        assert!(remaining.unwrap_or_default() <= Duration::from_millis(1));
    }

    #[test]
    fn test_throttle_future_deadline_yields_delay() {
        let future = Instant::now() + Duration::from_secs(10);
        let throttle = parking_lot::Mutex::new(future);
        let guard = throttle.lock();
        let remaining = guard.checked_duration_since(Instant::now());
        assert!(remaining.is_some());
        assert!(remaining.unwrap() > Duration::from_secs(5));
    }

    #[test]
    fn test_throttle_past_deadline_yields_no_delay() {
        // A deadline 1ms in the past means no delay.
        let past = Instant::now() - Duration::from_millis(1);
        let throttle = parking_lot::Mutex::new(past);
        let guard = throttle.lock();
        assert!(guard.checked_duration_since(Instant::now()).is_none());
    }

    #[test]
    fn test_extract_clock_skew_secs_valid_timestamp() {
        // Simulate an AWS error containing a server timestamp.
        // The exact skew depends on when the test runs, but the function
        // should return a non-zero value for a timestamp far from now.
        let msg = "Signature expired: 20200101T000000Z is now past";
        let skew = BrokerConnection::extract_clock_skew_secs(msg);
        // 2020-01-01 is in the past, so skew should be negative.
        assert!(skew < 0, "expected negative skew, got {skew}");
    }

    #[test]
    fn test_extract_clock_skew_secs_no_timestamp() {
        let msg = "some random error message";
        assert_eq!(BrokerConnection::extract_clock_skew_secs(msg), 0);
    }

    #[test]
    fn test_extract_clock_skew_secs_malformed_timestamp() {
        let msg = "Signature expired: 2020XXYYT000000Z";
        assert_eq!(BrokerConnection::extract_clock_skew_secs(msg), 0);
    }

    #[test]
    fn test_extract_clock_skew_secs_invalid_calendar_date() {
        // Month 13 / day 32 / hour 25 — the hand-rolled parser accepted these
        // with range checks; `time` rejects them at parse time.
        assert_eq!(
            BrokerConnection::extract_clock_skew_secs("foo 20201301T000000Z bar"),
            0
        );
        assert_eq!(
            BrokerConnection::extract_clock_skew_secs("foo 20200132T000000Z bar"),
            0
        );
        assert_eq!(
            BrokerConnection::extract_clock_skew_secs("foo 20200101T250000Z bar"),
            0
        );
    }

    #[test]
    fn test_extract_clock_skew_secs_leap_day() {
        // Feb 29 2020 is valid (leap year); Feb 29 2021 is not.
        assert_ne!(
            BrokerConnection::extract_clock_skew_secs("stamp=20200229T120000Z"),
            0
        );
        assert_eq!(
            BrokerConnection::extract_clock_skew_secs("stamp=20210229T120000Z"),
            0
        );
    }

    #[test]
    fn test_extract_clock_skew_secs_embedded_in_longer_message() {
        // Multiple 'T' chars earlier in the message should not fool the scanner.
        let msg = "RequestTime=THIS IS TEXT; expired; server 20200101T000000Z -- request rejected";
        let skew = BrokerConnection::extract_clock_skew_secs(msg);
        assert!(skew < 0);
    }

    #[test]
    fn test_msk_iam_clock_offset_default() {
        let config = ConnectionConfig::default();
        assert_eq!(config.msk_iam_clock_offset_secs.load(Ordering::Relaxed), 0);
    }

    #[test]
    fn test_msk_iam_clock_offset_clamps_to_sigv4_window() {
        assert_eq!(BrokerConnection::clamp_msk_iam_clock_offset_secs(450), 300);
        assert_eq!(
            BrokerConnection::clamp_msk_iam_clock_offset_secs(-450),
            -300
        );
        assert_eq!(BrokerConnection::clamp_msk_iam_clock_offset_secs(120), 120);
    }

    /// An in-flight request must be failed with a timeout error when no
    /// response arrives within `request_timeout`.  This tests the
    /// `DelayQueue`-driven per-request timeout path introduced in the H2 fix.
    #[tokio::test]
    async fn test_request_times_out_when_no_response() {
        let (client, _server) = tokio::io::duplex(4096);
        let (reader, writer) = tokio::io::split(client);
        let (_high_tx, high_rx) = mpsc::channel(4);
        let (normal_tx, normal_rx) = mpsc::channel(4);
        let stats = Arc::new(ConnectionStats::default());
        let metrics = Arc::new(ConnectionMetrics::default());

        // Very short timeout so the test completes quickly.
        let request_timeout = Duration::from_millis(50);

        tokio::spawn(BrokerConnection::run_connection_loop(
            "test-broker".to_string(),
            reader,
            writer,
            high_rx,
            normal_rx,
            request_timeout,
            stats,
            metrics,
            crate::protocol::MAX_MESSAGE_SIZE,
            256,
        ));

        let (response_tx, response_rx) = oneshot::channel();
        normal_tx
            .send(ConnectionCommand::Request {
                // Minimal 4-byte payload; the server side (_server) never replies.
                data: Bytes::from_static(b"test"),
                correlation_id: 42,
                api_key: ApiKey::Produce,
                api_version: 0,
                response_tx,
            })
            .await
            .unwrap();

        let err = response_rx.await.unwrap().unwrap_err();
        assert!(
            err.to_string().contains("timed out"),
            "expected timeout error, got: {err}"
        );
    }

    /// A response that arrives before the deadline must cancel the timer so
    /// no spurious timeout error is delivered after the successful response.
    #[tokio::test]
    async fn test_response_cancels_timeout() {
        use tokio::io::{AsyncReadExt, AsyncWriteExt};

        let (client, mut server) = tokio::io::duplex(4096);
        let (reader, writer) = tokio::io::split(client);
        let (_high_tx, high_rx) = mpsc::channel(4);
        let (normal_tx, normal_rx) = mpsc::channel(4);
        let stats = Arc::new(ConnectionStats::default());
        let metrics = Arc::new(ConnectionMetrics::default());

        // Long enough to not fire during the test.
        let request_timeout = Duration::from_secs(5);
        let correlation_id: i32 = 99;

        tokio::spawn(BrokerConnection::run_connection_loop(
            "test-broker".to_string(),
            reader,
            writer,
            high_rx,
            normal_rx,
            request_timeout,
            stats,
            metrics,
            crate::protocol::MAX_MESSAGE_SIZE,
            256,
        ));

        let (response_tx, response_rx) = oneshot::channel();
        normal_tx
            .send(ConnectionCommand::Request {
                data: Bytes::from_static(b"test"),
                correlation_id,
                api_key: ApiKey::Produce,
                api_version: 0,
                response_tx,
            })
            .await
            .unwrap();

        // Drain the request bytes the loop wrote to the wire.
        let mut buf = [0u8; 4];
        server.read_exact(&mut buf).await.unwrap();

        // Send a valid response: 4-byte length prefix + 4-byte correlation_id.
        // For Produce v0, ResponseHeader v0 = correlation_id only (4 bytes).
        let body = correlation_id.to_be_bytes();
        server.write_all(&(4i32).to_be_bytes()).await.unwrap();
        server.write_all(&body).await.unwrap();
        server.flush().await.unwrap();

        let result = response_rx.await.unwrap();
        assert!(
            result.is_ok(),
            "expected successful response before timeout, got: {:?}",
            result.unwrap_err()
        );
    }
}