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
//! Transactional producer for exactly-once semantics.
//!
//! The transactional producer enables atomic writes across multiple partitions
//! and topics. It guarantees that either all messages in a transaction are
//! committed or none are.
//!
//! # Transaction State and Recovery
//!
//! Transaction state (`TransactionState`) is held in-memory only. This is the
//! **expected and correct behavior** because:
//!
//! 1. **Broker-side coordination**: The transaction coordinator on the broker
//! side maintains the authoritative transaction state for each `transactional.id()`.
//!
//! 2. **Fencing**: When a new producer starts with the same `transactional.id()`,
//! the broker:
//! - Increments the producer epoch
//! - Aborts any pending (uncommitted) transactions from the old producer
//! - Issues a new Producer ID to the new producer
//!
//! 3. **Zombie fencing**: If the old producer tries to continue a transaction
//! after the new producer has started, it receives `ProducerFenced` error.
//!
//! ## Recovery Behavior
//!
//! On producer crash/restart:
//! - Any uncommitted transaction is automatically aborted by the broker
//! (after `transaction.timeout.ms` expires, or when a new producer with
//! the same `transactional.id()` calls `init_transactions()`)
//! - The new producer starts fresh with a new epoch
//! - No manual recovery is needed
//!
//! This matches the Kafka Java client behavior and Kafka's transaction protocol.
//!
//! # Example
//!
//! ```ignore
//! use krafka::producer::TransactionalProducer;
//!
//! let producer = TransactionalProducer::builder()
//! .bootstrap_servers("localhost:9092")
//! .transactional_id("my-transaction")
//! .build()
//! .await?;
//!
//! producer.init_transactions().await?;
//!
//! producer.begin_transaction()?;
//! producer.send("topic", Some(b"key"), b"value").await?;
//! producer.commit_transaction().await?;
//! ```
use std::future::Future;
use std::sync::Arc;
use std::sync::atomic::{AtomicBool, AtomicU8, Ordering};
use std::time::Duration;
use bytes::{BufMut as _, Bytes};
use tokio::sync::{Notify, RwLock};
use tracing::{debug, info, warn};
use crate::auth::AuthConfig;
use crate::error::{ErrorCode, KrafkaError, ProtocolErrorKind, Result};
use crate::metadata::ClusterMetadata;
use crate::network::{BrokerConnection, ConnectionConfig, ConnectionPool};
use crate::protocol::{
AddOffsetsToTxnRequest, AddOffsetsToTxnResponse, AddPartitionsToTxnRequest,
AddPartitionsToTxnResponse, ApiKey, Compression, EndTxnRequest, EndTxnResponse,
FindCoordinatorRequest, FindCoordinatorResponse, InitProducerIdRequest, InitProducerIdResponse,
ProducePartitionData, ProduceRequest, ProduceResponse, ProduceTopicData, RecordBatchBuilder,
TxnOffsetCommitRequest, TxnOffsetCommitResponse, VersionedDecode, VersionedEncode, versions,
};
use crate::{Offset, PartitionId};
use super::barrier::InFlightBarrier;
use super::config::Acks;
use super::idempotent::ProducerIdentity;
use super::partitioner::{DefaultPartitioner, Partitioner};
use super::record::{ProducerRecord, RecordMetadata, RoutedRecord, TopicHandle};
use super::retry::RetryPolicy;
use crate::schema_registry::SchemaEncoder;
/// Transaction state machine states.
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
#[non_exhaustive]
#[repr(u8)]
pub enum TransactionState {
/// Producer not yet initialized.
Uninitialized = 0,
/// Ready to begin a new transaction.
Ready = 1,
/// Transaction is in progress.
InTransaction = 2,
/// Transaction is committing.
Committing = 3,
/// Transaction is aborting.
Aborting = 4,
/// Fatal error occurred, producer must be recreated.
FatalError = 5,
/// Initialization in progress (prevents concurrent init_transactions calls).
Initializing = 6,
}
impl From<u8> for TransactionState {
fn from(v: u8) -> Self {
match v {
0 => Self::Uninitialized,
1 => Self::Ready,
2 => Self::InTransaction,
3 => Self::Committing,
4 => Self::Aborting,
5 => Self::FatalError,
6 => Self::Initializing,
_ => {
warn!(
discriminant = v,
"unknown TransactionState discriminant — treating as FatalError"
);
Self::FatalError
}
}
}
}
impl std::fmt::Display for TransactionState {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
f.write_str(match self {
Self::Uninitialized => "Uninitialized",
Self::Ready => "Ready",
Self::InTransaction => "InTransaction",
Self::Committing => "Committing",
Self::Aborting => "Aborting",
Self::FatalError => "FatalError",
Self::Initializing => "Initializing",
})
}
}
/// A topic-partition offset used with [`TransactionalProducer::send_offsets_to_transaction`].
///
/// The [`next_offset`](TopicPartitionOffset::next_offset) field must be
/// `last_consumed_offset + 1`, which matches the value returned by
/// [`Consumer::position`](crate::consumer::Consumer::position). Kafka commits
/// this value as the next offset the consumer group will start reading from,
/// so an off-by-one here permanently shifts the group's position.
#[derive(Debug, Clone, PartialEq, Eq)]
#[non_exhaustive]
pub struct TopicPartitionOffset {
/// Topic name.
pub topic: String,
/// Partition ID.
pub partition: PartitionId,
/// The **next** offset to be consumed (`last_consumed_offset + 1`).
pub next_offset: Offset,
}
impl TopicPartitionOffset {
/// Construct a new `TopicPartitionOffset`.
pub fn new(topic: impl Into<String>, partition: PartitionId, next_offset: Offset) -> Self {
Self {
topic: topic.into(),
partition,
next_offset,
}
}
}
/// Configuration for a transactional producer.
///
/// Use [`TransactionalProducer::builder()`] to construct. Direct field construction
/// is intentionally not supported to enforce invariant validation at build time.
#[derive(Debug, Clone)]
pub struct TransactionalProducerConfig {
/// Bootstrap servers.
bootstrap_servers: String,
/// Client ID.
client_id: String,
/// Transactional ID (required for transactions).
transactional_id: String,
/// Transaction timeout in milliseconds.
transaction_timeout_ms: i32,
/// Request timeout.
request_timeout: Duration,
/// Maximum encoded Kafka request frame size in bytes.
max_request_size: usize,
/// Compression.
compression: Compression,
/// Metadata max age.
metadata_max_age: Duration,
/// Authentication configuration.
auth: Option<AuthConfig>,
/// SOCKS5 proxy configuration (optional).
#[cfg(feature = "socks5")]
proxy: Option<crate::network::ProxyConfig>,
}
impl Default for TransactionalProducerConfig {
fn default() -> Self {
Self {
bootstrap_servers: String::new(),
client_id: "krafka-txn-producer".to_string(),
transactional_id: String::new(),
transaction_timeout_ms: 60000,
request_timeout: Duration::from_secs(30),
max_request_size: crate::protocol::MAX_MESSAGE_SIZE,
compression: Compression::None,
metadata_max_age: Duration::from_secs(300),
auth: None,
#[cfg(feature = "socks5")]
proxy: None,
}
}
}
/// State of a partition within the current transaction.
#[derive(Debug, Clone)]
enum PartitionAddState {
/// AddPartitionsToTxn RPC is in-flight; concurrent callers should wait.
Pending(Arc<Notify>),
/// Successfully registered with the transaction coordinator.
Added,
/// RPC failed with a non-retriable error. Waiters should propagate this
/// error immediately rather than making a redundant retry RPC.
Failed(Arc<KrafkaError>),
}
/// Result of attempting to begin adding a partition to the transaction.
#[cfg_attr(test, derive(Debug))]
enum BeginAddResult {
/// Partition already registered — nothing to do.
AlreadyAdded,
/// Another caller is registering this partition — wait on the Notify.
Wait(Arc<Notify>),
/// This caller must perform the RPC. Notify to signal waiters afterwards.
NeedAdd(Arc<Notify>),
/// A previous non-retriable RPC failure was recorded for this partition.
/// The caller should return this error without attempting the RPC again.
Fatal(Arc<KrafkaError>),
}
/// Partitions added to the current transaction.
#[derive(Debug, Default)]
struct TransactionPartitions {
/// Topic-partitions and their registration state (topic → partition → state).
partitions: std::collections::HashMap<
String,
std::collections::HashMap<PartitionId, PartitionAddState>,
>,
}
impl TransactionPartitions {
/// Begin adding a partition. Returns the action the caller must take.
fn begin_add(&mut self, topic: &str, partition: PartitionId) -> BeginAddResult {
if let Some(topic_map) = self.partitions.get(topic) {
match topic_map.get(&partition) {
Some(PartitionAddState::Added) => return BeginAddResult::AlreadyAdded,
Some(PartitionAddState::Pending(notify)) => {
return BeginAddResult::Wait(notify.clone());
}
Some(PartitionAddState::Failed(err)) => {
return BeginAddResult::Fatal(err.clone());
}
None => {}
}
}
let notify = Arc::new(Notify::new());
self.partitions
.entry(topic.to_string())
.or_default()
.insert(partition, PartitionAddState::Pending(notify.clone()));
BeginAddResult::NeedAdd(notify)
}
/// Confirm a partition was successfully registered.
fn confirm_add(&mut self, topic: &str, partition: PartitionId, notify: &Notify) {
self.partitions
.entry(topic.to_string())
.or_default()
.insert(partition, PartitionAddState::Added);
notify.notify_waiters();
}
/// Cancel a pending add due to a retriable / transient error.
///
/// Removes the partition entry so that waiters can retry the RPC
/// themselves on the next loop iteration.
fn cancel_add(&mut self, topic: &str, partition: PartitionId, notify: &Notify) {
if let Some(topic_map) = self.partitions.get_mut(topic) {
topic_map.remove(&partition);
if topic_map.is_empty() {
self.partitions.remove(topic);
}
}
notify.notify_waiters();
}
/// Record a non-retriable RPC failure for this partition.
///
/// Stores a `Failed` sentinel so that concurrent waiters receive the
/// error immediately via [`BeginAddResult::Fatal`] rather than making
/// a redundant retry RPC that will also fail.
fn fail_add(
&mut self,
topic: &str,
partition: PartitionId,
error: Arc<KrafkaError>,
notify: &Notify,
) {
self.partitions
.entry(topic.to_string())
.or_default()
.insert(partition, PartitionAddState::Failed(error));
notify.notify_waiters();
}
fn clear(&mut self) {
self.partitions.clear();
}
#[cfg(test)]
fn is_empty(&self) -> bool {
self.partitions.is_empty()
}
}
/// RAII guard that cancels a pending partition add if dropped without confirmation.
///
/// When the task performing the `AddPartitionsToTxn` RPC is cancelled (e.g.,
/// via `select!` or `timeout`), this guard ensures the partition is rolled back
/// from `Pending` to absent so that future callers can retry rather than
/// waiting on a `Notify` that will never fire.
struct PendingAddGuard {
txn_partitions: Arc<RwLock<TransactionPartitions>>,
topic: TopicHandle,
partition: PartitionId,
notify: Arc<Notify>,
/// Set to `true` when `confirm_add` or an explicit `cancel_add` is called,
/// preventing the drop impl from double-cancelling.
defused: bool,
}
impl PendingAddGuard {
/// Confirm the add succeeded. Consumes the guard without cancelling.
async fn confirm(mut self, topic: &str, partition: PartitionId) {
self.defused = true;
let mut txn_partitions = self.txn_partitions.write().await;
txn_partitions.confirm_add(topic, partition, &self.notify);
}
/// Explicitly cancel the add after a **retriable** error.
///
/// Removes the partition entry so that concurrent waiters can retry the
/// RPC on the next loop iteration.
async fn cancel(mut self, topic: &str, partition: PartitionId) {
self.defused = true;
let mut txn_partitions = self.txn_partitions.write().await;
txn_partitions.cancel_add(topic, partition, &self.notify);
}
/// Record a **non-retriable** failure for this partition.
///
/// Stores a `Failed` sentinel so that concurrent waiters receive the
/// error immediately instead of making an extra RPC that will also fail.
async fn fail(mut self, topic: &str, partition: PartitionId, error: Arc<KrafkaError>) {
self.defused = true;
let mut txn_partitions = self.txn_partitions.write().await;
txn_partitions.fail_add(topic, partition, error, &self.notify);
}
}
impl Drop for PendingAddGuard {
fn drop(&mut self) {
if !self.defused {
// Best-effort cancel: we can't await the lock in drop, so first
// try a non-blocking write. If the lock is contended and a Tokio
// runtime is available, spawn a task to perform the cancel.
let topic = self.topic.clone();
let partition = self.partition;
let notify = self.notify.clone();
if let Ok(mut tp) = self.txn_partitions.try_write() {
tp.cancel_add(&topic, partition, ¬ify);
} else if let Ok(handle) = tokio::runtime::Handle::try_current() {
let txn_partitions = self.txn_partitions.clone();
// Note: during runtime shutdown the spawned task may be
// cancelled before it runs. This is acceptable because
// the transaction state is ephemeral to the producer
// instance and will be abandoned on shutdown.
handle.spawn(async move {
let mut tp = txn_partitions.write().await;
tp.cancel_add(&topic, partition, ¬ify);
});
} else {
// No runtime available — use blocking write as last resort.
// This is safe because Handle::try_current() confirmed we are
// NOT on a runtime thread, so blocking_write() won't panic.
let mut tp = self.txn_partitions.blocking_write();
tp.cancel_add(&topic, partition, ¬ify);
}
}
}
}
/// A transactional Kafka producer.
///
/// Provides exactly-once semantics through transactions.
pub struct TransactionalProducer {
/// Configuration.
config: TransactionalProducerConfig,
/// Cluster metadata.
metadata: Arc<ClusterMetadata>,
/// Connection pool.
pool: Arc<ConnectionPool>,
/// Partitioner.
partitioner: Arc<dyn Partitioner>,
/// Transaction state.
state: AtomicU8,
/// Whether the current transaction hit an abortable error and must be
/// aborted before further send/commit operations are allowed.
abort_required: AtomicBool,
/// Transaction coordinator broker ID.
///
/// # Lock ordering
///
/// When both `coordinator_id` and `txn_partitions` are acquired in the
/// same task, always acquire `coordinator_id` first to avoid deadlocks.
coordinator_id: RwLock<Option<i32>>,
/// Partitions in current transaction.
///
/// Always acquired **after** `coordinator_id` (see lock-order note above).
txn_partitions: Arc<RwLock<TransactionPartitions>>,
/// Sequence number tracking for idempotent production.
identity: ProducerIdentity,
/// Retry policy for transient failures.
retry_policy: RetryPolicy,
/// Barrier over started transactional operations and shutdown state.
in_flight_barrier: Arc<InFlightBarrier>,
/// Optional key encoder applied transparently in `send_record`.
///
/// Equivalent to `key.serializer` in the Java `KafkaProducer`.
key_encoder: Option<Arc<dyn SchemaEncoder>>,
/// Optional value encoder applied transparently in `send_record`.
///
/// Equivalent to `value.serializer` in the Java `KafkaProducer`.
value_encoder: Option<Arc<dyn SchemaEncoder>>,
}
impl TransactionalProducer {
/// Create a new transactional producer builder.
pub fn builder() -> TransactionalProducerBuilder {
TransactionalProducerBuilder::default()
}
/// Get the current transaction state.
#[inline]
pub fn state(&self) -> TransactionState {
TransactionState::from(self.state.load(Ordering::SeqCst))
}
/// Return the transactional producer identity, failing fast when
/// `init_transactions()` has not established a valid PID/epoch yet.
fn checked_transactional_identity(&self) -> Result<(i64, i16)> {
let producer_id = self.identity.producer_id();
let producer_epoch = self.identity.producer_epoch();
if producer_id < 0 || producer_epoch < 0 {
return Err(KrafkaError::invalid_state(
"transactional producer identity not initialized",
));
}
debug_assert!(
producer_id >= 0 && producer_epoch >= 0,
"transactional producer identity must be initialized before sending"
);
Ok((producer_id, producer_epoch))
}
#[inline]
fn abort_required(&self) -> bool {
self.abort_required.load(Ordering::SeqCst)
}
fn ensure_transaction_can_continue(&self, operation: &str) -> Result<()> {
if self.abort_required() {
return Err(KrafkaError::broker(
ErrorCode::TransactionAbortable,
format!("cannot {operation}: abort_transaction() is required before continuing"),
));
}
Ok(())
}
fn mark_unknown_producer_id_abort_required(&self, operation: &str) -> KrafkaError {
self.abort_required.store(true, Ordering::SeqCst);
KrafkaError::broker(
ErrorCode::TransactionAbortable,
format!(
"{operation} failed with UnknownProducerId; abort_transaction() is required before continuing"
),
)
}
fn is_unknown_producer_id_error(error: &KrafkaError) -> bool {
matches!(
error,
KrafkaError::Broker {
code: ErrorCode::UnknownProducerId,
..
}
)
}
fn is_abortable_transaction_error(error: &KrafkaError) -> bool {
matches!(
error,
KrafkaError::Broker {
code: ErrorCode::TransactionAbortable,
..
}
)
}
/// Get a connection to the cached transaction coordinator.
///
/// If no coordinator is cached (e.g. after invalidation), automatically
/// re-discovers it via `FindCoordinator` before returning the connection.
async fn coordinator_connection(&self) -> Result<(i32, Arc<BrokerConnection>)> {
let coordinator_id = {
let cached = *self.coordinator_id.read().await;
match cached {
Some(id) => id,
None => {
let id = self.find_coordinator().await?;
*self.coordinator_id.write().await = Some(id);
debug!("Auto-discovered transaction coordinator: broker {}", id);
id
}
}
};
let brokers = self.metadata.brokers();
let broker = brokers
.iter()
.find(|b| b.id() == coordinator_id)
.ok_or_else(|| {
KrafkaError::protocol_kind(
ProtocolErrorKind::Malformed,
"coordinator not found in metadata",
)
})?;
let conn = self
.pool
.get_connection_by_id(broker.id(), broker.address())
.await?;
Ok((coordinator_id, conn))
}
/// Whether the error indicates the cached coordinator may be stale.
///
/// Returns `true` for coordinator-related broker errors (`NotCoordinator`,
/// `CoordinatorNotAvailable`, `CoordinatorLoadInProgress`) and for
/// network/timeout errors that suggest the coordinator broker is unreachable.
fn needs_coordinator_refresh(err: &KrafkaError) -> bool {
match err {
KrafkaError::Broker { code, .. } => matches!(
code,
ErrorCode::NotCoordinator
| ErrorCode::CoordinatorNotAvailable
| ErrorCode::CoordinatorLoadInProgress
),
KrafkaError::Network(_) | KrafkaError::Timeout { .. } => true,
_ => false,
}
}
/// Invalidate the cached transaction coordinator, forcing re-discovery
/// on the next coordinator RPC.
async fn invalidate_coordinator(&self) {
*self.coordinator_id.write().await = None;
}
/// Retry a coordinator RPC with exponential backoff.
///
/// On coordinator errors (`NotCoordinator`, `CoordinatorNotAvailable`,
/// `CoordinatorLoadInProgress`) or transient network/timeout failures the
/// cached coordinator is invalidated and re-discovered before the next
/// attempt. Non-retriable errors are returned immediately.
///
/// `op_name` is used in log messages to identify the RPC.
async fn retry_with_coordinator<F, Fut>(&self, op_name: &str, op: F) -> Result<()>
where
F: Fn() -> Fut,
Fut: Future<Output = Result<()>>,
{
let max_retries = self.retry_policy.max_retries;
for attempt in 0..=max_retries {
if attempt > 0 {
tokio::time::sleep(self.retry_policy.calculate_backoff(attempt)).await;
}
let result = op().await;
match &result {
Ok(()) => return Ok(()),
Err(e) if Self::is_unknown_producer_id_error(e) => return result,
Err(e) if Self::needs_coordinator_refresh(e) && attempt < max_retries => {
warn!(
attempt,
error = %e,
op_name,
"Coordinator error, refreshing and retrying"
);
self.invalidate_coordinator().await;
}
Err(e) if e.is_retriable() && attempt < max_retries => {
warn!(
attempt,
error = %e,
op_name,
"Retriable error, retrying"
);
}
Err(_) => return result,
}
}
Err(KrafkaError::protocol_kind(
ProtocolErrorKind::Malformed,
format!("{op_name} retry loop exhausted after {max_retries} retries"),
))
}
fn set_state(&self, state: TransactionState) {
self.state.store(state as u8, Ordering::SeqCst);
}
/// Atomically transition from `expected` to `new` state.
/// Returns `Err` with the actual state if the CAS failed.
fn try_transition(
&self,
expected: TransactionState,
new: TransactionState,
) -> std::result::Result<(), TransactionState> {
// AcqRel on success: the stored new state is Released (visible to
// readers), and we Acquire the current state (see any prior writes).
// Acquire on failure: we Acquire the actual current state so callers
// can act on it without a separate load. All downstream transaction
// data is behind an async Mutex, so no stronger ordering is needed.
self.state
.compare_exchange(
expected as u8,
new as u8,
Ordering::AcqRel,
Ordering::Acquire,
)
.map(|_| ())
.map_err(TransactionState::from)
}
/// Initialize transactions.
///
/// This must be called before any transactions can be started.
/// It fetches the producer ID and epoch from the transaction coordinator.
pub async fn init_transactions(&self) -> Result<()> {
// Atomic CAS: Uninitialized → Initializing
if let Err(actual) = self.try_transition(
TransactionState::Uninitialized,
TransactionState::Initializing,
) {
return Err(KrafkaError::invalid_state(format!(
"init_transactions can only be called once (state={:?})",
actual
)));
}
// Find transaction coordinator
let result = self.do_init_transactions().await;
if result.is_err() {
// Revert state so caller can retry
self.set_state(TransactionState::Uninitialized);
}
result
}
/// Inner initialization logic, separated for clean error handling.
///
/// Retries on coordinator errors (NotCoordinator, CoordinatorNotAvailable,
/// CoordinatorLoadInProgress) and transient network/timeout failures with
/// exponential backoff. On each retry the cached coordinator is invalidated
/// and re-discovered via `FindCoordinator`.
async fn do_init_transactions(&self) -> Result<()> {
self.retry_with_coordinator("InitProducerId", || async {
let (_coordinator_id, conn) = self.coordinator_connection().await?;
let ip_version = conn
.negotiate_api_version(
ApiKey::InitProducerId,
versions::INIT_PRODUCER_ID_MAX,
versions::INIT_PRODUCER_ID_MIN,
)
.await
.ok_or_else(|| {
KrafkaError::protocol_kind(
ProtocolErrorKind::UnknownApiVersion,
"no mutually supported InitProducerId API version",
)
})?;
let request = InitProducerIdRequest::transactional(
&self.config.transactional_id,
self.config.transaction_timeout_ms,
);
let response_bytes = conn
.send_request(ApiKey::InitProducerId, ip_version, |buf| {
request.encode_versioned(ip_version, buf)
})
.await?;
let mut buf = response_bytes;
let response = InitProducerIdResponse::decode_versioned(ip_version, &mut buf)?;
if !response.is_ok() {
return Err(KrafkaError::broker(
response.error_code,
"failed to initialize producer ID",
));
}
self.identity
.initialize(response.producer_id, response.producer_epoch);
self.abort_required.store(false, Ordering::SeqCst);
self.set_state(TransactionState::Ready);
info!(
"Transactional producer initialized: PID={}, epoch={}",
response.producer_id, response.producer_epoch
);
Ok(())
})
.await
}
/// Find the transaction coordinator.
async fn find_coordinator(&self) -> Result<i32> {
let brokers = self.metadata.brokers();
if brokers.is_empty() {
return Err(KrafkaError::protocol_kind(
ProtocolErrorKind::Malformed,
"no brokers available",
));
}
let broker = &brokers[0];
let conn = self
.pool
.get_connection_by_id(broker.id(), broker.address())
.await?;
let request = FindCoordinatorRequest::for_transaction(&self.config.transactional_id);
// Transaction coordinator lookup requires v1+ (key_type field).
// FIND_COORDINATOR_MIN is 1, so negotiate_api_version returns None
// (handled above) rather than v0 when the broker lacks v1+.
let fc_version = conn
.negotiate_api_version(
ApiKey::FindCoordinator,
versions::FIND_COORDINATOR_MAX,
versions::FIND_COORDINATOR_MIN,
)
.await
.ok_or_else(|| {
KrafkaError::protocol_kind(
ProtocolErrorKind::UnknownApiVersion,
"no mutually supported FindCoordinator API version; \
transactional coordinator lookup requires v1+",
)
})?;
let response_bytes = conn
.send_request(ApiKey::FindCoordinator, fc_version, |buf| {
request.encode_versioned(fc_version, buf)
})
.await?;
let mut buf = response_bytes;
let response = FindCoordinatorResponse::decode_versioned(fc_version, &mut buf)?;
if !response.error_code.is_ok() {
return Err(KrafkaError::broker(
response.error_code,
"failed to find transaction coordinator",
));
}
debug!(
"Found transaction coordinator: broker {} at {}:{}",
response.node_id, response.host, response.port
);
Ok(response.node_id)
}
/// Begin a new transaction.
///
/// Must be called after `init_transactions()`.
/// Begin a new transaction.
///
/// Transitions the producer from `Ready` to `InTransaction` state. Must be
/// called after [`init_transactions`](Self::init_transactions) and before
/// any [`send`](Self::send) calls.
///
/// # Non-blocking
///
/// This method is **synchronous and guaranteed non-blocking** — it performs
/// only an in-memory atomic state transition with no I/O. It is intentionally
/// not `async` for two reasons: it never waits on the network, and this
/// matches the Java `KafkaProducer.beginTransaction()` API.
///
/// # Errors
///
/// Returns `Err` if the producer is not in `Ready` state (e.g. not yet
/// initialised, or a previous transaction was not committed or aborted).
pub fn begin_transaction(&self) -> Result<()> {
// Atomic CAS: Ready → InTransaction
if let Err(actual) =
self.try_transition(TransactionState::Ready, TransactionState::InTransaction)
{
return Err(KrafkaError::invalid_state(format!(
"cannot begin transaction in state {:?}",
actual
)));
}
debug!("Transaction started");
Ok(())
}
/// Send a record within the current transaction.
pub async fn send(
&self,
topic: &str,
key: Option<&[u8]>,
value: &[u8],
) -> Result<RecordMetadata> {
let mut record = ProducerRecord::new(topic, Bytes::copy_from_slice(value));
if let Some(k) = key {
record = record.with_key(Bytes::copy_from_slice(k));
}
self.send_record(record).await
}
/// Send a producer record within the current transaction.
pub async fn send_record(&self, record: ProducerRecord) -> Result<RecordMetadata> {
let _operation_guard = self.in_flight_barrier.start("transactional producer")?;
let current = self.state();
if current != TransactionState::InTransaction {
return Err(KrafkaError::invalid_state(format!(
"cannot send in state {:?}",
current
)));
}
self.ensure_transaction_can_continue("send records")?;
// Transparently apply producer-level schema encoders if configured.
let mut record = record;
if let Some(enc) = &self.value_encoder {
record.value = enc
.encode(
record.value.clone(),
&record.topic,
record.record_name.as_deref(),
false,
)
.await?;
}
if let Some(enc) = &self.key_encoder {
let key = record.key.clone().unwrap_or_default();
record.key = Some(
enc.encode(key, &record.topic, record.record_name.as_deref(), true)
.await?,
);
}
// Validate record fields against Kafka protocol wire-format limits.
record.validate()?;
let _identity = self.checked_transactional_identity()?;
let routed = record.into_routed_parts();
let topic = routed.topic;
let record = routed.record;
// Determine partition
let partition = match routed.partition {
Some(p) => p,
None => {
let partition_count = self
.metadata
.partition_count(topic.as_ref())
.ok_or_else(|| KrafkaError::invalid_state(format!("unknown topic: {topic}")))?;
self.partitioner
.partition(topic.as_ref(), record.key_bytes(), partition_count)
}
};
// Add partition to transaction if not already registered.
// Uses Pending/Added states to prevent concurrent callers from
// skipping the RPC while an in-flight add has not yet completed.
loop {
let mut txn_partitions = self.txn_partitions.write().await;
match txn_partitions.begin_add(topic.as_ref(), partition) {
BeginAddResult::AlreadyAdded => break,
BeginAddResult::Fatal(err) => {
// A previous non-retriable RPC failure was stored for this
// partition. Return it immediately — no retry RPC.
return Err((*err).clone());
}
BeginAddResult::Wait(notify) => {
// Register interest in the Notify BEFORE releasing the
// write lock so that confirm_add/cancel_add/fail_add
// (which use notify_waiters) cannot be missed.
let notified = notify.notified();
tokio::pin!(notified);
notified.as_mut().enable();
drop(txn_partitions);
notified.await;
// Re-check state on next iteration: either AlreadyAdded
// (RPC succeeded), Fatal (RPC failed non-retriably), or
// NeedAdd (RPC failed retriably — this caller retries).
}
BeginAddResult::NeedAdd(notify) => {
// Drop the lock before the RPC. The guard ensures that
// if this task is cancelled, the Pending state is rolled
// back so waiters don't hang forever.
drop(txn_partitions);
let guard = PendingAddGuard {
txn_partitions: self.txn_partitions.clone(),
topic: topic.clone(),
partition,
notify,
defused: false,
};
match self.add_partition_to_txn(topic.as_ref(), partition).await {
Ok(()) => {
guard.confirm(topic.as_ref(), partition).await;
}
Err(e) if e.is_retriable() => {
// Retriable error: remove the entry so that
// concurrent waiters can retry the RPC themselves.
guard.cancel(topic.as_ref(), partition).await;
return Err(e);
}
Err(e) => {
// Non-retriable error: store it so that any
// concurrent waiters receive it immediately.
guard
.fail(topic.as_ref(), partition, Arc::new(e.clone()))
.await;
return Err(e);
}
}
break;
}
}
}
// Send the record
self.send_to_partition(topic, partition, record).await
}
/// Add a partition to the current transaction.
///
/// Retries on coordinator errors with exponential backoff, re-discovering
/// the transaction coordinator between attempts.
async fn add_partition_to_txn(&self, topic: &str, partition: PartitionId) -> Result<()> {
let result = self.retry_with_coordinator("AddPartitionsToTxn", || async {
let (_coordinator_id, conn) = self.coordinator_connection().await?;
let (producer_id, producer_epoch) = self.checked_transactional_identity()?;
let apt_version = conn
.negotiate_api_version(
ApiKey::AddPartitionsToTxn,
versions::ADD_PARTITIONS_TO_TXN_MAX,
versions::ADD_PARTITIONS_TO_TXN_MIN,
)
.await
.ok_or_else(|| {
KrafkaError::protocol_kind(ProtocolErrorKind::UnknownApiVersion, "no mutually supported AddPartitionsToTxn API version")
})?;
let request = AddPartitionsToTxnRequest::new(
&self.config.transactional_id,
producer_id,
producer_epoch,
)
.add_partition(topic, partition);
let response_bytes = conn
.send_request(ApiKey::AddPartitionsToTxn, apt_version, |buf| {
request.encode_versioned(apt_version, buf)
})
.await?;
let mut buf = response_bytes;
let response = AddPartitionsToTxnResponse::decode_versioned(apt_version, &mut buf)?;
if !response.is_ok() {
for topic_result in &response.results {
for partition_result in &topic_result.partitions {
if !partition_result.error_code.is_ok() {
return Err(KrafkaError::broker(
partition_result.error_code,
format!("failed to add {}-{} to transaction", topic, partition),
));
}
}
}
// Fallback: is_ok() was false but no individual partition error found
// (e.g. the target partition is missing from the response).
return Err(KrafkaError::protocol_kind(
ProtocolErrorKind::Malformed,
format!(
"failed to add {}-{} to transaction: response indicated error but no per-partition error found",
topic, partition
),
));
}
debug!("Added partition {}-{} to transaction", topic, partition);
Ok(())
})
.await;
match result {
Err(error) if Self::is_unknown_producer_id_error(&error) => {
Err(self.mark_unknown_producer_id_abort_required("AddPartitionsToTxn"))
}
other => other,
}
}
/// Send a record to a specific partition.
///
/// Includes retry logic with exponential backoff for transient failures.
/// On `OutOfOrderSequenceNumber`, resets the partition sequence and rebuilds
/// the batch with a fresh sequence before retrying.
async fn send_to_partition(
&self,
topic: TopicHandle,
partition: PartitionId,
record: RoutedRecord,
) -> Result<RecordMetadata> {
let retry_policy = &self.retry_policy;
let max_retries = retry_policy.max_retries;
let (producer_id, producer_epoch) = self.checked_transactional_identity()?;
// Allocate the sequence number once — retries must resend the same
// sequence to maintain idempotent semantics.
let mut sequence = self.next_sequence(topic.as_ref(), partition).await?;
// Build the record batch and request once before entering the retry loop.
// If encoding fails, roll back the sequence so the next send attempt
// starts from the correct value rather than creating a gap.
let mut request = match self.build_produce_request(
topic.as_ref(),
partition,
&record,
producer_id,
producer_epoch,
sequence,
) {
Ok(req) => req,
Err(e) => {
let _ = self.identity.rollback_sequence(topic.as_ref(), partition);
return Err(e);
}
};
for attempt in 0..=max_retries {
// Re-acquire connection on each attempt (leader may have moved).
let send_result: Result<RecordMetadata> = async {
let conn = self
.metadata
.get_leader_connection(topic.as_ref(), partition)
.await?;
// Transactions require Produce v3+ (transactional_id field).
let mut version = conn
.negotiate_api_version(
ApiKey::Produce,
versions::PRODUCE_MAX,
versions::PRODUCE_MIN,
)
.await
.ok_or_else(|| {
KrafkaError::protocol_kind(
ProtocolErrorKind::UnknownApiVersion,
"no mutually supported Produce API version; \
transactional produce requires v3+",
)
})?;
// KIP-516: Produce v13+ sends topic UUIDs instead of names.
// Fill IDs from cache; fall back to v12 if any UUID is not yet known.
if version >= 13 && !super::fill_produce_topic_ids(&mut request, &self.metadata) {
version = 12;
}
let encoded_body = super::encode_and_validate_produce_request(
&self.config.client_id,
self.config.max_request_size,
version,
&request,
)?;
let response = conn
.send_request(ApiKey::Produce, version, |buf| {
buf.put_slice(&encoded_body);
Ok(())
})
.await?;
let mut buf = response;
let produce_response = ProduceResponse::decode_versioned(version, &mut buf)?;
for topic_response in &produce_response.responses {
for partition_response in &topic_response.partition_responses {
if partition_response.index == partition {
if !partition_response.error_code.is_ok() {
if is_fatal_transaction_error(partition_response.error_code) {
self.set_state(TransactionState::FatalError);
}
return Err(KrafkaError::broker(
partition_response.error_code,
format!("produce failed for {topic}-{partition}"),
));
}
self.identity
.acknowledge(topic.as_ref(), partition, sequence);
return Ok(RecordMetadata {
topic: topic.to_string(),
partition,
offset: partition_response.base_offset,
timestamp: partition_response.log_append_time_ms,
});
}
}
}
Err(KrafkaError::protocol_kind(
ProtocolErrorKind::Malformed,
"partition not found in response",
))
}
.await;
match send_result {
Ok(metadata) => return Ok(metadata),
Err(e) => {
if Self::is_unknown_producer_id_error(&e) {
return Err(
self.mark_unknown_producer_id_abort_required("transactional produce")
);
}
// OutOfOrderSequenceNumber means the broker's expected
// sequence diverged from ours. Reset local state and
// rebuild the batch with a fresh sequence before retrying.
if let KrafkaError::Broker { code, .. } = &e
&& *code == ErrorCode::OutOfOrderSequenceNumber
{
if attempt >= max_retries {
return Err(e);
}
warn!(
topic = %topic,
partition = partition,
"OutOfOrderSequenceNumber, resetting sequence and rebuilding batch"
);
self.identity.reset_sequence(topic.as_ref(), partition);
sequence = self.next_sequence(topic.as_ref(), partition).await?;
request = self.build_produce_request(
topic.as_ref(),
partition,
&record,
producer_id,
producer_epoch,
sequence,
)?;
tokio::time::sleep(retry_policy.calculate_backoff(attempt + 1)).await;
continue;
}
if !e.is_retriable() || attempt >= max_retries {
return Err(e);
}
debug!(
topic = %topic,
partition = partition,
attempt = attempt + 1,
"Transient error in txn send, retrying: {}",
e
);
if should_refresh_metadata_after_txn_send_error(&e)
&& let Err(refresh_err) = self
.metadata
.refresh_for_topics(Some(&[topic.as_ref()]))
.await
{
debug!(error = %refresh_err, "Metadata refresh failed during txn retry");
}
tokio::time::sleep(retry_policy.calculate_backoff(attempt + 1)).await;
}
}
}
Err(KrafkaError::protocol_kind(
ProtocolErrorKind::Malformed,
format!("transactional produce retry loop exhausted after {max_retries} retries"),
))
}
/// Build a produce request for a single record to a partition.
fn build_produce_request(
&self,
topic: &str,
partition: PartitionId,
record: &RoutedRecord,
producer_id: i64,
producer_epoch: i16,
sequence: i32,
) -> Result<ProduceRequest> {
let mut batch_builder = RecordBatchBuilder::new()
.compression(self.config.compression)
.producer(producer_id, producer_epoch, sequence)
.transactional(true);
if let Some(ts) = record.timestamp {
batch_builder = batch_builder.base_timestamp(ts);
}
batch_builder = record.append_to_batch_builder(batch_builder);
let batch = batch_builder.build();
let batch_bytes = batch.encode()?;
Ok(ProduceRequest {
transactional_id: Some(self.config.transactional_id.clone()),
acks: Acks::All.to_i16(),
timeout_ms: crate::util::duration_to_millis_i32(self.config.request_timeout),
topic_data: vec![ProduceTopicData {
name: topic.to_string(),
topic_id: None,
partition_data: vec![ProducePartitionData {
index: partition,
records: batch_bytes,
}],
}],
})
}
/// Send consumer offsets within the current transaction.
///
/// This allows atomic commit of consumed offsets along with produced messages.
/// The `AddOffsetsToTxn` RPC (sent to the transaction coordinator) is retried
/// on coordinator errors. The `TxnOffsetCommit` RPC (sent to the group
/// coordinator) is retried with group coordinator re-discovery on
/// coordinator and retriable errors.
/// Atomically commit consumer offsets as part of the current transaction
/// (exactly-once consume-transform-produce).
///
/// Each [`TopicPartitionOffset`] entry specifies a partition and the **next**
/// offset to consume (`last_consumed + 1`, matching `Consumer::position()`).
/// Calling this with the wrong offset by one permanently shifts the group.
///
/// This is a two-phase operation:
/// 1. `AddOffsetsToTxn` — registers the consumer group with the transaction coordinator.
/// 2. `TxnOffsetCommit` — commits the offsets via the group coordinator, atomically
/// with the current transaction.
pub async fn send_offsets_to_transaction(
&self,
offsets: &[TopicPartitionOffset],
group_id: &str,
) -> Result<()> {
let current = self.state();
if current != TransactionState::InTransaction {
return Err(KrafkaError::invalid_state(format!(
"cannot send offsets in state {:?}",
current
)));
}
self.ensure_transaction_can_continue("send offsets")?;
let (producer_id, producer_epoch) = self.checked_transactional_identity()?;
// Phase 1: AddOffsetsToTxn — sent to transaction coordinator, with retry.
let add_offsets_result = self
.retry_with_coordinator("AddOffsetsToTxn", || async {
let (_coordinator_id, conn) = self.coordinator_connection().await?;
let add_request = AddOffsetsToTxnRequest::new(
&self.config.transactional_id,
producer_id,
producer_epoch,
group_id,
);
let aot_version = conn
.negotiate_api_version(
ApiKey::AddOffsetsToTxn,
versions::ADD_OFFSETS_TO_TXN_MAX,
versions::ADD_OFFSETS_TO_TXN_MIN,
)
.await
.ok_or_else(|| {
KrafkaError::protocol_kind(
ProtocolErrorKind::UnknownApiVersion,
"no mutually supported AddOffsetsToTxn API version",
)
})?;
let response_bytes = conn
.send_request(ApiKey::AddOffsetsToTxn, aot_version, |buf| {
add_request.encode_versioned(aot_version, buf)
})
.await?;
let mut buf = response_bytes;
let add_response =
AddOffsetsToTxnResponse::decode_versioned(aot_version, &mut buf)?;
if !add_response.is_ok() {
return Err(KrafkaError::broker(
add_response.error_code,
"failed to add offsets to transaction",
));
}
Ok(())
})
.await;
match add_offsets_result {
Err(error) if Self::is_unknown_producer_id_error(&error) => {
return Err(self.mark_unknown_producer_id_abort_required("AddOffsetsToTxn"));
}
Err(error) => return Err(error),
Ok(()) => {}
}
// Phase 2: TxnOffsetCommit — sent to the group coordinator, with retry.
// The Java client re-discovers the group coordinator and re-enqueues
// on coordinator or retriable errors; we mirror that with a retry loop.
let mut commit_request = TxnOffsetCommitRequest::new(
&self.config.transactional_id,
group_id,
producer_id,
producer_epoch,
);
for tpo in offsets {
commit_request =
commit_request.add_offset(&tpo.topic, tpo.partition, tpo.next_offset, None);
}
let max_retries = self.retry_policy.max_retries;
for attempt in 0..=max_retries {
if attempt > 0 {
tokio::time::sleep(self.retry_policy.calculate_backoff(attempt)).await;
}
let result: Result<()> = async {
let (group_node_id, group_host, group_port) =
self.find_group_coordinator(group_id).await?;
let group_addr = format!("{group_host}:{group_port}");
let group_conn = self
.pool
.get_connection_by_id(group_node_id, &group_addr)
.await?;
let toc_version = group_conn
.negotiate_api_version(
ApiKey::TxnOffsetCommit,
versions::TXN_OFFSET_COMMIT_MAX,
versions::TXN_OFFSET_COMMIT_MIN,
)
.await
.ok_or_else(|| {
KrafkaError::protocol_kind(
ProtocolErrorKind::UnknownApiVersion,
"no mutually supported TxnOffsetCommit API version",
)
})?;
let response_bytes = group_conn
.send_request(ApiKey::TxnOffsetCommit, toc_version, |buf| {
commit_request.encode_versioned(toc_version, buf)
})
.await?;
let mut buf = response_bytes;
let commit_response =
TxnOffsetCommitResponse::decode_versioned(toc_version, &mut buf)?;
if !commit_response.is_ok() {
// Extract the first per-partition error for actionable diagnostics.
for topic_result in &commit_response.topics {
for part_result in &topic_result.partitions {
if !part_result.error_code.is_ok() {
return Err(KrafkaError::broker(
part_result.error_code,
format!(
"failed to commit offset for {}-{} in transaction",
topic_result.name, part_result.partition
),
));
}
}
}
// Fallback if is_ok was false but no individual error found
return Err(KrafkaError::protocol_kind(
ProtocolErrorKind::Malformed,
"failed to commit offsets in transaction",
));
}
Ok(())
}
.await;
if let Err(error) = &result
&& Self::is_unknown_producer_id_error(error)
{
return Err(self.mark_unknown_producer_id_abort_required("TxnOffsetCommit"));
}
match &result {
Ok(()) => {
debug!("Added offsets to transaction for group {}", group_id);
return Ok(());
}
Err(e) if Self::needs_coordinator_refresh(e) && attempt < max_retries => {
warn!(
attempt,
error = %e,
"TxnOffsetCommit group coordinator error, re-discovering and retrying"
);
}
Err(e) if e.is_retriable() && attempt < max_retries => {
warn!(
attempt,
error = %e,
"TxnOffsetCommit retriable error, retrying"
);
}
Err(_) => return result,
}
}
Err(KrafkaError::protocol_kind(
ProtocolErrorKind::Malformed,
format!("TxnOffsetCommit retry loop exhausted after {max_retries} retries"),
))
}
/// Find the group coordinator, returning (node_id, host, port).
async fn find_group_coordinator(&self, group_id: &str) -> Result<(i32, String, i32)> {
let brokers = self.metadata.brokers();
if brokers.is_empty() {
return Err(KrafkaError::protocol_kind(
ProtocolErrorKind::Malformed,
"no brokers available",
));
}
let broker = &brokers[0];
let conn = self
.pool
.get_connection_by_id(broker.id(), broker.address())
.await?;
let request = FindCoordinatorRequest::for_group(group_id);
// Negotiate FindCoordinator version — requires v1+ (MIN).
let fc_version = conn
.negotiate_api_version(
ApiKey::FindCoordinator,
versions::FIND_COORDINATOR_MAX,
versions::FIND_COORDINATOR_MIN,
)
.await
.ok_or_else(|| {
KrafkaError::protocol_kind(
ProtocolErrorKind::UnknownApiVersion,
"no mutually supported FindCoordinator API version",
)
})?;
let response_bytes = conn
.send_request(ApiKey::FindCoordinator, fc_version, |buf| {
request.encode_versioned(fc_version, buf)
})
.await?;
let mut buf = response_bytes;
let response = FindCoordinatorResponse::decode_versioned(fc_version, &mut buf)?;
if !response.error_code.is_ok() {
return Err(KrafkaError::broker(
response.error_code,
"failed to find group coordinator",
));
}
Ok((response.node_id, response.host, response.port))
}
/// Commit the current transaction.
pub async fn commit_transaction(&self) -> Result<()> {
self.ensure_transaction_can_continue("commit transaction")?;
// Atomic CAS: InTransaction → Committing
if let Err(actual) = self.try_transition(
TransactionState::InTransaction,
TransactionState::Committing,
) {
return Err(KrafkaError::invalid_state(format!(
"cannot commit in state {:?}",
actual
)));
}
let result = match self.end_transaction(true).await {
Err(error) if Self::is_unknown_producer_id_error(&error) => {
Err(self.mark_unknown_producer_id_abort_required("commit_transaction"))
}
other => other,
};
match &result {
Ok(()) => {
self.set_state(TransactionState::Ready);
self.txn_partitions.write().await.clear();
info!("Transaction committed");
}
Err(e) if Self::is_abortable_transaction_error(e) => {
match self.try_transition(
TransactionState::Committing,
TransactionState::InTransaction,
) {
Ok(()) => {
warn!("Transaction commit failed (abort required): {}", e);
}
Err(actual) => {
warn!(
"Transaction commit failed (abort required): {}; \
state is now {:?} (concurrent abort may be in progress)",
e, actual
);
}
}
}
Err(e) => {
if e.is_retriable() {
// Use CAS to safely revert Committing → InTransaction.
// If abort_transaction() raced and moved to Aborting,
// the CAS fails and we leave the state alone.
match self.try_transition(
TransactionState::Committing,
TransactionState::InTransaction,
) {
Ok(()) => {
warn!("Transaction commit failed (retriable): {}", e);
}
Err(actual) => {
warn!(
"Transaction commit failed (retriable): {}; \
state is now {:?} (concurrent abort may be in progress)",
e, actual
);
}
}
} else {
// Fatal error — caller must abort
self.set_state(TransactionState::FatalError);
warn!("Transaction commit failed (fatal): {}", e);
}
}
}
result
}
/// Abort the current transaction.
pub async fn abort_transaction(&self) -> Result<()> {
// Atomic CAS: try InTransaction → Aborting first, then Committing → Aborting
let transition = self
.try_transition(TransactionState::InTransaction, TransactionState::Aborting)
.or_else(|_| {
self.try_transition(TransactionState::Committing, TransactionState::Aborting)
});
if let Err(actual) = transition {
return Err(KrafkaError::invalid_state(format!(
"cannot abort in state {:?}",
actual
)));
}
let needs_reinitialize = self.abort_required.swap(false, Ordering::SeqCst);
let result = if needs_reinitialize {
match self.end_transaction(false).await {
Ok(()) => self.do_init_transactions().await,
Err(error) if Self::is_unknown_producer_id_error(&error) => {
debug!(
"Abort observed UnknownProducerId after transactional error; reinitializing producer identity"
);
self.do_init_transactions().await
}
Err(error) => Err(error),
}
} else {
self.end_transaction(false).await
};
match &result {
Ok(()) => {
self.set_state(TransactionState::Ready);
self.txn_partitions.write().await.clear();
info!("Transaction aborted");
}
Err(_) => {
self.set_state(TransactionState::FatalError);
warn!("Transaction abort failed, producer is now in fatal error state");
}
}
result
}
/// End the transaction (commit or abort).
///
/// Retries on coordinator errors with exponential backoff, re-discovering
/// the transaction coordinator between attempts.
async fn end_transaction(&self, commit: bool) -> Result<()> {
self.retry_with_coordinator("EndTxn", || async {
let (_coordinator_id, conn) = self.coordinator_connection().await?;
let (producer_id, producer_epoch) = self.checked_transactional_identity()?;
let et_version = conn
.negotiate_api_version(ApiKey::EndTxn, versions::END_TXN_MAX, versions::END_TXN_MIN)
.await
.ok_or_else(|| {
KrafkaError::protocol_kind(
ProtocolErrorKind::UnknownApiVersion,
"no mutually supported EndTxn API version",
)
})?;
let request = if commit {
EndTxnRequest::commit(&self.config.transactional_id, producer_id, producer_epoch)
} else {
EndTxnRequest::abort(&self.config.transactional_id, producer_id, producer_epoch)
};
let response_bytes = conn
.send_request(ApiKey::EndTxn, et_version, |buf| {
request.encode_versioned(et_version, buf)
})
.await?;
let mut buf = response_bytes;
let response = EndTxnResponse::decode_versioned(et_version, &mut buf)?;
if !response.is_ok() {
return Err(KrafkaError::broker(
response.error_code,
if commit {
"failed to commit transaction"
} else {
"failed to abort transaction"
},
));
}
// KIP-890 (Kafka 3.9+): the broker returns the bumped PID and epoch
// in EndTxn v4+ responses. Apply the new identity so subsequent
// AddPartitionsToTxn requests use the correct epoch.
if let (Some(pid), Some(epoch)) = (response.producer_id, response.producer_epoch)
&& pid >= 0
&& epoch >= 0
{
debug!(
pid,
epoch, "KIP-890: applying broker-bumped epoch from EndTxn response"
);
self.identity.initialize(pid, epoch);
}
Ok(())
})
.await
}
/// Get the transactional ID.
#[inline]
pub fn transactional_id(&self) -> &str {
&self.config.transactional_id
}
/// Get the producer ID (once initialized).
#[inline]
pub fn producer_id(&self) -> i64 {
self.identity.producer_id()
}
/// Get the producer epoch (once initialized).
#[inline]
pub fn producer_epoch(&self) -> i16 {
self.identity.producer_epoch()
}
/// Get the next sequence number for a topic-partition.
async fn next_sequence(&self, topic: &str, partition: PartitionId) -> Result<i32> {
self.identity.next_sequence(topic, partition)
}
/// Close the transactional producer and release all resources.
///
/// If a transaction is in progress, it will be aborted before closing.
/// After calling `close()`, the producer cannot be used again.
/// Calling `close()` more than once is a no-op.
pub async fn close(&self) {
let _ = self.close_inner(None).await;
}
/// Close the transactional producer, giving up on graceful shutdown once
/// `timeout` expires.
///
/// On timeout, the connection pool is still torn down, causing any
/// remaining in-flight operations to fail fast.
pub async fn close_with_timeout(&self, timeout: Duration) -> Result<()> {
self.close_inner(Some(timeout)).await
}
async fn close_inner(&self, timeout: Option<Duration>) -> Result<()> {
let Some(target) = self.in_flight_barrier.begin_close() else {
return Ok(());
};
let graceful_close = async {
// Let already-started sends cross the ack boundary before aborting the
// active transaction or tearing down sockets.
self.in_flight_barrier.wait_for(target).await;
// If in-transaction, abort first to clean up broker state.
let current = self.state();
if current == TransactionState::InTransaction {
warn!("Closing transactional producer with active transaction — aborting");
self.abort_transaction().await?;
}
Ok::<(), KrafkaError>(())
};
let close_result = if let Some(timeout) = timeout {
tokio::time::timeout(timeout, graceful_close)
.await
.map_err(|_| KrafkaError::timeout("transactional producer close"))?
} else {
graceful_close.await
};
// Set state to prevent further use
self.set_state(TransactionState::FatalError);
// Close all connections in the pool
self.pool.close_all().await;
info!(
"TransactionalProducer closed: txn.id()={}",
self.config.transactional_id
);
close_result
}
/// Check if the transactional producer has been explicitly closed.
///
/// Returns `true` only when [`Self::close`] has been called. A producer in
/// [`TransactionState::FatalError`] due to a broker error is *not*
/// considered closed — use [`Self::state`] to check for fatal errors.
#[inline]
pub fn is_closed(&self) -> bool {
self.in_flight_barrier.is_closing()
}
}
/// Check if an error code is a fatal transaction error.
fn is_fatal_transaction_error(error_code: ErrorCode) -> bool {
matches!(
error_code,
ErrorCode::InvalidProducerEpoch
| ErrorCode::ProducerFenced
| ErrorCode::TransactionalIdAuthorizationFailed
| ErrorCode::InvalidTxnState
| ErrorCode::TransactionCoordinatorFenced
)
}
fn should_refresh_metadata_after_txn_send_error(error: &KrafkaError) -> bool {
error.is_retriable()
&& !matches!(
error,
KrafkaError::Broker {
code: ErrorCode::OutOfOrderSequenceNumber,
..
}
)
}
/// Builder for TransactionalProducer.
#[derive(Default)]
#[must_use = "builders do nothing until .build() is called"]
pub struct TransactionalProducerBuilder {
config: TransactionalProducerConfig,
retry_policy: RetryPolicy,
partitioner: Option<Arc<dyn Partitioner>>,
key_encoder: Option<Arc<dyn SchemaEncoder>>,
value_encoder: Option<Arc<dyn SchemaEncoder>>,
}
impl TransactionalProducerBuilder {
/// Set bootstrap servers.
pub fn bootstrap_servers(mut self, servers: impl Into<String>) -> Self {
self.config.bootstrap_servers = servers.into();
self
}
/// Set client ID.
pub fn client_id(mut self, client_id: impl Into<String>) -> Self {
self.config.client_id = client_id.into();
self
}
/// Set the transactional ID (required).
pub fn transactional_id(mut self, txn_id: impl Into<String>) -> Self {
self.config.transactional_id = txn_id.into();
self
}
/// Set the transaction timeout in milliseconds.
pub fn transaction_timeout_ms(mut self, timeout: i32) -> Self {
self.config.transaction_timeout_ms = timeout;
self
}
/// Set request timeout.
pub fn request_timeout(mut self, timeout: Duration) -> Self {
self.config.request_timeout = timeout;
self
}
/// Set the maximum encoded Kafka request frame size in bytes.
pub fn max_request_size(mut self, bytes: usize) -> Self {
self.config.max_request_size = bytes;
self
}
/// Set compression.
pub fn compression(mut self, compression: Compression) -> Self {
self.config.compression = compression;
self
}
/// Set a custom partitioner.
///
/// If not set, [`DefaultPartitioner`] is used, which applies murmur2 hashing
/// for keyed messages and round-robin for unkeyed messages.
pub fn partitioner(mut self, partitioner: impl Partitioner + 'static) -> Self {
self.partitioner = Some(Arc::new(partitioner));
self
}
/// Set authentication configuration.
pub fn auth(mut self, auth: AuthConfig) -> Self {
self.config.auth = Some(auth);
self
}
/// Set SOCKS5 proxy configuration.
///
/// Routes all broker connections through the specified SOCKS5 proxy.
#[cfg(feature = "socks5")]
pub fn proxy(mut self, proxy: crate::network::ProxyConfig) -> Self {
self.config.proxy = Some(proxy);
self
}
/// Configure SASL/PLAIN authentication.
pub fn sasl_plain(mut self, username: &str, password: &str) -> crate::Result<Self> {
self.config.auth = Some(AuthConfig::sasl_plain(username, password)?);
Ok(self)
}
/// Configure SASL/SCRAM-SHA-256 authentication.
pub fn sasl_scram_sha256(mut self, username: &str, password: &str) -> Self {
self.config.auth = Some(AuthConfig::sasl_scram_sha256(username, password));
self
}
/// Configure SASL/SCRAM-SHA-512 authentication.
pub fn sasl_scram_sha512(mut self, username: &str, password: &str) -> Self {
self.config.auth = Some(AuthConfig::sasl_scram_sha512(username, password));
self
}
/// Attach a key encoder applied automatically on every [`send_record`](TransactionalProducer::send_record) call.
///
/// Equivalent to `key.serializer` in the Java `KafkaProducer`. Configure
/// it once here and encoding is transparent on every send.
pub fn key_encoder(mut self, encoder: Arc<dyn SchemaEncoder>) -> Self {
self.key_encoder = Some(encoder);
self
}
/// Attach a value encoder applied automatically on every [`send_record`](TransactionalProducer::send_record) call.
///
/// Equivalent to `value.serializer` in the Java `KafkaProducer`.
pub fn value_encoder(mut self, encoder: Arc<dyn SchemaEncoder>) -> Self {
self.value_encoder = Some(encoder);
self
}
/// Set the maximum number of retries for retriable errors.
///
/// Default: 3.
pub fn retries(mut self, retries: u32) -> Self {
self.retry_policy = self.retry_policy.with_max_retries(retries);
self
}
/// Set the initial retry backoff duration.
///
/// Used as the base interval for exponential back-off between retries.
/// Default: 100 ms.
pub fn retry_backoff(mut self, backoff: Duration) -> Self {
self.retry_policy = self.retry_policy.with_initial_backoff(backoff);
self
}
/// Build the transactional producer.
pub async fn build(self) -> Result<TransactionalProducer> {
if self.config.bootstrap_servers.is_empty() {
return Err(KrafkaError::config("bootstrap.servers is required"));
}
if self.config.transactional_id.is_empty() {
return Err(KrafkaError::config("transactional_id is required"));
}
// Validate against Kafka's KafkaString wire limit (i16::MAX bytes).
const MAX_KAFKA_STRING_LEN: usize = i16::MAX as usize;
if self.config.transactional_id.len() > MAX_KAFKA_STRING_LEN {
return Err(KrafkaError::config(format!(
"transactional_id is {} bytes, exceeding the Kafka wire limit of {MAX_KAFKA_STRING_LEN}",
self.config.transactional_id.len()
)));
}
if self.config.client_id.len() > MAX_KAFKA_STRING_LEN {
return Err(KrafkaError::config(format!(
"client_id is {} bytes, exceeding the Kafka wire limit of {MAX_KAFKA_STRING_LEN}",
self.config.client_id.len()
)));
}
if self.config.transaction_timeout_ms <= 0 {
return Err(KrafkaError::config("transaction_timeout_ms must be > 0"));
}
if self.config.max_request_size == 0 {
return Err(KrafkaError::config("max_request_size must be >= 1"));
}
let mut pool_config_builder = ConnectionConfig::builder()
.client_id(&self.config.client_id)
.request_timeout(self.config.request_timeout);
if let Some(ref auth) = self.config.auth {
pool_config_builder = pool_config_builder.auth(auth.clone());
}
#[cfg(feature = "socks5")]
if let Some(ref proxy) = self.config.proxy {
pool_config_builder = pool_config_builder.proxy(proxy.clone());
}
let mut pool_config = pool_config_builder.build()?;
pool_config.init_tls().await?;
let pool = Arc::new(ConnectionPool::new(pool_config));
pool.start_idle_evictor();
let bootstrap_servers =
crate::util::parse_bootstrap_servers(&self.config.bootstrap_servers)?;
let metadata = Arc::new(ClusterMetadata::new(
bootstrap_servers,
pool.clone(),
self.config.metadata_max_age,
));
metadata.refresh().await?;
info!(
"TransactionalProducer created with transactional.id()={}",
self.config.transactional_id
);
Ok(TransactionalProducer {
config: self.config,
metadata,
pool,
partitioner: self
.partitioner
.unwrap_or_else(|| Arc::new(DefaultPartitioner::new())),
state: AtomicU8::new(TransactionState::Uninitialized as u8),
abort_required: AtomicBool::new(false),
coordinator_id: RwLock::new(None),
txn_partitions: Arc::new(RwLock::new(TransactionPartitions::default())),
identity: ProducerIdentity::new(),
retry_policy: self.retry_policy,
in_flight_barrier: Arc::new(InFlightBarrier::new()),
key_encoder: self.key_encoder,
value_encoder: self.value_encoder,
})
}
}
#[cfg(test)]
#[allow(clippy::unwrap_used, clippy::expect_used, clippy::panic)]
mod tests {
use super::*;
use crate::metadata::ClusterMetadata;
use crate::network::ConnectionPool;
#[test]
fn test_transaction_state() {
assert_eq!(TransactionState::from(0), TransactionState::Uninitialized);
assert_eq!(TransactionState::from(1), TransactionState::Ready);
assert_eq!(TransactionState::from(2), TransactionState::InTransaction);
assert_eq!(TransactionState::from(3), TransactionState::Committing);
assert_eq!(TransactionState::from(4), TransactionState::Aborting);
assert_eq!(TransactionState::from(5), TransactionState::FatalError);
assert_eq!(TransactionState::from(99), TransactionState::FatalError);
}
#[test]
fn test_transactional_producer_config_default() {
let config = TransactionalProducerConfig::default();
assert_eq!(config.client_id, "krafka-txn-producer");
assert_eq!(config.transaction_timeout_ms, 60000);
assert_eq!(config.max_request_size, crate::protocol::MAX_MESSAGE_SIZE);
}
#[test]
fn test_transaction_partitions() {
let mut partitions = TransactionPartitions::default();
assert!(partitions.is_empty());
// First add returns NeedAdd
let result = partitions.begin_add("topic1", 0);
let notify = match result {
BeginAddResult::NeedAdd(n) => n,
_ => panic!("expected NeedAdd"),
};
assert!(!partitions.is_empty());
// Same partition while Pending returns Wait
assert!(matches!(
partitions.begin_add("topic1", 0),
BeginAddResult::Wait(_)
));
// Confirm, then same partition returns AlreadyAdded
partitions.confirm_add("topic1", 0, ¬ify);
assert!(matches!(
partitions.begin_add("topic1", 0),
BeginAddResult::AlreadyAdded
));
// Different partition returns NeedAdd
assert!(matches!(
partitions.begin_add("topic1", 1),
BeginAddResult::NeedAdd(_)
));
partitions.clear();
assert!(partitions.is_empty());
}
#[test]
fn test_is_fatal_transaction_error() {
assert!(is_fatal_transaction_error(ErrorCode::InvalidProducerEpoch));
assert!(is_fatal_transaction_error(ErrorCode::ProducerFenced));
assert!(is_fatal_transaction_error(
ErrorCode::TransactionCoordinatorFenced
));
assert!(is_fatal_transaction_error(
ErrorCode::TransactionalIdAuthorizationFailed
));
assert!(is_fatal_transaction_error(ErrorCode::InvalidTxnState));
assert!(!is_fatal_transaction_error(ErrorCode::None));
assert!(!is_fatal_transaction_error(ErrorCode::UnknownServerError));
}
#[test]
fn test_needs_coordinator_refresh() {
// Coordinator-related broker errors → true
assert!(TransactionalProducer::needs_coordinator_refresh(
&KrafkaError::broker(ErrorCode::NotCoordinator, "test")
));
assert!(TransactionalProducer::needs_coordinator_refresh(
&KrafkaError::broker(ErrorCode::CoordinatorNotAvailable, "test")
));
assert!(TransactionalProducer::needs_coordinator_refresh(
&KrafkaError::broker(ErrorCode::CoordinatorLoadInProgress, "test")
));
// Network and timeout errors → true (coordinator may have moved)
assert!(TransactionalProducer::needs_coordinator_refresh(
&KrafkaError::network(std::io::Error::new(
std::io::ErrorKind::ConnectionRefused,
"refused"
))
));
assert!(TransactionalProducer::needs_coordinator_refresh(
&KrafkaError::timeout("test operation")
));
// Non-coordinator broker errors → false
assert!(!TransactionalProducer::needs_coordinator_refresh(
&KrafkaError::broker(ErrorCode::InvalidProducerEpoch, "test")
));
assert!(!TransactionalProducer::needs_coordinator_refresh(
&KrafkaError::broker(ErrorCode::TransactionCoordinatorFenced, "test")
));
// Other error types → false
assert!(!TransactionalProducer::needs_coordinator_refresh(
&KrafkaError::protocol_kind(ProtocolErrorKind::Other, "test")
));
assert!(!TransactionalProducer::needs_coordinator_refresh(
&KrafkaError::invalid_state("test")
));
}
#[test]
fn test_should_refresh_metadata_after_txn_send_error() {
assert!(!should_refresh_metadata_after_txn_send_error(
&KrafkaError::broker(ErrorCode::OutOfOrderSequenceNumber, "sequence mismatch")
));
assert!(should_refresh_metadata_after_txn_send_error(
&KrafkaError::broker(ErrorCode::LeaderNotAvailable, "leader moved")
));
assert!(should_refresh_metadata_after_txn_send_error(
&KrafkaError::timeout("produce")
));
assert!(!should_refresh_metadata_after_txn_send_error(
&KrafkaError::broker(ErrorCode::InvalidProducerEpoch, "fenced")
));
}
#[tokio::test]
async fn test_builder_missing_bootstrap() {
let result = TransactionalProducer::builder()
.transactional_id("my-txn")
.build()
.await;
assert!(result.is_err());
}
#[tokio::test]
async fn test_send_record_requires_initialized_transactional_identity() {
let pool = Arc::new(ConnectionPool::new(ConnectionConfig::default()));
let metadata = Arc::new(ClusterMetadata::new(
vec!["localhost:9092".to_string()],
pool.clone(),
Duration::from_secs(300),
));
let producer = TransactionalProducer {
config: TransactionalProducerConfig {
bootstrap_servers: "localhost:9092".to_string(),
transactional_id: "txn-test".to_string(),
..TransactionalProducerConfig::default()
},
metadata,
pool,
partitioner: Arc::new(DefaultPartitioner::new()),
state: AtomicU8::new(TransactionState::InTransaction as u8),
abort_required: AtomicBool::new(false),
coordinator_id: RwLock::new(None),
txn_partitions: Arc::new(RwLock::new(TransactionPartitions::default())),
identity: ProducerIdentity::new(),
retry_policy: RetryPolicy::default(),
in_flight_barrier: Arc::new(InFlightBarrier::new()),
key_encoder: None,
value_encoder: None,
};
let record = ProducerRecord::new("topic", Bytes::from_static(b"value")).with_partition(0);
let err = producer.send_record(record).await.unwrap_err();
assert!(
err.to_string()
.contains("transactional producer identity not initialized"),
"expected invalid identity guard, got: {err}"
);
}
#[tokio::test]
async fn test_builder_missing_txn_id() {
let result = TransactionalProducer::builder()
.bootstrap_servers("localhost:9092")
.build()
.await;
assert!(result.is_err());
}
#[test]
fn test_mark_unknown_producer_id_requires_abort() {
let pool = Arc::new(ConnectionPool::new(ConnectionConfig::default()));
let metadata = Arc::new(ClusterMetadata::new(
vec!["localhost:9092".to_string()],
pool.clone(),
Duration::from_secs(300),
));
let producer = TransactionalProducer {
config: TransactionalProducerConfig {
bootstrap_servers: "localhost:9092".to_string(),
transactional_id: "txn-test".to_string(),
..TransactionalProducerConfig::default()
},
metadata,
pool,
partitioner: Arc::new(DefaultPartitioner::new()),
state: AtomicU8::new(TransactionState::InTransaction as u8),
abort_required: AtomicBool::new(false),
coordinator_id: RwLock::new(None),
txn_partitions: Arc::new(RwLock::new(TransactionPartitions::default())),
identity: ProducerIdentity::new(),
retry_policy: RetryPolicy::default(),
in_flight_barrier: Arc::new(InFlightBarrier::new()),
key_encoder: None,
value_encoder: None,
};
let error = producer.mark_unknown_producer_id_abort_required("transactional produce");
assert!(matches!(
error,
KrafkaError::Broker {
code: ErrorCode::TransactionAbortable,
..
}
));
assert!(producer.abort_required());
let gate_error = producer
.ensure_transaction_can_continue("commit transaction")
.unwrap_err();
assert!(matches!(
gate_error,
KrafkaError::Broker {
code: ErrorCode::TransactionAbortable,
..
}
));
}
#[tokio::test]
async fn test_commit_transaction_rejects_abort_required() {
let pool = Arc::new(ConnectionPool::new(ConnectionConfig::default()));
let metadata = Arc::new(ClusterMetadata::new(
vec!["localhost:9092".to_string()],
pool.clone(),
Duration::from_secs(300),
));
let producer = TransactionalProducer {
config: TransactionalProducerConfig {
bootstrap_servers: "localhost:9092".to_string(),
transactional_id: "txn-test".to_string(),
..TransactionalProducerConfig::default()
},
metadata,
pool,
partitioner: Arc::new(DefaultPartitioner::new()),
state: AtomicU8::new(TransactionState::InTransaction as u8),
abort_required: AtomicBool::new(true),
coordinator_id: RwLock::new(None),
txn_partitions: Arc::new(RwLock::new(TransactionPartitions::default())),
identity: ProducerIdentity::new(),
retry_policy: RetryPolicy::default(),
in_flight_barrier: Arc::new(InFlightBarrier::new()),
key_encoder: None,
value_encoder: None,
};
let error = producer.commit_transaction().await.unwrap_err();
assert!(matches!(
error,
KrafkaError::Broker {
code: ErrorCode::TransactionAbortable,
..
}
));
assert_eq!(producer.state(), TransactionState::InTransaction);
}
#[test]
fn test_try_transition_success() {
let state = AtomicU8::new(TransactionState::Ready as u8);
let result = state.compare_exchange(
TransactionState::Ready as u8,
TransactionState::InTransaction as u8,
Ordering::SeqCst,
Ordering::SeqCst,
);
assert!(result.is_ok());
assert_eq!(
TransactionState::from(state.load(Ordering::SeqCst)),
TransactionState::InTransaction
);
}
#[test]
fn test_try_transition_failure() {
let state = AtomicU8::new(TransactionState::Uninitialized as u8);
let result = state.compare_exchange(
TransactionState::Ready as u8,
TransactionState::InTransaction as u8,
Ordering::SeqCst,
Ordering::SeqCst,
);
assert!(result.is_err());
// State should remain unchanged
assert_eq!(
TransactionState::from(state.load(Ordering::SeqCst)),
TransactionState::Uninitialized
);
}
#[test]
fn test_txn_builder_no_auth_by_default() {
let builder = TransactionalProducer::builder()
.bootstrap_servers("broker:9092")
.transactional_id("txn-1");
assert!(builder.config.auth.is_none());
}
#[test]
fn test_txn_builder_sets_max_request_size() {
let builder = TransactionalProducer::builder()
.bootstrap_servers("broker:9092")
.transactional_id("txn-1")
.max_request_size(65_536);
assert_eq!(builder.config.max_request_size, 65_536);
}
#[test]
fn test_txn_builder_sasl_plain() {
let builder = TransactionalProducer::builder()
.bootstrap_servers("broker:9093")
.transactional_id("txn-1")
.sasl_plain("user", "pass")
.unwrap();
let auth = builder.config.auth.as_ref().unwrap();
assert!(auth.requires_sasl());
assert!(auth.plain_credentials.is_some());
}
#[test]
fn test_txn_builder_sasl_scram_sha256() {
let builder = TransactionalProducer::builder()
.bootstrap_servers("broker:9093")
.transactional_id("txn-1")
.sasl_scram_sha256("user", "pass");
let auth = builder.config.auth.as_ref().unwrap();
assert!(auth.requires_sasl());
assert!(auth.scram_credentials.is_some());
}
#[test]
fn test_txn_builder_sasl_scram_sha512() {
let builder = TransactionalProducer::builder()
.bootstrap_servers("broker:9093")
.transactional_id("txn-1")
.sasl_scram_sha512("user", "pass");
let auth = builder.config.auth.as_ref().unwrap();
assert!(auth.requires_sasl());
assert!(auth.scram_credentials.is_some());
}
#[test]
fn test_txn_builder_auth_config() {
use crate::auth::AuthConfig;
let auth = AuthConfig::sasl_scram_sha256("admin", "secret");
let builder = TransactionalProducer::builder()
.bootstrap_servers("broker:9093")
.transactional_id("txn-1")
.auth(auth);
let auth = builder.config.auth.as_ref().unwrap();
assert!(auth.requires_sasl());
assert!(auth.scram_credentials.is_some());
}
#[test]
fn test_txn_builder_initializes_producer_identity() {
// Verify a built TransactionalProducer starts with uninitialized identity
// (pid=-1, epoch=-1 until init_transactions() is called)
let builder = TransactionalProducer::builder()
.bootstrap_servers("broker:9092")
.transactional_id("txn-test");
// The builder should have the transactional_id set
assert_eq!(builder.config.transactional_id, "txn-test");
}
#[test]
fn test_txn_builder_requires_transactional_id() {
let builder = TransactionalProducer::builder().bootstrap_servers("broker:9092");
// Without transactional_id, it defaults to empty string
assert!(builder.config.transactional_id.is_empty());
}
#[tokio::test]
async fn test_txn_builder_rejects_zero_timeout() {
let result = TransactionalProducer::builder()
.bootstrap_servers("localhost:9092")
.transactional_id("txn-1")
.transaction_timeout_ms(0)
.build()
.await;
match result {
Err(e) => assert!(e.to_string().contains("transaction_timeout_ms")),
Ok(_) => panic!("expected error for transaction_timeout_ms=0"),
}
}
#[tokio::test]
async fn test_txn_builder_rejects_zero_max_request_size() {
let result = TransactionalProducer::builder()
.bootstrap_servers("localhost:9092")
.transactional_id("txn-1")
.max_request_size(0)
.build()
.await;
match result {
Err(e) => assert!(e.to_string().contains("max_request_size")),
Ok(_) => panic!("expected error for max_request_size=0"),
}
}
#[tokio::test]
async fn test_txn_builder_rejects_negative_timeout() {
let result = TransactionalProducer::builder()
.bootstrap_servers("localhost:9092")
.transactional_id("txn-1")
.transaction_timeout_ms(-1)
.build()
.await;
assert!(result.is_err());
}
// ── R9.3: TransactionState::Initializing variant ──
#[test]
fn test_transaction_state_initializing_from_u8() {
assert_eq!(TransactionState::from(6), TransactionState::Initializing);
}
#[test]
fn test_transaction_state_initializing_value() {
assert_eq!(TransactionState::Initializing as u8, 6);
}
#[test]
fn test_transaction_state_initializing_round_trip() {
let state = TransactionState::Initializing;
let val = state as u8;
assert_eq!(TransactionState::from(val), TransactionState::Initializing);
}
#[test]
fn test_transaction_state_unknown_maps_to_fatal() {
// Values not explicitly mapped (except 5 which is FatalError) fall to FatalError
assert_eq!(TransactionState::from(7), TransactionState::FatalError);
assert_eq!(TransactionState::from(255), TransactionState::FatalError);
}
// ── R9.3: CAS transition with Initializing state ──
#[test]
fn test_try_transition_uninitialized_to_initializing() {
let state = AtomicU8::new(TransactionState::Uninitialized as u8);
let result = state.compare_exchange(
TransactionState::Uninitialized as u8,
TransactionState::Initializing as u8,
Ordering::SeqCst,
Ordering::SeqCst,
);
assert!(result.is_ok());
assert_eq!(
TransactionState::from(state.load(Ordering::SeqCst)),
TransactionState::Initializing
);
}
#[test]
fn test_try_transition_initializing_blocks_second_init() {
// Simulate: first call moved to Initializing, second call should fail
let state = AtomicU8::new(TransactionState::Initializing as u8);
let result = state.compare_exchange(
TransactionState::Uninitialized as u8,
TransactionState::Initializing as u8,
Ordering::SeqCst,
Ordering::SeqCst,
);
assert!(result.is_err());
// State stays Initializing
assert_eq!(
TransactionState::from(state.load(Ordering::SeqCst)),
TransactionState::Initializing
);
}
// ── R9.9: commit_transaction sets FatalError on non-retriable errors ──
#[test]
fn test_commit_fatal_error_state_machine() {
// Simulate the commit_transaction error-handling logic:
// On non-retriable error → state becomes FatalError
let state = AtomicU8::new(TransactionState::Committing as u8);
// Simulate a non-retriable error (e.g. InvalidProducerEpoch)
let error = KrafkaError::broker(ErrorCode::InvalidProducerEpoch, "epoch fenced");
assert!(!error.is_retriable());
// Apply the same logic as commit_transaction
if error.is_retriable() {
state.store(TransactionState::InTransaction as u8, Ordering::SeqCst);
} else {
state.store(TransactionState::FatalError as u8, Ordering::SeqCst);
}
assert_eq!(
TransactionState::from(state.load(Ordering::SeqCst)),
TransactionState::FatalError
);
}
#[test]
fn test_commit_retriable_error_reverts_to_in_transaction() {
// Simulate the commit_transaction error-handling logic:
// On retriable error → state reverts to InTransaction
let state = AtomicU8::new(TransactionState::Committing as u8);
let error = KrafkaError::broker(ErrorCode::CoordinatorNotAvailable, "coordinator down");
assert!(error.is_retriable());
if error.is_retriable() {
state.store(TransactionState::InTransaction as u8, Ordering::SeqCst);
} else {
state.store(TransactionState::FatalError as u8, Ordering::SeqCst);
}
assert_eq!(
TransactionState::from(state.load(Ordering::SeqCst)),
TransactionState::InTransaction
);
}
// ── R14: close() sets FatalError to prevent further use ──
#[test]
fn test_txn_close_sets_fatal_error_state() {
// Verify the close() contract: after close, state is FatalError
let state = AtomicU8::new(TransactionState::Ready as u8);
// Simulate close: set to FatalError
state.store(TransactionState::FatalError as u8, Ordering::SeqCst);
assert_eq!(
TransactionState::from(state.load(Ordering::SeqCst)),
TransactionState::FatalError
);
}
// ── R14: OutOfOrderSequenceNumber is retriable ──
#[test]
fn test_out_of_order_sequence_is_retriable() {
let error = KrafkaError::broker(ErrorCode::OutOfOrderSequenceNumber, "sequence mismatch");
assert!(error.is_retriable());
}
// ── R14: ProducerRecord timestamp propagation ──
#[test]
fn test_producer_record_with_timestamp() {
use crate::producer::ProducerRecord;
let record = ProducerRecord::new("topic", b"value".to_vec()).with_timestamp(1234567890);
assert_eq!(record.timestamp, Some(1234567890));
}
#[test]
fn test_transaction_partitions_state_machine() {
let mut tp = TransactionPartitions::default();
// First add returns NeedAdd
let result = tp.begin_add("topic", 0);
let notify = match result {
BeginAddResult::NeedAdd(n) => n,
_ => panic!("expected NeedAdd"),
};
// Concurrent add returns Wait
let result2 = tp.begin_add("topic", 0);
assert!(matches!(result2, BeginAddResult::Wait(_)));
// Confirm moves to Added
tp.confirm_add("topic", 0, ¬ify);
assert!(matches!(
tp.begin_add("topic", 0),
BeginAddResult::AlreadyAdded
));
// Different partition returns NeedAdd
let result3 = tp.begin_add("topic", 1);
let notify2 = match result3 {
BeginAddResult::NeedAdd(n) => n,
_ => panic!("expected NeedAdd"),
};
// Cancel removes — next call returns NeedAdd again
tp.cancel_add("topic", 1, ¬ify2);
assert!(matches!(
tp.begin_add("topic", 1),
BeginAddResult::NeedAdd(_)
));
// Clear empties everything
tp.clear();
assert!(tp.is_empty());
}
#[test]
fn test_transaction_partitions_fail_add_propagates_as_fatal() {
// Regression test for F-01/F-07: a non-retriable AddPartitionsToTxn
// failure must be stored as Failed so that any concurrent waiter
// receives Fatal immediately instead of making a redundant RPC or
// silently continuing with an unregistered partition.
let mut tp = TransactionPartitions::default();
// First caller gets NeedAdd and performs the RPC (which fails).
let notify = match tp.begin_add("t", 0) {
BeginAddResult::NeedAdd(n) => n,
other => panic!("expected NeedAdd, got {other:?}"),
};
// Second concurrent caller should be told to Wait.
assert!(matches!(tp.begin_add("t", 0), BeginAddResult::Wait(_)));
// RPC failed with a non-retriable error — store the sentinel.
let err = Arc::new(KrafkaError::invalid_state("fatal"));
tp.fail_add("t", 0, err.clone(), ¬ify);
// After fail_add, any new caller must get Fatal immediately.
assert!(
matches!(tp.begin_add("t", 0), BeginAddResult::Fatal(_)),
"expected Fatal after fail_add"
);
// The error stored in Failed is the same as the one passed in.
match tp.begin_add("t", 0) {
BeginAddResult::Fatal(stored) => {
assert_eq!(stored.to_string(), err.to_string());
}
other => panic!("expected Fatal, got {other:?}"),
}
}
#[test]
fn test_transactional_producer_is_send_sync() {
fn assert_send_sync<T: Send + Sync>() {}
assert_send_sync::<TransactionalProducer>();
}
}